From c8d3092208f73ee991a123052a71b7dbf7fffc27 Mon Sep 17 00:00:00 2001 From: Jimmy Hu Date: Mon, 3 Sep 2018 10:01:40 +0800 Subject: core: Add a new structure `Position` and move `ShardID`, `ChainID` and `Height` from `Block` (#89) --- core/agreement.go | 13 +- core/consensus.go | 16 ++- core/consensus_test.go | 52 +++++--- core/crypto.go | 19 +-- core/crypto_test.go | 8 +- core/interfaces.go | 2 +- core/leader-selector_test.go | 2 +- core/nonblocking-application.go | 4 +- core/nonblocking-application_test.go | 2 +- core/reliable-broadcast.go | 53 +++++---- core/reliable-broadcast_test.go | 126 +++++++++++++------- core/test/app.go | 2 +- core/test/blocks-generator.go | 6 +- core/test/blocks-generator_test.go | 8 +- core/total-ordering.go | 14 +-- core/total-ordering_test.go | 222 ++++++++++++++++++++++++----------- core/types/block.go | 14 +-- core/types/block_test.go | 20 ++-- core/types/position.go | 25 ++++ core/types/vote.go | 18 ++- 20 files changed, 411 insertions(+), 215 deletions(-) create mode 100644 core/types/position.go (limited to 'core') diff --git a/core/agreement.go b/core/agreement.go index 28bcc67..299971e 100644 --- a/core/agreement.go +++ b/core/agreement.go @@ -62,13 +62,6 @@ type agreementReceiver interface { confirmBlock(common.Hash) } -// position is the current round of the agreement. -type position struct { - ShardID uint64 - ChainID uint64 - Height uint64 -} - // agreementData is the data for agreementState. type agreementData struct { recv agreementReceiver @@ -146,12 +139,12 @@ func (a *agreement) clocks() int { } // agreementID returns the current agreementID. -func (a *agreement) agreementID() position { - return a.aID.Load().(position) +func (a *agreement) agreementID() types.Position { + return a.aID.Load().(types.Position) } // setAgreementID sets the current agreementID. -func (a *agreement) setAgreementID(ID position) { +func (a *agreement) setAgreementID(ID types.Position) { a.aID.Store(ID) } diff --git a/core/consensus.go b/core/consensus.go index 4d1a386..2618b54 100644 --- a/core/consensus.go +++ b/core/consensus.go @@ -122,7 +122,7 @@ func NewConsensus( func (con *Consensus) Run() { go con.processMsg(con.network.ReceiveChan()) - chainID := uint64(0) + chainID := uint32(0) hashes := make(common.Hashes, 0, len(con.gov.GetValidatorSet())) for vID := range con.gov.GetValidatorSet() { hashes = append(hashes, vID.Hash) @@ -130,14 +130,16 @@ func (con *Consensus) Run() { sort.Sort(hashes) for i, hash := range hashes { if hash == con.ID.Hash { - chainID = uint64(i) + chainID = uint32(i) break } } genesisBlock := &types.Block{ ProposerID: con.ID, - ChainID: chainID, + Position: types.Position{ + ChainID: chainID, + }, } if err := con.PrepareGenesisBlock(genesisBlock, time.Now().UTC()); err != nil { fmt.Println(err) @@ -156,7 +158,9 @@ ProposingBlockLoop: } block := &types.Block{ ProposerID: con.ID, - ChainID: chainID, + Position: types.Position{ + ChainID: chainID, + }, } if err := con.PrepareBlock(block, time.Now().UTC()); err != nil { fmt.Println(err) @@ -321,7 +325,7 @@ func (con *Consensus) PrepareBlock(b *types.Block, con.rbModule.prepareBlock(b) b.Timestamps[b.ProposerID] = proposeTime - b.Payloads = con.app.PreparePayloads(b.ShardID, b.ChainID, b.Height) + b.Payloads = con.app.PreparePayloads(b.Position) b.Hash, err = hashBlock(b) if err != nil { return @@ -343,7 +347,7 @@ func (con *Consensus) PrepareGenesisBlock(b *types.Block, err = ErrGenesisBlockNotEmpty return } - b.Height = 0 + b.Position.Height = 0 b.ParentHash = common.Hash{} b.Acks = make(map[common.Hash]struct{}) b.Timestamps = make(map[types.ValidatorID]time.Time) diff --git a/core/consensus_test.go b/core/consensus_test.go index 70ab5aa..aec0e4c 100644 --- a/core/consensus_test.go +++ b/core/consensus_test.go @@ -56,12 +56,14 @@ type ConsensusTestSuite struct { func (s *ConsensusTestSuite) prepareGenesisBlock( proposerID types.ValidatorID, - chainID uint64, + chainID uint32, con *Consensus) *types.Block { block := &types.Block{ ProposerID: proposerID, - ChainID: chainID, + Position: types.Position{ + ChainID: chainID, + }, } err := con.PrepareGenesisBlock(block, time.Now().UTC()) s.Require().Nil(err) @@ -140,7 +142,9 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() { time.Sleep(minInterval) b11 := &types.Block{ ProposerID: validators[1], - ChainID: 1, + Position: types.Position{ + ChainID: 1, + }, } b11.Hash, err = hashBlock(b11) s.Require().Nil(err) @@ -155,8 +159,10 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() { time.Sleep(minInterval) b01 := &types.Block{ ProposerID: validators[0], - ChainID: 0, - Hash: common.NewRandomHash(), + Position: types.Position{ + ChainID: 0, + }, + Hash: common.NewRandomHash(), } req.Nil(objs[validators[0]].con.PrepareBlock(b01, time.Now().UTC())) req.Len(b01.Acks, 4) @@ -165,8 +171,10 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() { time.Sleep(minInterval) b21 := &types.Block{ ProposerID: validators[2], - ChainID: 2, - Hash: common.NewRandomHash(), + Position: types.Position{ + ChainID: 2, + }, + Hash: common.NewRandomHash(), } req.Nil(objs[validators[2]].con.PrepareBlock(b21, time.Now().UTC())) req.Len(b21.Acks, 4) @@ -175,8 +183,10 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() { time.Sleep(minInterval) b31 := &types.Block{ ProposerID: validators[3], - ChainID: 3, - Hash: common.NewRandomHash(), + Position: types.Position{ + ChainID: 3, + }, + Hash: common.NewRandomHash(), } req.Nil(objs[validators[3]].con.PrepareBlock(b31, time.Now().UTC())) req.Len(b31.Acks, 4) @@ -190,8 +200,10 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() { time.Sleep(minInterval) b02 := &types.Block{ ProposerID: validators[0], - ChainID: 0, - Hash: common.NewRandomHash(), + Position: types.Position{ + ChainID: 0, + }, + Hash: common.NewRandomHash(), } req.Nil(objs[validators[0]].con.PrepareBlock(b02, time.Now().UTC())) req.Len(b02.Acks, 3) @@ -202,8 +214,10 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() { time.Sleep(minInterval) b12 := &types.Block{ ProposerID: validators[1], - ChainID: 1, - Hash: common.NewRandomHash(), + Position: types.Position{ + ChainID: 1, + }, + Hash: common.NewRandomHash(), } req.Nil(objs[validators[1]].con.PrepareBlock(b12, time.Now().UTC())) req.Len(b12.Acks, 4) @@ -215,8 +229,10 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() { time.Sleep(minInterval) b22 := &types.Block{ ProposerID: validators[2], - ChainID: 2, - Hash: common.NewRandomHash(), + Position: types.Position{ + ChainID: 2, + }, + Hash: common.NewRandomHash(), } req.Nil(objs[validators[2]].con.PrepareBlock(b22, time.Now().UTC())) req.Len(b22.Acks, 3) @@ -227,8 +243,10 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() { time.Sleep(minInterval) b32 := &types.Block{ ProposerID: validators[3], - ChainID: 3, - Hash: common.NewRandomHash(), + Position: types.Position{ + ChainID: 3, + }, + Hash: common.NewRandomHash(), } req.Nil(objs[validators[3]].con.PrepareBlock(b32, time.Now().UTC())) req.Len(b32.Acks, 3) diff --git a/core/crypto.go b/core/crypto.go index d6af360..f097703 100644 --- a/core/crypto.go +++ b/core/crypto.go @@ -50,7 +50,7 @@ func verifyNotarySignature(pubkey crypto.PublicKey, } func hashBlock(block *types.Block) (common.Hash, error) { - hashPosition := hashPosition(block.ShardID, block.ChainID, block.Height) + hashPosition := hashPosition(block.Position) // Handling Block.Acks. acks := make(common.Hashes, 0, len(block.Acks)) for ack := range block.Acks { @@ -103,10 +103,13 @@ func hashVote(vote *types.Vote) common.Hash { binaryPeriod := make([]byte, 8) binary.LittleEndian.PutUint64(binaryPeriod, vote.Period) + hashPosition := hashPosition(vote.Position) + hash := crypto.Keccak256Hash( vote.ProposerID.Hash[:], vote.BlockHash[:], binaryPeriod, + hashPosition[:], []byte{byte(vote.Type)}, ) return hash @@ -125,7 +128,7 @@ func verifyVoteSignature(vote *types.Vote, sigToPub SigToPubFn) (bool, error) { } func hashCRS(block *types.Block, crs common.Hash) common.Hash { - hashPos := hashPosition(block.ShardID, block.ChainID, block.Height) + hashPos := hashPosition(block.Position) return crypto.Keccak256Hash(crs[:], hashPos[:]) } @@ -142,15 +145,15 @@ func verifyCRSSignature(block *types.Block, crs common.Hash, sigToPub SigToPubFn return true, nil } -func hashPosition(shardID, chainID, height uint64) common.Hash { - binaryShardID := make([]byte, 8) - binary.LittleEndian.PutUint64(binaryShardID, shardID) +func hashPosition(position types.Position) common.Hash { + binaryShardID := make([]byte, 4) + binary.LittleEndian.PutUint32(binaryShardID, position.ShardID) - binaryChainID := make([]byte, 8) - binary.LittleEndian.PutUint64(binaryChainID, chainID) + binaryChainID := make([]byte, 4) + binary.LittleEndian.PutUint32(binaryChainID, position.ChainID) binaryHeight := make([]byte, 8) - binary.LittleEndian.PutUint64(binaryHeight, height) + binary.LittleEndian.PutUint64(binaryHeight, position.Height) return crypto.Keccak256Hash( binaryShardID, diff --git a/core/crypto_test.go b/core/crypto_test.go index f4013be..4fa0e1b 100644 --- a/core/crypto_test.go +++ b/core/crypto_test.go @@ -56,7 +56,9 @@ func (s *CryptoTestSuite) prepareBlock(prevBlock *types.Block) *types.Block { ParentHash: prevBlock.Hash, Acks: acks, Timestamps: timestamps, - Height: prevBlock.Height + 1, + Position: types.Position{ + Height: prevBlock.Position.Height + 1, + }, Notary: types.Notary{ ParentHash: parentHash, Timestamp: time.Now(), @@ -160,7 +162,7 @@ func (s *CryptoTestSuite) TestBlockSignature() { if !block.IsGenesis() { parentBlock, exist := blockMap[block.ParentHash] s.Require().True(exist) - s.True(parentBlock.Height == block.Height-1) + s.True(parentBlock.Position.Height == block.Position.Height-1) hash, err := hashBlock(parentBlock) s.Require().Nil(err) s.Equal(hash, block.ParentHash) @@ -205,7 +207,7 @@ func (s *CryptoTestSuite) TestCRSSignature() { block.CRSSignature, err = prv.Sign(hashCRS(block, crs)) s.Require().Nil(err) s.True(verifyCRSSignature(block, crs, eth.SigToPub)) - block.Height++ + block.Position.Height++ s.False(verifyCRSSignature(block, crs, eth.SigToPub)) } diff --git a/core/interfaces.go b/core/interfaces.go index 4376742..55df106 100644 --- a/core/interfaces.go +++ b/core/interfaces.go @@ -28,7 +28,7 @@ import ( // consensus core. type Application interface { // PreparePayload is called when consensus core is preparing a block. - PreparePayloads(shardID, chainID, height uint64) [][]byte + PreparePayloads(position types.Position) [][]byte // VerifyPayloads verifies if the payloads are valid. VerifyPayloads(payloads [][]byte) bool diff --git a/core/leader-selector_test.go b/core/leader-selector_test.go index f6028c0..e18dd49 100644 --- a/core/leader-selector_test.go +++ b/core/leader-selector_test.go @@ -114,7 +114,7 @@ func (s *LeaderSelectorTestSuite) TestPrepareBlock() { } s.Require().Nil(leader.prepareBlock(block, prv)) s.Nil(leader.processBlock(block)) - block.Height++ + block.Position.Height++ s.Error(ErrIncorrectCRSSignature, leader.processBlock(block)) } diff --git a/core/nonblocking-application.go b/core/nonblocking-application.go index 5d674dd..1e20fa4 100644 --- a/core/nonblocking-application.go +++ b/core/nonblocking-application.go @@ -122,8 +122,8 @@ func (app *nonBlockingApplication) wait() { // PreparePayloads cannot be non-blocking. func (app *nonBlockingApplication) PreparePayloads( - shardID, chainID, height uint64) [][]byte { - return app.app.PreparePayloads(shardID, chainID, height) + position types.Position) [][]byte { + return app.app.PreparePayloads(position) } // VerifyPayloads cannot be non-blocking. diff --git a/core/nonblocking-application_test.go b/core/nonblocking-application_test.go index 65c5700..1968445 100644 --- a/core/nonblocking-application_test.go +++ b/core/nonblocking-application_test.go @@ -47,7 +47,7 @@ func newSlowApp(sleep time.Duration) *slowApp { } } -func (app *slowApp) PreparePayloads(_, _, _ uint64) [][]byte { +func (app *slowApp) PreparePayloads(_ types.Position) [][]byte { return [][]byte{} } diff --git a/core/reliable-broadcast.go b/core/reliable-broadcast.go index 918bad9..6d5d15b 100644 --- a/core/reliable-broadcast.go +++ b/core/reliable-broadcast.go @@ -71,7 +71,7 @@ type rbcBlockInfo struct { block *types.Block receivedTime time.Time status blockStatus - ackedChain map[uint64]struct{} + ackedChain map[uint32]struct{} } // Errors for sanity check error. @@ -97,7 +97,7 @@ func newReliableBroadcast() *reliableBroadcast { func (rb *reliableBroadcast) sanityCheck(b *types.Block) error { // Check if the chain id is valid. - if b.ChainID >= uint64(len(rb.lattice)) { + if b.Position.ChainID >= uint32(len(rb.lattice)) { return ErrInvalidChainID } @@ -107,7 +107,8 @@ func (rb *reliableBroadcast) sanityCheck(b *types.Block) error { } // Check if it forks. - if bInLattice, exist := rb.lattice[b.ChainID].blocks[b.Height]; exist { + if bInLattice, exist := + rb.lattice[b.Position.ChainID].blocks[b.Position.Height]; exist { if b.Hash != bInLattice.Hash { return ErrForkBlock } @@ -115,12 +116,12 @@ func (rb *reliableBroadcast) sanityCheck(b *types.Block) error { } // Check non-genesis blocks if it acks its parent. - if b.Height > 0 { + if b.Position.Height > 0 { if _, exist := b.Acks[b.ParentHash]; !exist { return ErrNotAckParent } bParentStat, exists := rb.blockInfos[b.ParentHash] - if exists && bParentStat.block.Height != b.Height-1 { + if exists && bParentStat.block.Position.Height != b.Position.Height-1 { return ErrInvalidBlockHeight } } @@ -129,7 +130,8 @@ func (rb *reliableBroadcast) sanityCheck(b *types.Block) error { for hash := range b.Acks { if bAckStat, exist := rb.blockInfos[hash]; exist { bAck := bAckStat.block - if bAck.Height < rb.lattice[b.ChainID].nextAck[bAck.ChainID] { + if bAck.Position.Height < + rb.lattice[b.Position.ChainID].nextAck[bAck.Position.ChainID] { return ErrDoubleAck } } @@ -145,7 +147,8 @@ func (rb *reliableBroadcast) sanityCheck(b *types.Block) error { } */ - if bParent, exist := rb.lattice[b.ChainID].blocks[b.Height-1]; exist { + if bParent, exist := + rb.lattice[b.Position.ChainID].blocks[b.Position.Height-1]; exist { for hash := range b.Timestamps { if b.Timestamps[hash].Before(bParent.Timestamps[hash]) { return ErrInvalidTimestamp @@ -167,7 +170,8 @@ func (rb *reliableBroadcast) areAllAcksInLattice(b *types.Block) bool { } bAck := bAckStat.block - bAckInLattice, exist := rb.lattice[bAck.ChainID].blocks[bAck.Height] + bAckInLattice, exist := + rb.lattice[bAck.Position.ChainID].blocks[bAck.Position.Height] if !exist { return false } @@ -188,7 +192,7 @@ func (rb *reliableBroadcast) processBlock(block *types.Block) (err error) { rb.blockInfos[block.Hash] = &rbcBlockInfo{ block: block, receivedTime: time.Now().UTC(), - ackedChain: make(map[uint64]struct{}), + ackedChain: make(map[uint32]struct{}), } rb.receivedBlocks[block.Hash] = block @@ -219,31 +223,34 @@ func (rb *reliableBroadcast) processBlock(block *types.Block) (err error) { continue // TODO(mission): how to return for multiple errors? } - rb.lattice[b.ChainID].blocks[b.Height] = b + chainID := b.Position.ChainID + rb.lattice[chainID].blocks[b.Position.Height] = b delete(rb.receivedBlocks, b.Hash) for h := range b.Acks { bAckStat := rb.blockInfos[h] - // Update nextAck only when bAckStat.block.Height + 1 is greater. A - // block might ack blocks proposed by same validator with different - // height. - if rb.lattice[b.ChainID].nextAck[bAckStat.block.ChainID] < bAckStat.block.Height+1 { - rb.lattice[b.ChainID].nextAck[bAckStat.block.ChainID] = bAckStat.block.Height + 1 + // Update nextAck only when bAckStat.block.Position.Height + 1 + // is greater. A block might ack blocks proposed by same validator with + // different height. + if rb.lattice[chainID].nextAck[bAckStat.block.Position.ChainID] < + bAckStat.block.Position.Height+1 { + rb.lattice[chainID].nextAck[bAckStat.block.Position.ChainID] = + bAckStat.block.Position.Height + 1 } // Update ackedChain for each ack blocks and its parents. for { - if _, exist := bAckStat.ackedChain[b.ChainID]; exist { + if _, exist := bAckStat.ackedChain[chainID]; exist { break } if bAckStat.status > blockStatusInit { break } - bAckStat.ackedChain[b.ChainID] = struct{}{} + bAckStat.ackedChain[chainID] = struct{}{} // A block is strongly acked if it is acked by more than // 2 * (maximum number of byzatine validators) unique validators. if len(bAckStat.ackedChain) > 2*((len(rb.lattice)-1)/3) { blocksToAcked[bAckStat.block.Hash] = bAckStat.block } - if bAckStat.block.Height == 0 { + if bAckStat.block.Position.Height == 0 { break } bAckStat = rb.blockInfos[bAckStat.block.ParentHash] @@ -294,7 +301,7 @@ func (rb *reliableBroadcast) processBlock(block *types.Block) (err error) { break } if rb.blockInfos[b.Hash].status >= blockStatusOrdering { - delete(rb.lattice[vid].blocks, b.Height) + delete(rb.lattice[vid].blocks, b.Position.Height) delete(rb.blockInfos, b.Hash) } if min == 0 { @@ -355,7 +362,7 @@ func (rb *reliableBroadcast) extractBlocks() []*types.Block { // parent, these fields would be setup like a genesis block. func (rb *reliableBroadcast) prepareBlock(block *types.Block) { // Reset fields to make sure we got these information from parent block. - block.Height = 0 + block.Position.Height = 0 block.ParentHash = common.Hash{} // The helper function to accumulate timestamps. accumulateTimestamps := func( @@ -392,7 +399,7 @@ func (rb *reliableBroadcast) prepareBlock(block *types.Block) { // find height of the latest block for that validator. var ( curBlock *types.Block - nextHeight = rb.lattice[block.ChainID].nextAck[chainID] + nextHeight = rb.lattice[block.Position.ChainID].nextAck[chainID] ) for { @@ -408,9 +415,9 @@ func (rb *reliableBroadcast) prepareBlock(block *types.Block) { } acks[curBlock.Hash] = struct{}{} accumulateTimestamps(times, curBlock) - if uint64(chainID) == block.ChainID { + if uint32(chainID) == block.Position.ChainID { block.ParentHash = curBlock.Hash - block.Height = curBlock.Height + 1 + block.Position.Height = curBlock.Position.Height + 1 } } block.Timestamps = times diff --git a/core/reliable-broadcast_test.go b/core/reliable-broadcast_test.go index 0dcbb80..206b2fa 100644 --- a/core/reliable-broadcast_test.go +++ b/core/reliable-broadcast_test.go @@ -52,13 +52,15 @@ func (s *ReliableBroadcastTest) prepareGenesisBlock( b = &types.Block{ ProposerID: proposerID, ParentHash: common.Hash{}, - Height: 0, + Position: types.Position{ + Height: 0, + }, Acks: make(map[common.Hash]struct{}), Timestamps: genTimestamps(validatorIDs), } for i, vID := range validatorIDs { if proposerID == vID { - b.ChainID = uint64(i) + b.Position.ChainID = uint32(i) break } } @@ -114,8 +116,10 @@ func genTestCase1(s *ReliableBroadcastTest, rb *reliableBroadcast) []types.Valid ParentHash: h, Hash: common.NewRandomHash(), Timestamps: genTimestamps(vids), - Height: 1, - ChainID: 0, + Position: types.Position{ + ChainID: 0, + Height: 1, + }, Acks: map[common.Hash]struct{}{ h: struct{}{}, }, @@ -131,8 +135,10 @@ func genTestCase1(s *ReliableBroadcastTest, rb *reliableBroadcast) []types.Valid b = &types.Block{ ProposerID: vids[0], ParentHash: h, - Height: 2, - ChainID: 0, + Position: types.Position{ + ChainID: 0, + Height: 2, + }, Timestamps: genTimestamps(vids), Acks: map[common.Hash]struct{}{ h: struct{}{}, @@ -151,8 +157,10 @@ func genTestCase1(s *ReliableBroadcastTest, rb *reliableBroadcast) []types.Valid ParentHash: h, Hash: common.NewRandomHash(), Timestamps: genTimestamps(vids), - Height: 3, - ChainID: 0, + Position: types.Position{ + ChainID: 0, + Height: 3, + }, Acks: map[common.Hash]struct{}{ h: struct{}{}, }, @@ -169,8 +177,10 @@ func genTestCase1(s *ReliableBroadcastTest, rb *reliableBroadcast) []types.Valid ParentHash: h, Hash: common.NewRandomHash(), Timestamps: genTimestamps(vids), - Height: 1, - ChainID: 3, + Position: types.Position{ + ChainID: 3, + Height: 1, + }, Acks: map[common.Hash]struct{}{ h: struct{}{}, }, @@ -205,9 +215,11 @@ func (s *ReliableBroadcastTest) TestSanityCheck() { b = &types.Block{ ProposerID: vids[0], ParentHash: common.NewRandomHash(), - Height: 10, - ChainID: 0, - Acks: make(map[common.Hash]struct{}), + Position: types.Position{ + ChainID: 0, + Height: 10, + }, + Acks: make(map[common.Hash]struct{}), } b.Hash, err = hashBlock(b) s.Require().Nil(err) @@ -219,8 +231,10 @@ func (s *ReliableBroadcastTest) TestSanityCheck() { b = &types.Block{ ProposerID: vids[1], ParentHash: common.NewRandomHash(), - Height: 1, - ChainID: 1, + Position: types.Position{ + ChainID: 1, + Height: 1, + }, Acks: map[common.Hash]struct{}{ rb.lattice[2].blocks[0].Hash: struct{}{}, }, @@ -236,8 +250,10 @@ func (s *ReliableBroadcastTest) TestSanityCheck() { b = &types.Block{ ProposerID: vids[1], ParentHash: h, - Height: 2, - ChainID: 1, + Position: types.Position{ + ChainID: 1, + Height: 2, + }, Acks: map[common.Hash]struct{}{ h: struct{}{}, }, @@ -253,7 +269,9 @@ func (s *ReliableBroadcastTest) TestSanityCheck() { b = &types.Block{ ProposerID: types.ValidatorID{Hash: common.NewRandomHash()}, ParentHash: h, - Height: 1, + Position: types.Position{ + Height: 1, + }, Acks: map[common.Hash]struct{}{ h: struct{}{}, }, @@ -269,8 +287,10 @@ func (s *ReliableBroadcastTest) TestSanityCheck() { b = &types.Block{ ProposerID: vids[1], ParentHash: h, - Height: 1, - ChainID: 100, + Position: types.Position{ + ChainID: 100, + Height: 1, + }, Acks: map[common.Hash]struct{}{ h: struct{}{}, }, @@ -286,8 +306,10 @@ func (s *ReliableBroadcastTest) TestSanityCheck() { b = &types.Block{ ProposerID: vids[0], ParentHash: h, - Height: 1, - ChainID: 0, + Position: types.Position{ + ChainID: 0, + Height: 1, + }, Acks: map[common.Hash]struct{}{ h: struct{}{}, }, @@ -306,8 +328,10 @@ func (s *ReliableBroadcastTest) TestSanityCheck() { b = &types.Block{ ProposerID: vids[0], ParentHash: h, - Height: 4, - ChainID: 0, + Position: types.Position{ + ChainID: 0, + Height: 4, + }, Acks: map[common.Hash]struct{}{ h: struct{}{}, rb.lattice[1].blocks[0].Hash: struct{}{}, @@ -324,8 +348,10 @@ func (s *ReliableBroadcastTest) TestSanityCheck() { b = &types.Block{ ProposerID: vids[1], ParentHash: h, - Height: 1, - ChainID: 1, + Position: types.Position{ + ChainID: 1, + Height: 1, + }, Acks: map[common.Hash]struct{}{ h: struct{}{}, common.NewRandomHash(): struct{}{}, @@ -385,8 +411,10 @@ func (s *ReliableBroadcastTest) TestStrongAck() { ProposerID: vids[1], ParentHash: rb.lattice[1].blocks[0].Hash, Hash: common.NewRandomHash(), - Height: 1, - ChainID: 1, + Position: types.Position{ + ChainID: 1, + Height: 1, + }, Timestamps: genTimestamps(vids), Acks: map[common.Hash]struct{}{ rb.lattice[0].blocks[2].Hash: struct{}{}, @@ -409,8 +437,10 @@ func (s *ReliableBroadcastTest) TestStrongAck() { ProposerID: vids[2], ParentHash: rb.lattice[2].blocks[0].Hash, Hash: common.NewRandomHash(), - Height: 1, - ChainID: 2, + Position: types.Position{ + ChainID: 2, + Height: 1, + }, Timestamps: genTimestamps(vids), Acks: map[common.Hash]struct{}{ rb.lattice[0].blocks[2].Hash: struct{}{}, @@ -437,8 +467,10 @@ func (s *ReliableBroadcastTest) TestExtractBlocks() { ProposerID: vids[1], ParentHash: rb.lattice[1].blocks[0].Hash, Hash: common.NewRandomHash(), - Height: 1, - ChainID: 1, + Position: types.Position{ + ChainID: 1, + Height: 1, + }, Timestamps: genTimestamps(vids), Acks: map[common.Hash]struct{}{ rb.lattice[0].blocks[2].Hash: struct{}{}, @@ -456,8 +488,10 @@ func (s *ReliableBroadcastTest) TestExtractBlocks() { ProposerID: vids[2], ParentHash: rb.lattice[2].blocks[0].Hash, Hash: common.NewRandomHash(), - Height: 1, - ChainID: 2, + Position: types.Position{ + ChainID: 2, + Height: 1, + }, Timestamps: genTimestamps(vids), Acks: map[common.Hash]struct{}{ rb.lattice[0].blocks[2].Hash: struct{}{}, @@ -518,8 +552,10 @@ func (s *ReliableBroadcastTest) TestRandomIntensiveAcking() { b := &types.Block{ ProposerID: vid, ParentHash: parentHash, - Height: height, - ChainID: uint64(id), + Position: types.Position{ + ChainID: uint32(id), + Height: height, + }, Timestamps: genTimestamps(vids), Acks: acks, } @@ -639,7 +675,9 @@ func (s *ReliableBroadcastTest) TestPrepareBlock() { // prepareBlock. b11 := &types.Block{ ProposerID: validators[1], - ChainID: 1, + Position: types.Position{ + ChainID: 1, + }, } rb.prepareBlock(b11) var err error @@ -658,12 +696,14 @@ func (s *ReliableBroadcastTest) TestPrepareBlock() { req.Equal(b11.Timestamps[validators[3]], b30.Timestamps[b30.ProposerID].Add(time.Millisecond)) req.Equal(b11.ParentHash, b10.Hash) - req.Equal(b11.Height, uint64(1)) + req.Equal(b11.Position.Height, uint64(1)) s.Require().Nil(rb.processBlock(b11)) // Propose/Process a block based on collected info. b12 := &types.Block{ ProposerID: validators[1], - ChainID: 1, + Position: types.Position{ + ChainID: 1, + }, } rb.prepareBlock(b12) b12.Hash, err = hashBlock(b12) @@ -672,12 +712,14 @@ func (s *ReliableBroadcastTest) TestPrepareBlock() { req.Len(b12.Acks, 1) req.Contains(b12.Acks, b11.Hash) req.Equal(b12.ParentHash, b11.Hash) - req.Equal(b12.Height, uint64(2)) + req.Equal(b12.Position.Height, uint64(2)) // When calling with other validator ID, we should be able to // get 4 blocks to ack. b01 := &types.Block{ ProposerID: validators[0], - ChainID: 0, + Position: types.Position{ + ChainID: 0, + }, } rb.prepareBlock(b01) b01.Hash, err = hashBlock(b01) @@ -688,7 +730,7 @@ func (s *ReliableBroadcastTest) TestPrepareBlock() { req.Contains(b01.Acks, b20.Hash) req.Contains(b01.Acks, b30.Hash) req.Equal(b01.ParentHash, b00.Hash) - req.Equal(b01.Height, uint64(1)) + req.Equal(b01.Position.Height, uint64(1)) } func TestReliableBroadcast(t *testing.T) { diff --git a/core/test/app.go b/core/test/app.go index 3ed65f7..60b74ac 100644 --- a/core/test/app.go +++ b/core/test/app.go @@ -103,7 +103,7 @@ func NewApp() *App { } // PreparePayloads implements Application interface. -func (app *App) PreparePayloads(shardID, chainID, height uint64) [][]byte { +func (app *App) PreparePayloads(position types.Position) [][]byte { return [][]byte{} } diff --git a/core/test/blocks-generator.go b/core/test/blocks-generator.go index 92271f7..18e4995 100644 --- a/core/test/blocks-generator.go +++ b/core/test/blocks-generator.go @@ -158,14 +158,16 @@ func (vs *validatorSetStatus) proposeBlock( newBlock := &types.Block{ ProposerID: proposerID, ParentHash: parentHash, - Height: uint64(len(status.blocks)), + Position: types.Position{ + Height: uint64(len(status.blocks)), + }, Acks: acks, Timestamps: ts, // TODO(mission.liao): Generate timestamp. } for i, vID := range vs.validatorIDs { if vID == proposerID { - newBlock.ChainID = uint64(i) + newBlock.Position.ChainID = uint32(i) } } var err error diff --git a/core/test/blocks-generator_test.go b/core/test/blocks-generator_test.go index 5c85fbd..b144100 100644 --- a/core/test/blocks-generator_test.go +++ b/core/test/blocks-generator_test.go @@ -74,7 +74,7 @@ func (s *BlocksGeneratorTestCase) TestGenerate() { // Check genesis block. genesisBlock := blocks[0] s.Equal(genesisBlock.ParentHash, common.Hash{}) - s.Equal(genesisBlock.Height, uint64(0)) + s.Equal(genesisBlock.Position.Height, uint64(0)) s.Empty(genesisBlock.Acks) // Check normal blocks. @@ -90,14 +90,14 @@ func (s *BlocksGeneratorTestCase) TestGenerate() { prevAckingHeight, exists := lastAckingHeights[ackedBlock.ProposerID] if exists { - s.True(prevAckingHeight < ackedBlock.Height) + s.True(prevAckingHeight < ackedBlock.Position.Height) } - lastAckingHeights[ackedBlock.ProposerID] = ackedBlock.Height + lastAckingHeights[ackedBlock.ProposerID] = ackedBlock.Position.Height // Block Height should always incremental by 1. // // Because we iterate blocks slice from 1, // we need to add 1 to the index. - s.Equal(block.Height, uint64(index+1)) + s.Equal(block.Position.Height, uint64(index+1)) } s.True(parentAcked) } diff --git a/core/total-ordering.go b/core/total-ordering.go index 1edccdf..3c1c23e 100644 --- a/core/total-ordering.go +++ b/core/total-ordering.go @@ -300,10 +300,10 @@ func (v *totalOrderingCandidateInfo) addBlock( rec := v.ackedStatus[proposerIndex] if rec.count == 0 { - rec.minHeight = b.Height + rec.minHeight = b.Position.Height rec.count = 1 } else { - if b.Height < rec.minHeight { + if b.Position.Height < rec.minHeight { err = ErrNotValidDAG return } @@ -489,7 +489,7 @@ func (global *totalOrderingGlobalVector) addBlock( blocksFromProposer := global.blocks[proposerIndex] if len(blocksFromProposer) > 0 { lastBlock := blocksFromProposer[len(blocksFromProposer)-1] - if b.Height-lastBlock.Height != 1 { + if b.Position.Height-lastBlock.Position.Height != 1 { err = ErrNotValidDAG return } @@ -516,7 +516,7 @@ func (global *totalOrderingGlobalVector) updateCandidateInfo( continue } rec = info.ackedStatus[idx] - rec.minHeight = blocks[0].Height + rec.minHeight = blocks[0].Position.Height rec.count = uint64(len(blocks)) } global.cachedCandidateInfo = info @@ -529,7 +529,7 @@ func (global *totalOrderingGlobalVector) updateCandidateInfo( continue } rec = info.ackedStatus[idx] - rec.minHeight = blocks[0].Height + rec.minHeight = blocks[0].Position.Height rec.count = uint64(len(blocks)) } } @@ -716,7 +716,7 @@ func (to *totalOrdering) prepareCandidate( sort.Ints(to.allocatedCandidateSlotIndexes) info.ackedStatus[proposerIndex] = &totalOrderingHeightRecord{ - minHeight: candidate.Height, + minHeight: candidate.Position.Height, count: uint64(len(to.globalVector.blocks[proposerIndex])), } ackedsForCandidate, exists := to.acked[candidate.Hash] @@ -736,7 +736,7 @@ func (to *totalOrdering) prepareCandidate( // If this block acks this candidate, all newer blocks // from the same validator also 'indirect' acks it. rec = info.ackedStatus[idx] - rec.minHeight = b.Height + rec.minHeight = b.Position.Height rec.count = uint64(len(blocks) - i) break } diff --git a/core/total-ordering_test.go b/core/total-ordering_test.go index ae6675e..4ef6f97 100644 --- a/core/total-ordering_test.go +++ b/core/total-ordering_test.go @@ -40,8 +40,10 @@ func (s *TotalOrderingTestSuite) genGenesisBlock( ProposerID: vID, ParentHash: common.Hash{}, Hash: common.NewRandomHash(), - Height: 0, - Acks: acks, + Position: types.Position{ + Height: 0, + }, + Acks: acks, } } @@ -79,7 +81,9 @@ func (s *TotalOrderingTestSuite) TestBlockRelation() { ProposerID: vID, ParentHash: blockA.Hash, Hash: common.NewRandomHash(), - Height: 1, + Position: types.Position{ + Height: 1, + }, Acks: map[common.Hash]struct{}{ blockA.Hash: struct{}{}, }, @@ -88,7 +92,9 @@ func (s *TotalOrderingTestSuite) TestBlockRelation() { ProposerID: vID, ParentHash: blockB.Hash, Hash: common.NewRandomHash(), - Height: 2, + Position: types.Position{ + Height: 2, + }, Acks: map[common.Hash]struct{}{ blockB.Hash: struct{}{}, }, @@ -245,7 +251,9 @@ func (s *TotalOrderingTestSuite) TestCycleDetection() { ProposerID: validators[0], ParentHash: b00.Hash, Hash: common.NewRandomHash(), - Height: 1, + Position: types.Position{ + Height: 1, + }, Acks: map[common.Hash]struct{}{ b00.Hash: struct{}{}, }, @@ -254,7 +262,9 @@ func (s *TotalOrderingTestSuite) TestCycleDetection() { ProposerID: validators[0], ParentHash: b01.Hash, Hash: common.NewRandomHash(), - Height: 2, + Position: types.Position{ + Height: 2, + }, Acks: map[common.Hash]struct{}{ b01.Hash: struct{}{}, }, @@ -263,7 +273,9 @@ func (s *TotalOrderingTestSuite) TestCycleDetection() { ProposerID: validators[0], ParentHash: b02.Hash, Hash: cycledHash, - Height: 3, + Position: types.Position{ + Height: 3, + }, Acks: map[common.Hash]struct{}{ b02.Hash: struct{}{}, }, @@ -328,7 +340,9 @@ func (s *TotalOrderingTestSuite) TestEarlyDeliver() { ProposerID: b.ProposerID, ParentHash: b.Hash, Hash: common.NewRandomHash(), - Height: b.Height + 1, + Position: types.Position{ + Height: b.Position.Height + 1, + }, Acks: map[common.Hash]struct{}{ b.Hash: struct{}{}, }, @@ -365,7 +379,8 @@ func (s *TotalOrderingTestSuite) TestEarlyDeliver() { candidate := to.candidates[to.candidateIndexMapping[b00.Hash]] s.Require().NotNil(candidate) - s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, b00.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, + b00.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[0]].count, uint64(3)) s.checkNotDeliver(to, b10) @@ -382,13 +397,17 @@ func (s *TotalOrderingTestSuite) TestEarlyDeliver() { candidate = to.candidates[to.candidateIndexMapping[b00.Hash]] s.Require().NotNil(candidate) - s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, b00.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, + b00.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[0]].count, uint64(3)) - s.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, b10.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, + b10.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[1]].count, uint64(3)) - s.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, b20.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, + b20.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[2]].count, uint64(3)) - s.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, b30.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, + b30.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[3]].count, uint64(2)) blocks, early, err := to.processBlock(b32) @@ -403,25 +422,29 @@ func (s *TotalOrderingTestSuite) TestEarlyDeliver() { // Check b01. candidate = to.candidates[to.candidateIndexMapping[b01.Hash]] s.Require().NotNil(candidate) - s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, b01.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, + b01.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[0]].count, uint64(2)) // Check b10. candidate = to.candidates[to.candidateIndexMapping[b10.Hash]] s.Require().NotNil(candidate) - s.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, b10.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, + b10.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[1]].count, uint64(3)) // Check b20. candidate = to.candidates[to.candidateIndexMapping[b20.Hash]] s.Require().NotNil(candidate) - s.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, b20.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, + b20.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[2]].count, uint64(3)) // Check b30. candidate = to.candidates[to.candidateIndexMapping[b30.Hash]] s.Require().NotNil(candidate) - s.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, b30.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, + b30.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[3]].count, uint64(3)) // Make sure b00 doesn't exist in current working set: @@ -450,7 +473,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { ProposerID: validators[1], ParentHash: b10.Hash, Hash: common.NewRandomHash(), - Height: 1, + Position: types.Position{ + Height: 1, + }, Acks: map[common.Hash]struct{}{ b10.Hash: struct{}{}, b00.Hash: struct{}{}, @@ -460,7 +485,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { ProposerID: validators[0], ParentHash: b00.Hash, Hash: common.NewRandomHash(), - Height: 1, + Position: types.Position{ + Height: 1, + }, Acks: map[common.Hash]struct{}{ b00.Hash: struct{}{}, b11.Hash: struct{}{}, @@ -470,7 +497,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { ProposerID: validators[2], ParentHash: b20.Hash, Hash: common.NewRandomHash(), - Height: 1, + Position: types.Position{ + Height: 1, + }, Acks: map[common.Hash]struct{}{ b20.Hash: struct{}{}, b01.Hash: struct{}{}, @@ -480,7 +509,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { ProposerID: validators[3], ParentHash: b30.Hash, Hash: common.NewRandomHash(), - Height: 1, + Position: types.Position{ + Height: 1, + }, Acks: map[common.Hash]struct{}{ b30.Hash: struct{}{}, b21.Hash: struct{}{}, @@ -490,7 +521,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { ProposerID: validators[0], ParentHash: b01.Hash, Hash: common.NewRandomHash(), - Height: 2, + Position: types.Position{ + Height: 2, + }, Acks: map[common.Hash]struct{}{ b01.Hash: struct{}{}, b21.Hash: struct{}{}, @@ -500,7 +533,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { ProposerID: validators[1], ParentHash: b11.Hash, Hash: common.NewRandomHash(), - Height: 2, + Position: types.Position{ + Height: 2, + }, Acks: map[common.Hash]struct{}{ b11.Hash: struct{}{}, b21.Hash: struct{}{}, @@ -510,7 +545,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { ProposerID: validators[3], ParentHash: b31.Hash, Hash: common.NewRandomHash(), - Height: 2, + Position: types.Position{ + Height: 2, + }, Acks: map[common.Hash]struct{}{ b31.Hash: struct{}{}, }, @@ -519,7 +556,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { ProposerID: validators[2], ParentHash: b21.Hash, Hash: common.NewRandomHash(), - Height: 2, + Position: types.Position{ + Height: 2, + }, Acks: map[common.Hash]struct{}{ b21.Hash: struct{}{}, b32.Hash: struct{}{}, @@ -529,7 +568,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { ProposerID: validators[2], ParentHash: b22.Hash, Hash: common.NewRandomHash(), - Height: 3, + Position: types.Position{ + Height: 3, + }, Acks: map[common.Hash]struct{}{ b22.Hash: struct{}{}, }, @@ -538,7 +579,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { ProposerID: validators[0], ParentHash: b02.Hash, Hash: common.NewRandomHash(), - Height: 3, + Position: types.Position{ + Height: 3, + }, Acks: map[common.Hash]struct{}{ b02.Hash: struct{}{}, b22.Hash: struct{}{}, @@ -548,7 +591,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { ProposerID: validators[1], ParentHash: b12.Hash, Hash: common.NewRandomHash(), - Height: 3, + Position: types.Position{ + Height: 3, + }, Acks: map[common.Hash]struct{}{ b12.Hash: struct{}{}, b22.Hash: struct{}{}, @@ -558,7 +603,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { ProposerID: validators[1], ParentHash: b13.Hash, Hash: common.NewRandomHash(), - Height: 4, + Position: types.Position{ + Height: 4, + }, Acks: map[common.Hash]struct{}{ b13.Hash: struct{}{}, }, @@ -567,7 +614,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { ProposerID: validators[4], ParentHash: b40.Hash, Hash: common.NewRandomHash(), - Height: 1, + Position: types.Position{ + Height: 1, + }, Acks: map[common.Hash]struct{}{ b40.Hash: struct{}{}, }, @@ -576,7 +625,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { ProposerID: validators[4], ParentHash: b41.Hash, Hash: common.NewRandomHash(), - Height: 2, + Position: types.Position{ + Height: 2, + }, Acks: map[common.Hash]struct{}{ b41.Hash: struct{}{}, }, @@ -626,26 +677,34 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { candidate := to.candidates[to.candidateIndexMapping[b00.Hash]] s.Require().NotNil(candidate) s.NotContains(candidate.ackedStatus, validatorIndexes[4]) - s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, b00.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, + b00.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[0]].count, uint64(2)) - s.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, b11.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, + b11.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[1]].count, uint64(2)) - s.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, b21.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, + b21.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[2]].count, uint64(2)) - s.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, b31.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, + b31.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[3]].count, uint64(2)) // Check b10's height vector. candidate = to.candidates[to.candidateIndexMapping[b10.Hash]] s.Require().NotNil(candidate) s.NotContains(candidate.ackedStatus, validatorIndexes[4]) - s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, b01.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, + b01.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[0]].count, uint64(1)) - s.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, b10.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, + b10.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[1]].count, uint64(3)) - s.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, b20.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, + b20.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[2]].count, uint64(3)) - s.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, b30.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, + b30.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[3]].count, uint64(3)) // Check the first deliver. @@ -665,26 +724,34 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { candidate = to.candidates[to.candidateIndexMapping[b11.Hash]] s.Require().NotNil(candidate) s.NotContains(candidate.ackedStatus, validators[4]) - s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, b01.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, + b01.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[0]].count, uint64(2)) - s.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, b11.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, + b11.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[1]].count, uint64(2)) - s.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, b21.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, + b21.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[2]].count, uint64(2)) - s.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, b11.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, + b11.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[3]].count, uint64(2)) // Check b20's height vector. candidate = to.candidates[to.candidateIndexMapping[b20.Hash]] s.Require().NotNil(candidate) s.NotContains(candidate.ackedStatus, validators[4]) - s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, b02.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, + b02.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[0]].count, uint64(1)) - s.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, b12.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, + b12.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[1]].count, uint64(1)) - s.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, b20.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, + b20.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[2]].count, uint64(3)) - s.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, b30.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, + b30.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[3]].count, uint64(3)) s.checkNotDeliver(to, b13) @@ -710,25 +777,33 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { candidate = to.candidates[to.candidateIndexMapping[b01.Hash]] s.Require().NotNil(candidate) s.NotContains(candidate.ackedStatus, validatorIndexes[4]) - s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, b01.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, + b01.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[0]].count, uint64(3)) - s.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, b12.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, + b12.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[1]].count, uint64(3)) - s.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, b21.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, + b21.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[2]].count, uint64(2)) - s.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, b31.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, + b31.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[3]].count, uint64(2)) candidate = to.candidates[to.candidateIndexMapping[b30.Hash]] s.Require().NotNil(candidate) s.NotContains(candidate.ackedStatus, validatorIndexes[4]) - s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, b03.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, + b03.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[0]].count, uint64(1)) - s.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, b13.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, + b13.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[1]].count, uint64(2)) - s.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, b22.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, + b22.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[2]].count, uint64(1)) - s.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, b30.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, + b30.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[3]].count, uint64(3)) candidate = to.candidates[to.candidateIndexMapping[b40.Hash]] @@ -737,7 +812,8 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK2() { s.NotContains(candidate.ackedStatus, validatorIndexes[1]) s.NotContains(candidate.ackedStatus, validatorIndexes[2]) s.NotContains(candidate.ackedStatus, validatorIndexes[3]) - s.Equal(candidate.ackedStatus[validatorIndexes[4]].minHeight, b40.Height) + s.Equal(candidate.ackedStatus[validatorIndexes[4]].minHeight, + b40.Position.Height) s.Equal(candidate.ackedStatus[validatorIndexes[4]].count, uint64(3)) // Make 'Acking Node Set' contains blocks from all validators, @@ -789,7 +865,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK0() { ProposerID: validators[0], ParentHash: b00.Hash, Hash: common.NewRandomHash(), - Height: 1, + Position: types.Position{ + Height: 1, + }, Acks: map[common.Hash]struct{}{ b00.Hash: struct{}{}, b10.Hash: struct{}{}, @@ -799,7 +877,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK0() { ProposerID: validators[1], ParentHash: b10.Hash, Hash: common.NewRandomHash(), - Height: 1, + Position: types.Position{ + Height: 1, + }, Acks: map[common.Hash]struct{}{ b10.Hash: struct{}{}, b20.Hash: struct{}{}, @@ -809,7 +889,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK0() { ProposerID: validators[2], ParentHash: b20.Hash, Hash: common.NewRandomHash(), - Height: 1, + Position: types.Position{ + Height: 1, + }, Acks: map[common.Hash]struct{}{ b20.Hash: struct{}{}, }, @@ -818,7 +900,9 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK0() { ProposerID: validators[3], ParentHash: b30.Hash, Hash: common.NewRandomHash(), - Height: 1, + Position: types.Position{ + Height: 1, + }, Acks: map[common.Hash]struct{}{ b21.Hash: struct{}{}, b30.Hash: struct{}{}, @@ -840,23 +924,29 @@ func (s *TotalOrderingTestSuite) TestBasicCaseForK0() { // Check candidate status before delivering. candidate := to.candidates[to.candidateIndexMapping[b00.Hash]] req.NotNil(candidate) - req.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, b00.Height) + req.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, + b00.Position.Height) req.Equal(candidate.ackedStatus[validatorIndexes[0]].count, uint64(2)) candidate = to.candidates[to.candidateIndexMapping[b10.Hash]] req.NotNil(candidate) - req.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, b01.Height) + req.Equal(candidate.ackedStatus[validatorIndexes[0]].minHeight, + b01.Position.Height) req.Equal(candidate.ackedStatus[validatorIndexes[0]].count, uint64(1)) - req.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, b10.Height) + req.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, + b10.Position.Height) req.Equal(candidate.ackedStatus[validatorIndexes[1]].count, uint64(2)) candidate = to.candidates[to.candidateIndexMapping[b20.Hash]] req.NotNil(candidate) - req.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, b11.Height) + req.Equal(candidate.ackedStatus[validatorIndexes[1]].minHeight, + b11.Position.Height) req.Equal(candidate.ackedStatus[validatorIndexes[1]].count, uint64(1)) - req.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, b20.Height) + req.Equal(candidate.ackedStatus[validatorIndexes[2]].minHeight, + b20.Position.Height) req.Equal(candidate.ackedStatus[validatorIndexes[2]].count, uint64(2)) - req.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, b30.Height) + req.Equal(candidate.ackedStatus[validatorIndexes[3]].minHeight, + b30.Position.Height) req.Equal(candidate.ackedStatus[validatorIndexes[3]].count, uint64(2)) // This new block should trigger non-early deliver. diff --git a/core/types/block.go b/core/types/block.go index f13d868..ab2ef84 100644 --- a/core/types/block.go +++ b/core/types/block.go @@ -65,9 +65,7 @@ type Block struct { ProposerID ValidatorID `json:"proposer_id"` ParentHash common.Hash `json:"parent_hash"` Hash common.Hash `json:"hash"` - ShardID uint64 `json:"shard_id"` - ChainID uint64 `json:"chain_id"` - Height uint64 `json:"height"` + Position Position `json:"position"` Timestamps map[ValidatorID]time.Time `json:"timestamps"` Acks map[common.Hash]struct{} `json:"acks"` Payloads [][]byte `json:"payloads"` @@ -88,9 +86,9 @@ func (b *Block) Clone() (bcopy *Block) { bcopy.ProposerID = b.ProposerID bcopy.ParentHash = b.ParentHash bcopy.Hash = b.Hash - bcopy.ShardID = b.ShardID - bcopy.ChainID = b.ChainID - bcopy.Height = b.Height + bcopy.Position.ShardID = b.Position.ShardID + bcopy.Position.ChainID = b.Position.ChainID + bcopy.Position.Height = b.Position.Height bcopy.Signature = b.Signature.Clone() bcopy.CRSSignature = b.CRSSignature.Clone() bcopy.Notary.Timestamp = b.Notary.Timestamp @@ -118,7 +116,7 @@ func (b *Block) Clone() (bcopy *Block) { // IsGenesis checks if the block is a genesisBlock func (b *Block) IsGenesis() bool { - return b.Height == 0 && b.ParentHash == common.Hash{} + return b.Position.Height == 0 && b.ParentHash == common.Hash{} } // ByHash is the helper type for sorting slice of blocks by hash. @@ -144,7 +142,7 @@ func (b ByHeight) Len() int { } func (b ByHeight) Less(i int, j int) bool { - return b[i].Height < b[j].Height + return b[i].Position.Height < b[j].Position.Height } func (b ByHeight) Swap(i int, j int) { diff --git a/core/types/block_test.go b/core/types/block_test.go index 2087373..4763ad1 100644 --- a/core/types/block_test.go +++ b/core/types/block_test.go @@ -49,10 +49,10 @@ func (s *BlockTestSuite) TestSortByHash() { } func (s *BlockTestSuite) TestSortByHeight() { - b0 := &Block{Height: 0} - b1 := &Block{Height: 1} - b2 := &Block{Height: 2} - b3 := &Block{Height: 3} + b0 := &Block{Position: Position{Height: 0}} + b1 := &Block{Position: Position{Height: 1}} + b2 := &Block{Position: Position{Height: 2}} + b3 := &Block{Position: Position{Height: 3}} blocks := []*Block{b3, b2, b1, b0} sort.Sort(ByHeight(blocks)) @@ -64,17 +64,23 @@ func (s *BlockTestSuite) TestSortByHeight() { func (s *BlockTestSuite) TestGenesisBlock() { b0 := &Block{ - Height: 0, + Position: Position{ + Height: 0, + }, ParentHash: common.Hash{}, } s.True(b0.IsGenesis()) b1 := &Block{ - Height: 1, + Position: Position{ + Height: 1, + }, ParentHash: common.Hash{}, } s.False(b1.IsGenesis()) b2 := &Block{ - Height: 0, + Position: Position{ + Height: 0, + }, ParentHash: common.NewRandomHash(), } s.False(b2.IsGenesis()) diff --git a/core/types/position.go b/core/types/position.go new file mode 100644 index 0000000..a894c4b --- /dev/null +++ b/core/types/position.go @@ -0,0 +1,25 @@ +// Copyright 2018 The dexon-consensus-core Authors +// This file is part of the dexon-consensus-core library. +// +// The dexon-consensus-core library is free software: you can redistribute it +// and/or modify it under the terms of the GNU Lesser General Public License as +// published by the Free Software Foundation, either version 3 of the License, +// or (at your option) any later version. +// +// The dexon-consensus-core library is distributed in the hope that it will be +// useful, but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU Lesser +// General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the dexon-consensus-core library. If not, see +// . + +package types + +// Position describes the position in the block lattice of an entity. +type Position struct { + ShardID uint32 `json:"shard_id"` + ChainID uint32 `json:"chain_id"` + Height uint64 `json:"height"` +} diff --git a/core/types/vote.go b/core/types/vote.go index 1247f84..158c209 100644 --- a/core/types/vote.go +++ b/core/types/vote.go @@ -38,11 +38,12 @@ const ( // Vote is the vote structure defined in Crypto Shuffle Algorithm. type Vote struct { - ProposerID ValidatorID - Type VoteType - BlockHash common.Hash - Period uint64 - Signature crypto.Signature + ProposerID ValidatorID `json:"proposer_id"` + Type VoteType `json:"type"` + BlockHash common.Hash `json:"block_hash"` + Period uint64 `json:"period"` + Position Position `json:"position"` + Signature crypto.Signature `json:"signature"` } func (v *Vote) String() string { @@ -57,6 +58,11 @@ func (v *Vote) Clone() *Vote { Type: v.Type, BlockHash: v.BlockHash, Period: v.Period, - Signature: v.Signature.Clone(), + Position: Position{ + ShardID: v.Position.ShardID, + ChainID: v.Position.ChainID, + Height: v.Position.Height, + }, + Signature: v.Signature.Clone(), } } -- cgit v1.2.3