diff options
author | Wei-Ning Huang <w@dexon.org> | 2018-10-01 18:19:52 +0800 |
---|---|---|
committer | GitHub <noreply@github.com> | 2018-10-01 18:19:52 +0800 |
commit | 9c33b9dc8aa59d414a6697f1e2d036e5581860ee (patch) | |
tree | 74a4a77b5c28fa9a550fe1ba40b5b2efac0e7630 /core | |
parent | 9b78db7a4744176070bf968d50dbebb191b28b69 (diff) | |
download | tangerine-consensus-9c33b9dc8aa59d414a6697f1e2d036e5581860ee.tar tangerine-consensus-9c33b9dc8aa59d414a6697f1e2d036e5581860ee.tar.gz tangerine-consensus-9c33b9dc8aa59d414a6697f1e2d036e5581860ee.tar.bz2 tangerine-consensus-9c33b9dc8aa59d414a6697f1e2d036e5581860ee.tar.lz tangerine-consensus-9c33b9dc8aa59d414a6697f1e2d036e5581860ee.tar.xz tangerine-consensus-9c33b9dc8aa59d414a6697f1e2d036e5581860ee.tar.zst tangerine-consensus-9c33b9dc8aa59d414a6697f1e2d036e5581860ee.zip |
core: update data model to reflect new model (#157)
Update data model:
1) Remove witness ack.
2) Add round to block.
3) Update governance interface.
Diffstat (limited to 'core')
-rw-r--r-- | core/authenticator.go | 21 | ||||
-rw-r--r-- | core/authenticator_test.go | 3 | ||||
-rw-r--r-- | core/blocklattice.go | 1 | ||||
-rw-r--r-- | core/compaction-chain.go | 166 | ||||
-rw-r--r-- | core/compaction-chain_test.go | 65 | ||||
-rw-r--r-- | core/consensus.go | 66 | ||||
-rw-r--r-- | core/consensus_test.go | 4 | ||||
-rw-r--r-- | core/crypto.go | 28 | ||||
-rw-r--r-- | core/crypto_test.go | 60 | ||||
-rw-r--r-- | core/interfaces.go | 18 | ||||
-rw-r--r-- | core/nonblocking.go | 30 | ||||
-rw-r--r-- | core/nonblocking_test.go | 21 | ||||
-rw-r--r-- | core/shard.go | 7 | ||||
-rw-r--r-- | core/test/app.go | 41 | ||||
-rw-r--r-- | core/test/app_test.go | 5 | ||||
-rw-r--r-- | core/types/block.go | 11 | ||||
-rw-r--r-- | core/types/nodeset.go | 5 | ||||
-rw-r--r-- | core/types/position.go | 2 | ||||
-rw-r--r-- | core/types/vote.go | 1 | ||||
-rw-r--r-- | core/types/witness.go | 61 |
20 files changed, 45 insertions, 571 deletions
diff --git a/core/authenticator.go b/core/authenticator.go index 91477c5..de22c1f 100644 --- a/core/authenticator.go +++ b/core/authenticator.go @@ -69,27 +69,6 @@ func (au *Authenticator) SignCRS(b *types.Block, crs common.Hash) (err error) { return } -// SignAsWitnessAck create a witness ack from a confirmed block. -func (au *Authenticator) SignAsWitnessAck( - b *types.Block) (wAck *types.WitnessAck, err error) { - - hash, err := hashWitness(b) - if err != nil { - return - } - sig, err := au.prvKey.Sign(hash) - if err != nil { - return - } - wAck = &types.WitnessAck{ - ProposerID: au.proposerID, - WitnessBlockHash: b.Hash, - Signature: sig, - Hash: hash, - } - return -} - // SignDKGComplaint signs a DKG complaint. func (au *Authenticator) SignDKGComplaint( complaint *types.DKGComplaint) (err error) { diff --git a/core/authenticator_test.go b/core/authenticator_test.go index b6c08d7..074d521 100644 --- a/core/authenticator_test.go +++ b/core/authenticator_test.go @@ -42,7 +42,6 @@ func (s *AuthenticatorTestSuite) TestBlock() { b := &types.Block{ ParentHash: common.NewRandomHash(), Position: types.Position{ - ShardID: 1, ChainID: 2, Height: 3, }, @@ -60,7 +59,6 @@ func (s *AuthenticatorTestSuite) TestVote() { BlockHash: common.NewRandomHash(), Period: 123, Position: types.Position{ - ShardID: 2, ChainID: 4, Height: 6, }} @@ -75,7 +73,6 @@ func (s *AuthenticatorTestSuite) TestCRS() { b := &types.Block{ ParentHash: common.NewRandomHash(), Position: types.Position{ - ShardID: 7, ChainID: 8, Height: 9, }, diff --git a/core/blocklattice.go b/core/blocklattice.go index 4dc43a4..3fa0736 100644 --- a/core/blocklattice.go +++ b/core/blocklattice.go @@ -134,7 +134,6 @@ func (s *chainStatus) purge() (purged common.Hashes) { // nextPosition returns a valid position for new block in this chain. func (s *chainStatus) nextPosition(shardID uint32) types.Position { return types.Position{ - ShardID: shardID, ChainID: s.ID, Height: s.minHeight + uint64(len(s.blocks)), } diff --git a/core/compaction-chain.go b/core/compaction-chain.go index b8f41e4..21b8412 100644 --- a/core/compaction-chain.go +++ b/core/compaction-chain.go @@ -20,83 +20,46 @@ package core import ( - "fmt" "sync" "time" "github.com/dexon-foundation/dexon-consensus-core/common" "github.com/dexon-foundation/dexon-consensus-core/core/blockdb" - "github.com/dexon-foundation/dexon-consensus-core/core/crypto" "github.com/dexon-foundation/dexon-consensus-core/core/types" ) -// Errors for compaction chain. -var ( - ErrNoWitnessToAck = fmt.Errorf( - "no witness to ack") - ErrIncorrectWitnessHash = fmt.Errorf( - "hash of witness ack is incorrect") - ErrIncorrectWitnessSignature = fmt.Errorf( - "signature of witness ack is incorrect") -) - type pendingAck struct { receivedTime time.Time - witnessAck *types.WitnessAck } type compactionChain struct { - db blockdb.Reader - pendingAckLock sync.RWMutex - pendingAck map[common.Hash]*pendingAck - prevBlockLock sync.RWMutex - prevBlock *types.Block - witnessAcksLock sync.RWMutex - latestWitnessAcks map[types.NodeID]*types.WitnessAck + db blockdb.Reader + pendingAckLock sync.RWMutex + pendingAck map[common.Hash]*pendingAck + prevBlockLock sync.RWMutex + prevBlock *types.Block + witnessAcksLock sync.RWMutex } func newCompactionChain( db blockdb.Reader, ) *compactionChain { return &compactionChain{ - db: db, - pendingAck: make(map[common.Hash]*pendingAck), - latestWitnessAcks: make(map[types.NodeID]*types.WitnessAck), + db: db, + pendingAck: make(map[common.Hash]*pendingAck), } } -func (cc *compactionChain) sanityCheck( - witnessAck *types.WitnessAck, witnessBlock *types.Block) error { - if witnessBlock != nil { - hash, err := hashWitness(witnessBlock) - if err != nil { - return err - } - if hash != witnessAck.Hash { - return ErrIncorrectWitnessHash - } - } - pubKey, err := crypto.SigToPub(witnessAck.Hash, witnessAck.Signature) - if err != nil { - return err - } - if witnessAck.ProposerID != types.NewNodeID(pubKey) { - return ErrIncorrectWitnessSignature - } +func (cc *compactionChain) sanityCheck(witnessBlock *types.Block) error { return nil } -// TODO(jimmy-dexon): processBlock and prepareWitnessAck can be extraced to +// TODO(jimmy-dexon): processBlock can be extraced to // another struct. func (cc *compactionChain) processBlock(block *types.Block) error { prevBlock := cc.lastBlock() if prevBlock != nil { - hash, err := hashWitness(prevBlock) - if err != nil { - return err - } block.Witness.Height = prevBlock.Witness.Height + 1 - block.Witness.ParentHash = hash } cc.prevBlockLock.Lock() defer cc.prevBlockLock.Unlock() @@ -104,115 +67,6 @@ func (cc *compactionChain) processBlock(block *types.Block) error { return nil } -func (cc *compactionChain) processWitnessAck(witnessAck *types.WitnessAck) ( - err error) { - // Before getting the Block from witnessAck.WitnessBlockHash, we can still - // do some sanityCheck to prevent invalid ack appending to pendingAck. - if err = cc.sanityCheck(witnessAck, nil); err != nil { - return - } - pendingFinished := make(chan struct{}) - go func() { - cc.processPendingWitnessAcks() - pendingFinished <- struct{}{} - }() - defer func() { - <-pendingFinished - }() - witnessBlock, err := cc.db.Get(witnessAck.WitnessBlockHash) - if err != nil { - if err == blockdb.ErrBlockDoesNotExist { - cc.pendingAckLock.Lock() - defer cc.pendingAckLock.Unlock() - cc.pendingAck[witnessAck.Hash] = &pendingAck{ - receivedTime: time.Now().UTC(), - witnessAck: witnessAck, - } - err = nil - } - return - } - return cc.processOneWitnessAck(witnessAck, &witnessBlock) -} - -func (cc *compactionChain) processOneWitnessAck( - witnessAck *types.WitnessAck, witnessBlock *types.Block) ( - err error) { - if err = cc.sanityCheck(witnessAck, witnessBlock); err != nil { - return - } - lastWitnessAck, exist := func() (ack *types.WitnessAck, exist bool) { - cc.witnessAcksLock.RLock() - defer cc.witnessAcksLock.RUnlock() - ack, exist = cc.latestWitnessAcks[witnessAck.ProposerID] - return - }() - if exist { - lastWitnessBlock, err2 := cc.db.Get(lastWitnessAck.WitnessBlockHash) - err = err2 - if err != nil { - return - } - if lastWitnessBlock.Witness.Height > witnessBlock.Witness.Height { - return - } - } - cc.witnessAcksLock.Lock() - defer cc.witnessAcksLock.Unlock() - cc.latestWitnessAcks[witnessAck.ProposerID] = witnessAck - return -} - -func (cc *compactionChain) processPendingWitnessAcks() { - pendingAck := func() map[common.Hash]*pendingAck { - pendingAck := make(map[common.Hash]*pendingAck) - cc.pendingAckLock.RLock() - defer cc.pendingAckLock.RUnlock() - for k, v := range cc.pendingAck { - pendingAck[k] = v - } - return pendingAck - }() - - for hash, ack := range pendingAck { - // TODO(jimmy-dexon): customizable timeout. - if ack.receivedTime.Add(30 * time.Second).Before(time.Now().UTC()) { - delete(pendingAck, hash) - continue - } - } - for hash, ack := range pendingAck { - witnessBlock, err := cc.db.Get(ack.witnessAck.WitnessBlockHash) - if err != nil { - if err == blockdb.ErrBlockDoesNotExist { - continue - } - // TODO(jimmy-dexon): this error needs to be handled properly. - fmt.Println(err) - delete(pendingAck, hash) - } - delete(pendingAck, hash) - cc.processOneWitnessAck(ack.witnessAck, &witnessBlock) - } - - cc.pendingAckLock.Lock() - defer cc.pendingAckLock.Unlock() - for k, v := range cc.pendingAck { - pendingAck[k] = v - } - cc.pendingAck = pendingAck -} - -func (cc *compactionChain) witnessAcks() map[types.NodeID]*types.WitnessAck { - cc.witnessAcksLock.RLock() - defer cc.witnessAcksLock.RUnlock() - acks := make(map[types.NodeID]*types.WitnessAck) - for k, v := range cc.latestWitnessAcks { - acks[k] = v.Clone() - } - return acks -} - func (cc *compactionChain) lastBlock() *types.Block { cc.prevBlockLock.RLock() defer cc.prevBlockLock.RUnlock() diff --git a/core/compaction-chain_test.go b/core/compaction-chain_test.go index 634cc68..b8f446e 100644 --- a/core/compaction-chain_test.go +++ b/core/compaction-chain_test.go @@ -23,7 +23,6 @@ import ( "github.com/dexon-foundation/dexon-consensus-core/common" "github.com/dexon-foundation/dexon-consensus-core/core/blockdb" - "github.com/dexon-foundation/dexon-consensus-core/core/crypto/ecdsa" "github.com/dexon-foundation/dexon-consensus-core/core/types" "github.com/stretchr/testify/suite" ) @@ -81,74 +80,10 @@ func (s *CompactionChainTestSuite) TestProcessBlock() { s.Equal(cc.prevBlock, prevBlock) err := cc.processBlock(block) s.Require().Nil(err) - if prevBlock != nil { - s.Equal(block.Witness.Height, prevBlock.Witness.Height+1) - prevHash, err := hashWitness(prevBlock) - s.Require().Nil(err) - s.Equal(prevHash, block.Witness.ParentHash) - } prevBlock = block } } -func (s *CompactionChainTestSuite) TestProcessWitnessAck() { - cc := s.newCompactionChain() - blocks := s.generateBlocks(10, cc) - prv1, err := ecdsa.NewPrivateKey() - s.Require().Nil(err) - prv2, err := ecdsa.NewPrivateKey() - s.Require().Nil(err) - nID1 := types.NewNodeID(prv1.PublicKey()) - nID2 := types.NewNodeID(prv2.PublicKey()) - auth1 := NewAuthenticator(prv1) - auth2 := NewAuthenticator(prv2) - witnessAcks1 := []*types.WitnessAck{} - witnessAcks2 := []*types.WitnessAck{} - for _, block := range blocks { - cc.prevBlock = block - witnessAck1, err := auth1.SignAsWitnessAck(block) - s.Require().Nil(err) - witnessAck2, err := auth2.SignAsWitnessAck(block) - s.Require().Nil(err) - witnessAcks1 = append(witnessAcks1, witnessAck1) - witnessAcks2 = append(witnessAcks2, witnessAck2) - } - // The acked block is not yet in db. - err = cc.processWitnessAck(witnessAcks1[0]) - s.Nil(err) - s.Equal(0, len(cc.witnessAcks())) - err = cc.processWitnessAck(witnessAcks2[1]) - s.Nil(err) - s.Equal(0, len(cc.witnessAcks())) - // Insert to block to db and trigger processPendingWitnessAck. - s.Require().Nil(s.db.Put(*blocks[0])) - s.Require().Nil(s.db.Put(*blocks[1])) - err = cc.processWitnessAck(witnessAcks1[2]) - s.Nil(err) - s.Equal(2, len(cc.witnessAcks())) - - // Test the witnessAcks should be the last witnessAck. - s.Require().Nil(s.db.Put(*blocks[2])) - s.Require().Nil(s.db.Put(*blocks[3])) - s.Nil(cc.processWitnessAck(witnessAcks1[3])) - - acks := cc.witnessAcks() - s.Equal(blocks[3].Hash, acks[nID1].WitnessBlockHash) - s.Equal(blocks[1].Hash, acks[nID2].WitnessBlockHash) - - // Test that witnessAck on less Witness.Height should be ignored. - s.Require().Nil(s.db.Put(*blocks[4])) - s.Require().Nil(s.db.Put(*blocks[5])) - s.Nil(cc.processWitnessAck(witnessAcks1[5])) - s.Nil(cc.processWitnessAck(witnessAcks2[5])) - s.Nil(cc.processWitnessAck(witnessAcks1[4])) - s.Nil(cc.processWitnessAck(witnessAcks2[4])) - - acks = cc.witnessAcks() - s.Equal(blocks[5].Hash, acks[nID1].WitnessBlockHash) - s.Equal(blocks[5].Hash, acks[nID2].WitnessBlockHash) -} - func TestCompactionChain(t *testing.T) { suite.Run(t, new(CompactionChainTestSuite)) } diff --git a/core/consensus.go b/core/consensus.go index 96abd36..d6ab930 100644 --- a/core/consensus.go +++ b/core/consensus.go @@ -313,7 +313,6 @@ func (con *Consensus) Run() { ticks = append(ticks, tick) go con.runBA(i, tick) } - go con.processWitnessData() // Reset ticker. <-con.tickerObj.Tick() @@ -355,7 +354,6 @@ BALoop: types.NewNotarySetTarget(con.gov.GetCRS(con.round), 0, chainID)) } aID := types.Position{ - ShardID: 0, ChainID: chainID, Height: con.rbModule.nextHeight(chainID), } @@ -434,10 +432,6 @@ func (con *Consensus) processMsg(msgChan <-chan interface{}) { if err := con.preProcessBlock(val); err != nil { log.Println(err) } - case *types.WitnessAck: - if err := con.ProcessWitnessAck(val); err != nil { - log.Println(err) - } case *types.Vote: if err := con.ProcessVote(val); err != nil { log.Println(err) @@ -484,44 +478,10 @@ func (con *Consensus) ProcessVote(vote *types.Vote) (err error) { return err } -// processWitnessData process witness acks. -func (con *Consensus) processWitnessData() { - ch := con.nbModule.BlockProcessedChan() - - for { - select { - case <-con.ctx.Done(): - return - case result := <-ch: - block, err := con.db.Get(result.BlockHash) - if err != nil { - panic(err) - } - block.Witness.Data = result.Data - if err := con.db.Update(block); err != nil { - panic(err) - } - // TODO(w): move the acking interval into governance. - if block.Witness.Height%5 != 0 { - continue - } - witnessAck, err := con.authModule.SignAsWitnessAck(&block) - if err != nil { - panic(err) - } - err = con.ProcessWitnessAck(witnessAck) - if err != nil { - panic(err) - } - con.nbModule.WitnessAckDelivered(witnessAck) - } - } -} - // sanityCheck checks if the block is a valid block func (con *Consensus) sanityCheck(b *types.Block) (err error) { // Check block.Position. - if b.Position.ShardID != 0 || b.Position.ChainID >= con.rbModule.chainNum() { + if b.Position.ChainID >= con.rbModule.chainNum() { return ErrIncorrectBlockPosition } // Check the timestamp of block. @@ -644,7 +604,7 @@ func (con *Consensus) prepareBlock(b *types.Block, con.rbModule.prepareBlock(b) b.Timestamp = proposeTime - b.Payload = con.nbModule.PreparePayload(b.Position) + b.Payload, b.Witness.Data = con.nbModule.PrepareBlock(b.Position) if err = con.authModule.SignBlock(b); err != nil { return } @@ -669,25 +629,3 @@ func (con *Consensus) PrepareGenesisBlock(b *types.Block, } return } - -// ProcessWitnessAck is the entry point to submit one witness ack. -func (con *Consensus) ProcessWitnessAck(witnessAck *types.WitnessAck) (err error) { - witnessAck = witnessAck.Clone() - // TODO(mission): check witness set for that round. - var round uint64 - exists, err := con.nodeSetCache.Exists(round, witnessAck.ProposerID) - if err != nil { - return - } - if !exists { - err = ErrProposerNotInNodeSet - return - } - err = con.ccModule.processWitnessAck(witnessAck) - return -} - -// WitnessAcks returns the latest WitnessAck received from all other nodes. -func (con *Consensus) WitnessAcks() map[types.NodeID]*types.WitnessAck { - return con.ccModule.witnessAcks() -} diff --git a/core/consensus_test.go b/core/consensus_test.go index 144e4c9..583a2e5 100644 --- a/core/consensus_test.go +++ b/core/consensus_test.go @@ -41,10 +41,6 @@ func (n *network) BroadcastVote(vote *types.Vote) {} func (n *network) BroadcastBlock(block *types.Block) { } -// BroadcastWitnessAck broadcasts witnessAck to all nodes in DEXON network. -func (n *network) BroadcastWitnessAck(witnessAck *types.WitnessAck) { -} - // SendDKGPrivateShare sends PrivateShare to a DKG participant. func (n *network) SendDKGPrivateShare( recv crypto.PublicKey, prvShare *types.DKGPrivateShare) { diff --git a/core/crypto.go b/core/crypto.go index 9a0d99d..ceb38f0 100644 --- a/core/crypto.go +++ b/core/crypto.go @@ -25,30 +25,6 @@ import ( "github.com/dexon-foundation/dexon-consensus-core/core/types" ) -func hashWitness(block *types.Block) (common.Hash, error) { - binaryTime, err := block.Witness.Timestamp.MarshalBinary() - if err != nil { - return common.Hash{}, err - } - binaryHeight := make([]byte, 8) - binary.LittleEndian.PutUint64(binaryHeight, block.Witness.Height) - hash := crypto.Keccak256Hash( - block.Witness.ParentHash[:], - binaryTime, - binaryHeight, - block.Witness.Data[:]) - return hash, nil -} - -func verifyWitnessSignature(pubkey crypto.PublicKey, - witnessBlock *types.Block, sig crypto.Signature) (bool, error) { - hash, err := hashWitness(witnessBlock) - if err != nil { - return false, err - } - return pubkey.VerifySignature(hash, sig), nil -} - func hashBlock(block *types.Block) (common.Hash, error) { hashPosition := hashPosition(block.Position) // Handling Block.Acks. @@ -129,9 +105,6 @@ func verifyCRSSignature(block *types.Block, crs common.Hash) ( } func hashPosition(position types.Position) common.Hash { - binaryShardID := make([]byte, 4) - binary.LittleEndian.PutUint32(binaryShardID, position.ShardID) - binaryChainID := make([]byte, 4) binary.LittleEndian.PutUint32(binaryChainID, position.ChainID) @@ -139,7 +112,6 @@ func hashPosition(position types.Position) common.Hash { binary.LittleEndian.PutUint64(binaryHeight, position.Height) return crypto.Keccak256Hash( - binaryShardID, binaryChainID, binaryHeight, ) diff --git a/core/crypto_test.go b/core/crypto_test.go index 4c4955b..85bfb63 100644 --- a/core/crypto_test.go +++ b/core/crypto_test.go @@ -48,10 +48,7 @@ func (s *CryptoTestSuite) prepareBlock(prevBlock *types.Block) *types.Block { }, } } - parentHash, err := hashWitness(prevBlock) - s.Require().Nil(err) s.Require().NotEqual(prevBlock.Hash, common.Hash{}) - acks = append(acks, parentHash) return &types.Block{ ParentHash: prevBlock.Hash, Acks: common.NewSortedHashes(acks), @@ -60,9 +57,8 @@ func (s *CryptoTestSuite) prepareBlock(prevBlock *types.Block) *types.Block { Height: prevBlock.Position.Height + 1, }, Witness: types.Witness{ - ParentHash: parentHash, - Timestamp: time.Now(), - Height: prevBlock.Witness.Height + 1, + Timestamp: time.Now(), + Height: prevBlock.Witness.Height + 1, }, } } @@ -76,63 +72,15 @@ func (s *CryptoTestSuite) newBlock(prevBlock *types.Block) *types.Block { } func (s *CryptoTestSuite) generateCompactionChain( - length int, prv crypto.PrivateKey) ( - []*types.Block, []types.WitnessAck) { + length int, prv crypto.PrivateKey) []*types.Block { blocks := make([]*types.Block, length) - witnessAcks := make([]types.WitnessAck, length) var prevBlock *types.Block for idx := range blocks { block := s.newBlock(prevBlock) prevBlock = block blocks[idx] = block - var err error - witnessAcks[idx].Hash, err = hashWitness(blocks[idx]) - s.Require().Nil(err) - witnessAcks[idx].WitnessBlockHash = blocks[idx].Hash - witnessAcks[idx].Signature, err = prv.Sign(witnessAcks[idx].Hash) - s.Require().Nil(err) - if idx > 0 { - block.Witness.ParentHash = witnessAcks[idx-1].Hash - } - } - return blocks, witnessAcks -} - -func (s *CryptoTestSuite) TestWitnessAckSignature() { - prv, err := ecdsa.NewPrivateKey() - pub := prv.PublicKey() - s.Require().Nil(err) - blocks, witnessAcks := s.generateCompactionChain(10, prv) - blockMap := make(map[common.Hash]*types.Block) - for _, block := range blocks { - blockMap[block.Hash] = block - } - parentBlock := blocks[0] - for _, witnessAck := range witnessAcks { - witnessBlock, exist := blockMap[witnessAck.WitnessBlockHash] - s.Require().True(exist) - if witnessBlock.Witness.Height == 0 { - continue - } - s.True(parentBlock.Witness.Height == witnessBlock.Witness.Height-1) - hash, err := hashWitness(parentBlock) - s.Require().Nil(err) - s.Equal(hash, witnessBlock.Witness.ParentHash) - s.True(verifyWitnessSignature( - pub, witnessBlock, witnessAck.Signature)) - parentBlock = witnessBlock - - } - // Modify Block.Witness.Timestamp and verify signature again. - for _, witnessAck := range witnessAcks { - block, exist := blockMap[witnessAck.WitnessBlockHash] - s.Require().True(exist) - block.Witness.Timestamp = time.Time{} - ackingBlock, exist := blockMap[witnessAck.WitnessBlockHash] - s.Require().True(exist) - s.False(verifyWitnessSignature( - pub, ackingBlock, witnessAck.Signature)) } + return blocks } func (s *CryptoTestSuite) generateBlockChain( diff --git a/core/interfaces.go b/core/interfaces.go index 37adf36..465a3ce 100644 --- a/core/interfaces.go +++ b/core/interfaces.go @@ -28,21 +28,14 @@ import ( // Application describes the application interface that interacts with DEXON // consensus core. type Application interface { - // PreparePayload is called when consensus core is preparing a block. - PreparePayload(position types.Position) []byte + // PrepareBlock is called when consensus core is preparing a block. + PrepareBlock(position types.Position) (payload []byte, witnessData []byte) - // VerifyPayload verifies if the payload is valid. - VerifyPayload(payload []byte) bool + // VerifyBlock verifies if the block is valid. + VerifyBlock(block *types.Block) bool // BlockDelivered is called when a block is add to the compaction chain. BlockDelivered(block types.Block) - - // BlockProcessedChan returns a channel to receive the block hashes that have - // finished processing by the application. - BlockProcessedChan() <-chan types.WitnessResult - - // WitnessAckDelivered is called when a witness ack is created. - WitnessAckDelivered(witnessAck *types.WitnessAck) } // Debug describes the application interface that requires @@ -68,9 +61,6 @@ type Network interface { // BroadcastBlock broadcasts block to all nodes in DEXON network. BroadcastBlock(block *types.Block) - // BroadcastWitnessAck broadcasts witnessAck to all nodes in DEXON network. - BroadcastWitnessAck(witnessAck *types.WitnessAck) - // SendDKGPrivateShare sends PrivateShare to a DKG participant. SendDKGPrivateShare(pub crypto.PublicKey, prvShare *types.DKGPrivateShare) diff --git a/core/nonblocking.go b/core/nonblocking.go index 1dd1ded..2e5bfeb 100644 --- a/core/nonblocking.go +++ b/core/nonblocking.go @@ -42,10 +42,6 @@ type blockDeliveredEvent struct { block *types.Block } -type witnessAckEvent struct { - witnessAck *types.WitnessAck -} - // nonBlocking implements these interfaces and is a decorator for // them that makes the methods to be non-blocking. // - Application @@ -103,8 +99,6 @@ func (nb *nonBlocking) run() { nb.debug.TotalOrderingDelivered(e.blockHashes, e.early) case blockDeliveredEvent: nb.app.BlockDelivered(*e.block) - case witnessAckEvent: - nb.app.WitnessAckDelivered(e.witnessAck) default: fmt.Printf("Unknown event %v.", e) } @@ -123,15 +117,14 @@ func (nb *nonBlocking) wait() { nb.running.Wait() } -// PreparePayload cannot be non-blocking. -func (nb *nonBlocking) PreparePayload( - position types.Position) []byte { - return nb.app.PreparePayload(position) +// PrepareBlock cannot be non-blocking. +func (nb *nonBlocking) PrepareBlock(position types.Position) ([]byte, []byte) { + return nb.app.PrepareBlock(position) } -// VerifyPayload cannot be non-blocking. -func (nb *nonBlocking) VerifyPayload(payload []byte) bool { - return nb.app.VerifyPayload(payload) +// VerifyBlock cannot be non-blocking. +func (nb *nonBlocking) VerifyBlock(block *types.Block) bool { + return nb.app.VerifyBlock(block) } // BlockConfirmed is called when a block is confirmed and added to lattice. @@ -161,14 +154,3 @@ func (nb *nonBlocking) TotalOrderingDelivered( func (nb *nonBlocking) BlockDelivered(block types.Block) { nb.addEvent(blockDeliveredEvent{&block}) } - -// BlockProcessedChan returns a channel to receive the block hashes that have -// finished processing by the application. -func (nb *nonBlocking) BlockProcessedChan() <-chan types.WitnessResult { - return nb.app.BlockProcessedChan() -} - -// WitnessAckDelivered is called when a witness ack is created. -func (nb *nonBlocking) WitnessAckDelivered(witnessAck *types.WitnessAck) { - nb.addEvent(witnessAckEvent{witnessAck}) -} diff --git a/core/nonblocking_test.go b/core/nonblocking_test.go index d4ffb26..8c3cda9 100644 --- a/core/nonblocking_test.go +++ b/core/nonblocking_test.go @@ -33,8 +33,6 @@ type slowApp struct { stronglyAcked map[common.Hash]struct{} totalOrderingDelivered map[common.Hash]struct{} blockDelivered map[common.Hash]struct{} - witnessAck map[common.Hash]struct{} - witnessResultChan chan types.WitnessResult } func newSlowApp(sleep time.Duration) *slowApp { @@ -44,16 +42,14 @@ func newSlowApp(sleep time.Duration) *slowApp { stronglyAcked: make(map[common.Hash]struct{}), totalOrderingDelivered: make(map[common.Hash]struct{}), blockDelivered: make(map[common.Hash]struct{}), - witnessAck: make(map[common.Hash]struct{}), - witnessResultChan: make(chan types.WitnessResult), } } -func (app *slowApp) PreparePayload(_ types.Position) []byte { - return []byte{} +func (app *slowApp) PrepareBlock(_ types.Position) ([]byte, []byte) { + return []byte{}, []byte{} } -func (app *slowApp) VerifyPayload(_ []byte) bool { +func (app *slowApp) VerifyBlock(_ *types.Block) bool { return true } @@ -79,15 +75,6 @@ func (app *slowApp) BlockDelivered(block types.Block) { app.blockDelivered[block.Hash] = struct{}{} } -func (app *slowApp) BlockProcessedChan() <-chan types.WitnessResult { - return app.witnessResultChan -} - -func (app *slowApp) WitnessAckDelivered(witnessAck *types.WitnessAck) { - time.Sleep(app.sleep) - app.witnessAck[witnessAck.Hash] = struct{}{} -} - type NonBlockingTestSuite struct { suite.Suite } @@ -111,7 +98,6 @@ func (s *NonBlockingTestSuite) TestNonBlocking() { Hash: hash, Witness: types.Witness{Timestamp: time.Now().UTC()}, }) - nbModule.WitnessAckDelivered(&types.WitnessAck{Hash: hash}) } nbModule.TotalOrderingDelivered(hashes, true) @@ -124,7 +110,6 @@ func (s *NonBlockingTestSuite) TestNonBlocking() { s.Contains(app.stronglyAcked, hash) s.Contains(app.totalOrderingDelivered, hash) s.Contains(app.blockDelivered, hash) - s.Contains(app.witnessAck, hash) } } diff --git a/core/shard.go b/core/shard.go index 2033d66..32f1b79 100644 --- a/core/shard.go +++ b/core/shard.go @@ -80,7 +80,7 @@ func (s *Shard) PrepareBlock( // TODO(mission): the proposeTime might be earlier than tip block of // that chain. We should let blockLattice suggest the time. b.Timestamp = proposeTime - b.Payload = s.app.PreparePayload(b.Position) + b.Payload, b.Witness.Data = s.app.PrepareBlock(b.Position) if err = s.authModule.SignBlock(b); err != nil { return } @@ -92,11 +92,6 @@ func (s *Shard) PrepareBlock( // If some acking blocks don't exists, Shard would help to cache this block // and retry when lattice updated in Shard.ProcessBlock. func (s *Shard) SanityCheck(b *types.Block) (err error) { - // Check block.Position. - if b.Position.ShardID != s.ID { - err = ErrIncorrectBlockPosition - return - } // Check the hash of block. hash, err := hashBlock(b) if err != nil || hash != b.Hash { diff --git a/core/test/app.go b/core/test/app.go index 4cac36c..3ec670c 100644 --- a/core/test/app.go +++ b/core/test/app.go @@ -50,10 +50,6 @@ var ( // and delivered are different. ErrMismatchTotalOrderingAndDelivered = fmt.Errorf( "mismatch total ordering and delivered sequence") - // ErrWitnessAckUnknownBlock means the witness ack is acking on the unknown - // block. - ErrWitnessAckUnknownBlock = fmt.Errorf( - "witness ack on unknown block") ) // AppAckedRecord caches information when this application received @@ -87,9 +83,6 @@ type App struct { Delivered map[common.Hash]*AppDeliveredRecord DeliverSequence common.Hashes deliveredLock sync.RWMutex - WitnessAckSequence []*types.WitnessAck - witnessAckLock sync.RWMutex - witnessResultChan chan types.WitnessResult } // NewApp constructs a TestApp instance. @@ -100,17 +93,16 @@ func NewApp() *App { TotalOrderedByHash: make(map[common.Hash]*AppTotalOrderRecord), Delivered: make(map[common.Hash]*AppDeliveredRecord), DeliverSequence: common.Hashes{}, - witnessResultChan: make(chan types.WitnessResult), } } -// PreparePayload implements Application interface. -func (app *App) PreparePayload(position types.Position) []byte { - return []byte{} +// PrepareBlock implements Application interface. +func (app *App) PrepareBlock(position types.Position) ([]byte, []byte) { + return []byte{}, []byte{} } -// VerifyPayload implements Application. -func (app *App) VerifyPayload(payload []byte) bool { +// VerifyBlock implements Application. +func (app *App) VerifyBlock(block *types.Block) bool { return true } @@ -157,20 +149,6 @@ func (app *App) BlockDelivered(block types.Block) { app.DeliverSequence = append(app.DeliverSequence, block.Hash) } -// BlockProcessedChan returns a channel to receive the block hashes that have -// finished processing by the application. -func (app *App) BlockProcessedChan() <-chan types.WitnessResult { - return app.witnessResultChan -} - -// WitnessAckDelivered implements Application interface. -func (app *App) WitnessAckDelivered(witnessAck *types.WitnessAck) { - app.witnessAckLock.Lock() - defer app.witnessAckLock.Unlock() - - app.WitnessAckSequence = append(app.WitnessAckSequence, witnessAck) -} - // Compare performs these checks against another App instance // and return erros if not passed: // - deliver sequence by comparing block hashes. @@ -256,15 +234,6 @@ Loop: // by total ordering. return ErrMismatchTotalOrderingAndDelivered } - - // Make sure that witnessAck is acking the correct block. - app.witnessAckLock.RLock() - defer app.witnessAckLock.RUnlock() - for _, witnessAck := range app.WitnessAckSequence { - if _, exists := app.Delivered[witnessAck.WitnessBlockHash]; !exists { - return ErrWitnessAckUnknownBlock - } - } return nil } diff --git a/core/test/app_test.go b/core/test/app_test.go index 9d35c67..baea838 100644 --- a/core/test/app_test.go +++ b/core/test/app_test.go @@ -165,11 +165,6 @@ func (s *AppTestSuite) TestVerify() { // Witness ack on unknown block. app5 := NewApp() s.setupAppByTotalOrderDeliver(app5, s.to1) - app5.WitnessAckDelivered(&types.WitnessAck{ - Hash: common.NewRandomHash(), - WitnessBlockHash: common.NewRandomHash(), - }) - req.Equal(ErrWitnessAckUnknownBlock, app5.Verify()) } func TestApp(t *testing.T) { diff --git a/core/types/block.go b/core/types/block.go index e4b4466..1d994e6 100644 --- a/core/types/block.go +++ b/core/types/block.go @@ -39,6 +39,13 @@ var ( } ) +// Witness represents the consensus information on the compaction chain. +type Witness struct { + Timestamp time.Time `json:"timestamp"` + Height uint64 `json:"height"` + Data []byte `json:"data"` +} + // RecycleBlock put unused block into cache, which might be reused if // not garbage collected. func RecycleBlock(b *Block) { @@ -61,11 +68,10 @@ type Block struct { Timestamp time.Time `json:"timestamps"` Acks common.SortedHashes `json:"acks"` Payload []byte `json:"payload"` + Witness Witness `json:"witness"` Signature crypto.Signature `json:"signature"` CRSSignature crypto.Signature `json:"crs_signature"` - - Witness Witness `json:"witness"` } func (b *Block) String() string { @@ -79,7 +85,6 @@ func (b *Block) Clone() (bcopy *Block) { bcopy.ProposerID = b.ProposerID bcopy.ParentHash = b.ParentHash bcopy.Hash = b.Hash - bcopy.Position.ShardID = b.Position.ShardID bcopy.Position.ChainID = b.Position.ChainID bcopy.Position.Height = b.Position.Height bcopy.Signature = b.Signature.Clone() diff --git a/core/types/nodeset.go b/core/types/nodeset.go index e6efd76..6b78cab 100644 --- a/core/types/nodeset.go +++ b/core/types/nodeset.go @@ -72,13 +72,10 @@ func NewNodeSet() *NodeSet { // NewNotarySetTarget is the target for getting Notary Set. func NewNotarySetTarget(crs []byte, shardID, chainID uint32) SubSetTarget { - binaryShardID := make([]byte, 4) - binary.LittleEndian.PutUint32(binaryShardID, shardID) - binaryChainID := make([]byte, 4) binary.LittleEndian.PutUint32(binaryChainID, chainID) - return newTarget(targetNotarySet, crs, binaryShardID, binaryChainID) + return newTarget(targetNotarySet, crs, binaryChainID) } // NewWitnessSetTarget is the target for getting DKG Set. diff --git a/core/types/position.go b/core/types/position.go index a894c4b..af57b29 100644 --- a/core/types/position.go +++ b/core/types/position.go @@ -19,7 +19,7 @@ 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"` + Round uint64 `json:"round"` Height uint64 `json:"height"` } diff --git a/core/types/vote.go b/core/types/vote.go index 294bc74..0d4f8ab 100644 --- a/core/types/vote.go +++ b/core/types/vote.go @@ -60,7 +60,6 @@ func (v *Vote) Clone() *Vote { BlockHash: v.BlockHash, Period: v.Period, Position: Position{ - ShardID: v.Position.ShardID, ChainID: v.Position.ChainID, Height: v.Position.Height, }, diff --git a/core/types/witness.go b/core/types/witness.go deleted file mode 100644 index 3c455d8..0000000 --- a/core/types/witness.go +++ /dev/null @@ -1,61 +0,0 @@ -// 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 -// <http://www.gnu.org/licenses/>. - -// TODO(jimmy-dexon): remove comments of WitnessAck before open source. - -package types - -import ( - "time" - - "github.com/dexon-foundation/dexon-consensus-core/common" - "github.com/dexon-foundation/dexon-consensus-core/core/crypto" -) - -// WitnessAck represents the acking to the compaction chain. -type WitnessAck struct { - ProposerID NodeID `json:"proposer_id"` - WitnessBlockHash common.Hash `json:"witness_block_hash"` - Hash common.Hash `json:"hash"` - // WitnessSignature is the signature of the hash value of BlockWitness. - Signature crypto.Signature `json:"signature"` -} - -// Clone returns a deep copy of a WitnessAck. -func (a *WitnessAck) Clone() *WitnessAck { - return &WitnessAck{ - ProposerID: a.ProposerID, - WitnessBlockHash: a.WitnessBlockHash, - Hash: a.Hash, - Signature: a.Signature, - } -} - -// Witness represents the consensus information on the compaction chain. -type Witness struct { - ParentHash common.Hash `json:"parent_hash"` - Timestamp time.Time `json:"timestamp"` - Height uint64 `json:"height"` - Data []byte `json:"data"` -} - -// WitnessResult is the result pass from application containing the witness -// data. -type WitnessResult struct { - BlockHash common.Hash - Data []byte -} |