aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rw-r--r--core/authenticator.go21
-rw-r--r--core/authenticator_test.go3
-rw-r--r--core/blocklattice.go1
-rw-r--r--core/compaction-chain.go166
-rw-r--r--core/compaction-chain_test.go65
-rw-r--r--core/consensus.go66
-rw-r--r--core/consensus_test.go4
-rw-r--r--core/crypto.go28
-rw-r--r--core/crypto_test.go60
-rw-r--r--core/interfaces.go18
-rw-r--r--core/nonblocking.go30
-rw-r--r--core/nonblocking_test.go21
-rw-r--r--core/shard.go7
-rw-r--r--core/test/app.go41
-rw-r--r--core/test/app_test.go5
-rw-r--r--core/types/block.go11
-rw-r--r--core/types/nodeset.go5
-rw-r--r--core/types/position.go2
-rw-r--r--core/types/vote.go1
-rw-r--r--core/types/witness.go61
-rw-r--r--simulation/app.go29
-rw-r--r--simulation/marshaller.go8
-rw-r--r--simulation/network.go9
23 files changed, 51 insertions, 611 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
-}
diff --git a/simulation/app.go b/simulation/app.go
index cb8c129..5f1a85a 100644
--- a/simulation/app.go
+++ b/simulation/app.go
@@ -40,7 +40,6 @@ type simApp struct {
unconfirmedBlocks map[types.NodeID]common.Hashes
blockByHash map[common.Hash]*types.Block
blockByHashMutex sync.RWMutex
- witnessResultChan chan types.WitnessResult
}
// newSimApp returns point to a new instance of simApp.
@@ -52,7 +51,6 @@ func newSimApp(id types.NodeID, netModule *network) *simApp {
blockSeen: make(map[common.Hash]time.Time),
unconfirmedBlocks: make(map[types.NodeID]common.Hashes),
blockByHash: make(map[common.Hash]*types.Block),
- witnessResultChan: make(chan types.WitnessResult),
}
}
@@ -60,8 +58,8 @@ func newSimApp(id types.NodeID, netModule *network) *simApp {
func (a *simApp) BlockConfirmed(_ common.Hash) {
}
-// VerifyPayload implements core.Application.
-func (a *simApp) VerifyPayload(payload []byte) bool {
+// VerifyBlock implements core.Application.
+func (a *simApp) VerifyBlock(block *types.Block) bool {
return true
}
@@ -91,9 +89,9 @@ func (a *simApp) getAckedBlocks(ackHash common.Hash) (output common.Hashes) {
return
}
-// PreparePayload implements core.Application.
-func (a *simApp) PreparePayload(position types.Position) []byte {
- return []byte{}
+// PrepareBlock implements core.Application.
+func (a *simApp) PrepareBlock(position types.Position) ([]byte, []byte) {
+ return []byte{}, []byte{}
}
// StronglyAcked is called when a block is strongly acked by DEXON
@@ -150,21 +148,4 @@ func (a *simApp) BlockDelivered(block types.Block) {
Payload: jsonPayload,
}
a.netModule.report(msg)
-
- go func() {
- a.witnessResultChan <- types.WitnessResult{
- BlockHash: block.Hash,
- Data: []byte(fmt.Sprintf("Block %s", block.Hash)),
- }
- }()
-}
-
-// BlockProcessedChan returns a channel to receive the block hashes that have
-// finished processing by the application.
-func (a *simApp) BlockProcessedChan() <-chan types.WitnessResult {
- return a.witnessResultChan
-}
-
-// WitnessAckDelivered is called when a witness ack is created.
-func (a *simApp) WitnessAckDelivered(witnessAck *types.WitnessAck) {
}
diff --git a/simulation/marshaller.go b/simulation/marshaller.go
index f2d17af..cdf080b 100644
--- a/simulation/marshaller.go
+++ b/simulation/marshaller.go
@@ -57,12 +57,6 @@ func (m *jsonMarshaller) Unmarshal(
break
}
msg = block
- case "witness-ack":
- nAck := &types.WitnessAck{}
- if err = json.Unmarshal(payload, nAck); err != nil {
- break
- }
- msg = nAck
case "vote":
vote := &types.Vote{}
if err = json.Unmarshal(payload, vote); err != nil {
@@ -115,8 +109,6 @@ func (m *jsonMarshaller) Marshal(msg interface{}) (
msgType = "info-status"
case *types.Block:
msgType = "block"
- case *types.WitnessAck:
- msgType = "witness-ack"
case *types.Vote:
msgType = "vote"
case *types.DKGPrivateShare:
diff --git a/simulation/network.go b/simulation/network.go
index f1f586a..0faacf7 100644
--- a/simulation/network.go
+++ b/simulation/network.go
@@ -132,13 +132,6 @@ func (n *network) BroadcastBlock(block *types.Block) {
}
}
-// BroadcastWitnessAck implements core.Network interface.
-func (n *network) BroadcastWitnessAck(witnessAck *types.WitnessAck) {
- if err := n.trans.Broadcast(witnessAck); err != nil {
- panic(err)
- }
-}
-
// broadcast message to all other nodes in the network.
func (n *network) broadcast(message interface{}) {
if err := n.trans.Broadcast(message); err != nil {
@@ -205,7 +198,7 @@ func (n *network) run() {
// to consensus or node, that's the question.
disp := func(e *test.TransportEnvelope) {
switch e.Msg.(type) {
- case *types.Block, *types.Vote, *types.WitnessAck,
+ case *types.Block, *types.Vote,
*types.DKGPrivateShare, *types.DKGPartialSignature:
n.toConsensus <- e.Msg
default: