aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorJimmy Hu <jimmy.hu@dexon.org>2018-09-11 13:28:44 +0800
committerGitHub <noreply@github.com>2018-09-11 13:28:44 +0800
commit582a491aa0bcb784ac7b65ebbfb42139945ea703 (patch)
tree8589bad986f512455717728012c3d9edf3b68c4f
parent2439f49063d8498eadf26d4fa1220c5eac8412a8 (diff)
downloadtangerine-consensus-582a491aa0bcb784ac7b65ebbfb42139945ea703.tar
tangerine-consensus-582a491aa0bcb784ac7b65ebbfb42139945ea703.tar.gz
tangerine-consensus-582a491aa0bcb784ac7b65ebbfb42139945ea703.tar.bz2
tangerine-consensus-582a491aa0bcb784ac7b65ebbfb42139945ea703.tar.lz
tangerine-consensus-582a491aa0bcb784ac7b65ebbfb42139945ea703.tar.xz
tangerine-consensus-582a491aa0bcb784ac7b65ebbfb42139945ea703.tar.zst
tangerine-consensus-582a491aa0bcb784ac7b65ebbfb42139945ea703.zip
core: timestamp (#98)
-rw-r--r--blockdb/level-db_test.go6
-rw-r--r--core/consensus-timestamp.go78
-rw-r--r--core/consensus-timestamp_test.go199
-rw-r--r--core/consensus.go11
-rw-r--r--core/consensus_test.go11
-rw-r--r--core/crypto.go20
-rw-r--r--core/crypto_test.go9
-rw-r--r--core/reliable-broadcast.go53
-rw-r--r--core/reliable-broadcast_test.go53
-rw-r--r--core/test/blocks-generator.go40
-rw-r--r--core/types/block.go29
-rw-r--r--core/utils.go23
-rw-r--r--simulation/app.go2
13 files changed, 173 insertions, 361 deletions
diff --git a/blockdb/level-db_test.go b/blockdb/level-db_test.go
index d1c8146..06829f0 100644
--- a/blockdb/level-db_test.go
+++ b/blockdb/level-db_test.go
@@ -73,9 +73,7 @@ func (s *LevelDBTestSuite) TestBasicUsage() {
// Test Update.
now := time.Now().UTC()
- queried.Timestamps = map[types.ValidatorID]time.Time{
- queried.ProposerID: now,
- }
+ queried.Timestamp = now
err = db.Update(queried)
s.Nil(err)
@@ -84,7 +82,7 @@ func (s *LevelDBTestSuite) TestBasicUsage() {
queried, err = db.Get(block1.Hash)
s.Nil(err)
- s.Equal(now, queried.Timestamps[queried.ProposerID])
+ s.Equal(now, queried.Timestamp)
}
func (s *LevelDBTestSuite) TestSyncIndex() {
diff --git a/core/consensus-timestamp.go b/core/consensus-timestamp.go
index 90489ed..62bdf0c 100644
--- a/core/consensus-timestamp.go
+++ b/core/consensus-timestamp.go
@@ -19,20 +19,20 @@ package core
import (
"errors"
+ "time"
"github.com/dexon-foundation/dexon-consensus-core/core/types"
)
// consensusTimestamp is for Concensus Timestamp Algorithm.
type consensusTimestamp struct {
- lastMainChainBlock *types.Block
- blocksNotInMainChain []*types.Block
+ chainTimestamps []time.Time
}
var (
- // ErrInvalidMainChain would be reported if the invalid result from
- // main chain selection algorithm is detected.
- ErrInvalidMainChain = errors.New("invalid main chain")
+ // ErrTimestampNotIncrease would be reported if the timestamp is not strickly
+ // increasing on the same chain.
+ ErrTimestampNotIncrease = errors.New("timestamp is not increasing")
)
// newConsensusTimestamp create timestamper object.
@@ -41,68 +41,24 @@ func newConsensusTimestamp() *consensusTimestamp {
}
// ProcessBlocks is the entry function.
-func (ct *consensusTimestamp) processBlocks(blocks []*types.Block) (
- blocksWithTimestamp []*types.Block, mainChain []*types.Block, err error) {
- if len(blocks) == 0 {
- // TODO (jimmy-dexon): Remove this panic before release.
- panic("Unexpected empty block list.")
- }
- outputFirstBlock := true
- blocks = append(ct.blocksNotInMainChain, blocks...)
- if ct.lastMainChainBlock != nil {
- // TODO (jimmy-dexon): The performance here can be optimized.
- blocks = append([]*types.Block{ct.lastMainChainBlock}, blocks...)
- outputFirstBlock = false
- }
- mainChain, nonMainChain := ct.selectMainChain(blocks)
- ct.blocksNotInMainChain = nonMainChain
- ct.lastMainChainBlock = mainChain[len(mainChain)-1]
- blocksWithTimestamp = blocks[:len(blocks)-len(nonMainChain)]
- leftMainChainIdx := 0
- rightMainChainIdx := 0
- idxMainChain := 0
- for idx, block := range blocksWithTimestamp {
- if idxMainChain >= len(mainChain) {
- err = ErrInvalidMainChain
- return
- } else if block.Hash == mainChain[idxMainChain].Hash {
- rightMainChainIdx = idx
- blocksWithTimestamp[idx].Notary.Timestamp, err =
- getMedianTime(block)
+func (ct *consensusTimestamp) processBlocks(blocks []*types.Block) (err error) {
+ for _, block := range blocks {
+ if !block.IsGenesis() {
+ block.Notary.Timestamp, err = getMedianTime(ct.chainTimestamps)
if err != nil {
return
}
- // Process Non-MainChain blocks.
- if rightMainChainIdx > leftMainChainIdx {
- for idx, timestamp := range interpoTime(
- blocksWithTimestamp[leftMainChainIdx].Notary.Timestamp,
- blocksWithTimestamp[rightMainChainIdx].Notary.Timestamp,
- rightMainChainIdx-leftMainChainIdx-1) {
- blocksWithTimestamp[leftMainChainIdx+idx+1].Notary.Timestamp =
- timestamp
- }
- }
- leftMainChainIdx = idx
- idxMainChain++
}
- }
- if !outputFirstBlock {
- blocksWithTimestamp = blocksWithTimestamp[1:]
- }
- return
-}
-func (ct *consensusTimestamp) selectMainChain(blocks []*types.Block) (
- mainChain []*types.Block, nonMainChain []*types.Block) {
- for _, block := range blocks {
- if len(mainChain) != 0 {
- if _, exists := block.Acks[mainChain[len(mainChain)-1].Hash]; !exists {
- nonMainChain = append(nonMainChain, block)
- continue
- }
+ for uint32(len(ct.chainTimestamps)) <= block.Position.ChainID {
+ ct.chainTimestamps = append(ct.chainTimestamps, time.Time{})
}
- nonMainChain = []*types.Block{}
- mainChain = append(mainChain, block)
+
+ if !block.Timestamp.After(ct.chainTimestamps[block.Position.ChainID]) {
+ return ErrTimestampNotIncrease
+ }
+
+ ct.chainTimestamps[block.Position.ChainID] = block.Timestamp
}
return
}
diff --git a/core/consensus-timestamp_test.go b/core/consensus-timestamp_test.go
index 6dff111..56dd306 100644
--- a/core/consensus-timestamp_test.go
+++ b/core/consensus-timestamp_test.go
@@ -33,174 +33,109 @@ type ConsensusTimestampTest struct {
suite.Suite
}
-func generateBlocksWithAcks(blockNum, maxAcks int) []*types.Block {
- chain := []*types.Block{
- &types.Block{
- Hash: common.NewRandomHash(),
- Acks: make(map[common.Hash]struct{}),
- },
+func (s *ConsensusTimestampTest) generateBlocksWithTimestamp(
+ blockNum, chainNum int,
+ step, sigma time.Duration) []*types.Block {
+ blocks := make([]*types.Block, blockNum)
+ chainIDs := make([]uint32, len(blocks))
+ for i := range chainIDs {
+ chainIDs[i] = uint32(i % chainNum)
}
- for i := 1; i < blockNum; i++ {
- acks := make(map[common.Hash]struct{})
- ackNum := rand.Intn(maxAcks) + 1
- for j := 0; j < ackNum; j++ {
- ack := rand.Intn(len(chain))
- acks[chain[ack].Hash] = struct{}{}
+ rand.Shuffle(len(chainIDs), func(i, j int) {
+ chainIDs[i], chainIDs[j] = chainIDs[j], chainIDs[i]
+ })
+ chainTimestamps := make(map[uint32]time.Time)
+ for idx := range blocks {
+ blocks[idx] = &types.Block{}
+ block := blocks[idx]
+ if idx < chainNum {
+ // Genesis blocks.
+ block.Position.ChainID = uint32(idx)
+ block.ParentHash = common.Hash{}
+ block.Position.Height = 0
+ s.Require().True(block.IsGenesis())
+ chainTimestamps[uint32(idx)] = time.Now().UTC()
+ } else {
+ block.Position.ChainID = chainIDs[idx]
+ // Assign 1 to height to make this block non-genesis.
+ block.Position.Height = 1
+ s.Require().False(block.IsGenesis())
}
- block := &types.Block{
- Hash: common.NewRandomHash(),
- Acks: acks,
- }
- chain = append(chain, block)
+ block.Timestamp = chainTimestamps[block.Position.ChainID]
+ // Update timestamp for next block.
+ diffSeconds := rand.NormFloat64() * sigma.Seconds()
+ diffSeconds = math.Min(diffSeconds, step.Seconds()/2.1)
+ diffSeconds = math.Max(diffSeconds, -step.Seconds()/2.1)
+ diffDuration := time.Duration(diffSeconds*1000) * time.Millisecond
+ chainTimestamps[block.Position.ChainID] =
+ chainTimestamps[block.Position.ChainID].Add(step).Add(diffDuration)
+ s.Require().True(block.Timestamp.Before(
+ chainTimestamps[block.Position.ChainID]))
}
- return chain
+ return blocks
}
-func fillBlocksTimestamps(blocks []*types.Block, validatorNum int,
- step, sigma time.Duration) {
- curTime := time.Now().UTC()
- vIDs := make([]types.ValidatorID, validatorNum)
- for i := 0; i < validatorNum; i++ {
- vIDs[i] = types.ValidatorID{Hash: common.NewRandomHash()}
- }
+func (s *ConsensusTimestampTest) extractTimestamps(
+ blocks []*types.Block) []time.Time {
+ timestamps := make([]time.Time, 0, len(blocks))
for _, block := range blocks {
- block.Timestamps = make(map[types.ValidatorID]time.Time)
- for _, vID := range vIDs {
- diffSeconds := rand.NormFloat64() * sigma.Seconds()
- diffSeconds = math.Min(diffSeconds, step.Seconds()/2)
- diffSeconds = math.Max(diffSeconds, -step.Seconds()/2)
- diffDuration := time.Duration(diffSeconds*1000) * time.Millisecond
- block.Timestamps[vID] = curTime.Add(diffDuration)
+ if block.IsGenesis() {
+ continue
}
- curTime = curTime.Add(step)
- }
-}
-
-func extractTimestamps(blocks []*types.Block) []time.Time {
- timestamps := make([]time.Time, len(blocks))
- for idx, block := range blocks {
- timestamps[idx] = block.Notary.Timestamp
+ timestamps = append(timestamps, block.Notary.Timestamp)
}
return timestamps
}
-func (s *ConsensusTimestampTest) TestMainChainSelection() {
- ct := newConsensusTimestamp()
- ct2 := newConsensusTimestamp()
- blockNums := []int{50, 100, 30}
- maxAcks := 5
- for _, blockNum := range blockNums {
- chain := generateBlocksWithAcks(blockNum, maxAcks)
- mainChain, _ := ct.selectMainChain(chain)
- // Verify the selected main chain.
- for i := 1; i < len(mainChain); i++ {
- _, exists := mainChain[i].Acks[mainChain[i-1].Hash]
- s.True(exists)
- }
- // Verify if selectMainChain is stable.
- mainChain2, _ := ct2.selectMainChain(chain)
- s.Equal(mainChain, mainChain2)
- }
-}
-
+// TestTimestampPartition verifies that processing segments of compatction chain
+// should have the same result as processing the whole chain at once.
func (s *ConsensusTimestampTest) TestTimestampPartition() {
blockNums := []int{50, 100, 30}
validatorNum := 19
sigma := 100 * time.Millisecond
- maxAcks := 5
- totalMainChain := make([]*types.Block, 1)
- totalChain := make([]*types.Block, 0)
totalTimestamps := make([]time.Time, 0)
ct := newConsensusTimestamp()
- var lastMainChainBlock *types.Block
+ totalBlockNum := 0
for _, blockNum := range blockNums {
- chain := generateBlocksWithAcks(blockNum, maxAcks)
- fillBlocksTimestamps(chain, validatorNum, time.Second, sigma)
- blocksWithTimestamps, mainChain, err := ct.processBlocks(chain)
- s.Require().Nil(err)
- timestamps := extractTimestamps(blocksWithTimestamps)
- if lastMainChainBlock != nil {
- s.Require().Equal(mainChain[0], lastMainChainBlock)
- }
- s.Require().Equal(mainChain[len(mainChain)-1], ct.lastMainChainBlock)
- lastMainChainBlock = ct.lastMainChainBlock
- totalMainChain =
- append(totalMainChain[:len(totalMainChain)-1], mainChain...)
+ totalBlockNum += blockNum
+ }
+ totalChain := s.generateBlocksWithTimestamp(
+ totalBlockNum, validatorNum, time.Second, sigma)
+ for _, blockNum := range blockNums {
+ var chain []*types.Block
+ chain, totalChain = totalChain[:blockNum], totalChain[blockNum:]
+ err := ct.processBlocks(chain)
+ s.Require().NoError(err)
+ timestamps := s.extractTimestamps(chain)
totalChain = append(totalChain, chain...)
totalTimestamps = append(totalTimestamps, timestamps...)
}
ct2 := newConsensusTimestamp()
- blocksWithTimestamps2, mainChain2, err := ct2.processBlocks(totalChain)
- s.Require().Nil(err)
- timestamps2 := extractTimestamps(blocksWithTimestamps2)
- s.Equal(totalMainChain, mainChain2)
+ err := ct2.processBlocks(totalChain)
+ s.Require().NoError(err)
+ timestamps2 := s.extractTimestamps(totalChain)
s.Equal(totalTimestamps, timestamps2)
}
-func timeDiffWithinTolerance(t1, t2 time.Time, tolerance time.Duration) bool {
- if t1.After(t2) {
- return timeDiffWithinTolerance(t2, t1, tolerance)
- }
- return t1.Add(tolerance).After(t2)
-}
-
func (s *ConsensusTimestampTest) TestTimestampIncrease() {
validatorNum := 19
sigma := 100 * time.Millisecond
ct := newConsensusTimestamp()
- chain := generateBlocksWithAcks(1000, 5)
- fillBlocksTimestamps(chain, validatorNum, time.Second, sigma)
- blocksWithTimestamps, _, err := ct.processBlocks(chain)
- s.Require().Nil(err)
- timestamps := extractTimestamps(blocksWithTimestamps)
+ chain := s.generateBlocksWithTimestamp(1000, validatorNum, time.Second, sigma)
+ err := ct.processBlocks(chain)
+ s.Require().NoError(err)
+ timestamps := s.extractTimestamps(chain)
for i := 1; i < len(timestamps); i++ {
- s.True(timestamps[i].After(timestamps[i-1]))
+ s.False(timestamps[i].Before(timestamps[i-1]))
}
// Test if the processBlocks is stable.
ct2 := newConsensusTimestamp()
- blocksWithTimestamps2, _, err := ct2.processBlocks(chain)
- s.Require().Nil(err)
- timestamps2 := extractTimestamps(blocksWithTimestamps2)
+ ct2.processBlocks(chain)
+ s.Require().NoError(err)
+ timestamps2 := s.extractTimestamps(chain)
s.Equal(timestamps, timestamps2)
}
-func (s *ConsensusTimestampTest) TestByzantineBiasTime() {
- // Test that Byzantine node cannot bias the timestamps.
- validatorNum := 19
- sigma := 100 * time.Millisecond
- tolerance := 4 * sigma
- ct := newConsensusTimestamp()
- chain := generateBlocksWithAcks(1000, 5)
- fillBlocksTimestamps(chain, validatorNum, time.Second, sigma)
- blocksWithTimestamps, _, err := ct.processBlocks(chain)
- s.Require().Nil(err)
- timestamps := extractTimestamps(blocksWithTimestamps)
- byzantine := validatorNum / 3
- validators := make([]types.ValidatorID, 0, validatorNum)
- for vID := range chain[0].Timestamps {
- validators = append(validators, vID)
- }
- // The number of Byzantine node is at most N/3.
- for i := 0; i < byzantine; i++ {
- // Pick one validator to be Byzantine node.
- // It is allowed to have the vID be duplicated,
- // because the number of Byzantine node is between 1 and N/3.
- vID := validators[rand.Intn(validatorNum)]
- for _, block := range chain {
- block.Timestamps[vID] = time.Time{}
- }
- }
- ctByzantine := newConsensusTimestamp()
- blocksWithTimestampsB, _, err := ctByzantine.processBlocks(chain)
- s.Require().Nil(err)
- timestampsWithByzantine := extractTimestamps(blocksWithTimestampsB)
- for idx, timestamp := range timestamps {
- timestampWithByzantine := timestampsWithByzantine[idx]
- s.True(timeDiffWithinTolerance(
- timestamp, timestampWithByzantine, tolerance))
- }
-}
-
func TestConsensusTimestamp(t *testing.T) {
suite.Run(t, new(ConsensusTimestampTest))
}
diff --git a/core/consensus.go b/core/consensus.go
index 6d74bd7..a9ec6a5 100644
--- a/core/consensus.go
+++ b/core/consensus.go
@@ -467,8 +467,7 @@ func (con *Consensus) ProcessBlock(block *types.Block) (err error) {
}
con.app.TotalOrderingDeliver(hashes, earlyDelivered)
// Perform timestamp generation.
- deliveredBlocks, _, err = con.ctModule.processBlocks(
- deliveredBlocks)
+ err = con.ctModule.processBlocks(deliveredBlocks)
if err != nil {
return
}
@@ -518,7 +517,7 @@ func (con *Consensus) PrepareBlock(b *types.Block,
defer con.lock.RUnlock()
con.rbModule.prepareBlock(b)
- b.Timestamps[b.ProposerID] = proposeTime
+ b.Timestamp = proposeTime
b.Payloads = con.app.PreparePayloads(b.Position)
b.Hash, err = hashBlock(b)
if err != nil {
@@ -544,11 +543,7 @@ func (con *Consensus) PrepareGenesisBlock(b *types.Block,
b.Position.Height = 0
b.ParentHash = common.Hash{}
b.Acks = make(map[common.Hash]struct{})
- b.Timestamps = make(map[types.ValidatorID]time.Time)
- for vID := range con.gov.GetValidatorSet() {
- b.Timestamps[vID] = time.Time{}
- }
- b.Timestamps[b.ProposerID] = proposeTime
+ b.Timestamp = proposeTime
b.Hash, err = hashBlock(b)
if err != nil {
return
diff --git a/core/consensus_test.go b/core/consensus_test.go
index aec0e4c..a66a459 100644
--- a/core/consensus_test.go
+++ b/core/consensus_test.go
@@ -290,7 +290,12 @@ func (s *ConsensusTestSuite) TestSimpleDeliverBlock() {
// Check timestamps, there is no direct way to know which block is
// selected as main chain, we can only detect it by making sure
// its ConsensusTimestamp is not interpolated.
- t, err := getMedianTime(b11)
+ timestamps := make([]time.Time, 4)
+ timestamps[0] = b00.Timestamp
+ timestamps[1] = b10.Timestamp
+ timestamps[2] = b20.Timestamp
+ timestamps[3] = b30.Timestamp
+ t, err := getMedianTime(timestamps)
req.Nil(err)
req.Equal(t, app.Delivered[b11.Hash].ConsensusTime)
}
@@ -355,8 +360,8 @@ func (s *ConsensusTestSuite) TestPrepareBlock() {
// Make sure we would assign 'now' to the timestamp belongs to
// the proposer.
req.True(
- b11.Timestamps[validators[1]].Sub(
- b10.Timestamps[validators[1]]) > 100*time.Millisecond)
+ b11.Timestamp.Sub(
+ b10.Timestamp) > 100*time.Millisecond)
for _, obj := range objs {
con := obj.con
req.Nil(con.ProcessBlock(b11))
diff --git a/core/crypto.go b/core/crypto.go
index f097703..656dd74 100644
--- a/core/crypto.go
+++ b/core/crypto.go
@@ -62,22 +62,10 @@ func hashBlock(block *types.Block) (common.Hash, error) {
binaryAcks[idx] = acks[idx][:]
}
hashAcks := crypto.Keccak256Hash(binaryAcks...)
- // Handle Block.Timestamps.
- // TODO(jimmy-dexon): Store and get the sorted validatorIDs.
- validators := make(types.ValidatorIDs, 0, len(block.Timestamps))
- for vID := range block.Timestamps {
- validators = append(validators, vID)
- }
- sort.Sort(validators)
- binaryTimestamps := make([][]byte, len(block.Timestamps))
- for idx, vID := range validators {
- var err error
- binaryTimestamps[idx], err = block.Timestamps[vID].MarshalBinary()
- if err != nil {
- return common.Hash{}, err
- }
+ binaryTimestamp, err := block.Timestamp.MarshalBinary()
+ if err != nil {
+ return common.Hash{}, err
}
- hashTimestamps := crypto.Keccak256Hash(binaryTimestamps...)
payloadHash := crypto.Keccak256Hash(block.Payloads...)
hash := crypto.Keccak256Hash(
@@ -85,7 +73,7 @@ func hashBlock(block *types.Block) (common.Hash, error) {
block.ParentHash[:],
hashPosition[:],
hashAcks[:],
- hashTimestamps[:],
+ binaryTimestamp[:],
payloadHash[:])
return hash, nil
}
diff --git a/core/crypto_test.go b/core/crypto_test.go
index 4fa0e1b..29a45f6 100644
--- a/core/crypto_test.go
+++ b/core/crypto_test.go
@@ -36,12 +36,11 @@ var myVID = types.ValidatorID{Hash: common.NewRandomHash()}
func (s *CryptoTestSuite) prepareBlock(prevBlock *types.Block) *types.Block {
acks := make(map[common.Hash]struct{})
- timestamps := make(map[types.ValidatorID]time.Time)
- timestamps[myVID] = time.Now().UTC()
+ now := time.Now().UTC()
if prevBlock == nil {
return &types.Block{
- Acks: acks,
- Timestamps: timestamps,
+ Acks: acks,
+ Timestamp: now,
Notary: types.Notary{
Timestamp: time.Now(),
Height: 0,
@@ -55,7 +54,7 @@ func (s *CryptoTestSuite) prepareBlock(prevBlock *types.Block) *types.Block {
return &types.Block{
ParentHash: prevBlock.Hash,
Acks: acks,
- Timestamps: timestamps,
+ Timestamp: now,
Position: types.Position{
Height: prevBlock.Position.Height + 1,
},
diff --git a/core/reliable-broadcast.go b/core/reliable-broadcast.go
index 7cca4f1..7d3ba2d 100644
--- a/core/reliable-broadcast.go
+++ b/core/reliable-broadcast.go
@@ -140,22 +140,11 @@ func (rb *reliableBroadcast) sanityCheck(b *types.Block) error {
}
}
- // TODO(jimmy-dexon): verify the timestamps.
- /*
- // Check if its timestamp is valid.
- for h := range rb.lattice {
- if _, exist := b.Timestamps[h]; !exist {
- return ErrInvalidTimestamp
- }
- }
- */
-
+ // Check if its timestamp is valid.
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
- }
+ if !b.Timestamp.After(bParent.Timestamp) {
+ return ErrInvalidTimestamp
}
}
@@ -370,36 +359,6 @@ func (rb *reliableBroadcast) prepareBlock(block *types.Block) {
// Reset fields to make sure we got these information from parent block.
block.Position.Height = 0
block.ParentHash = common.Hash{}
- // The helper function to accumulate timestamps.
- accumulateTimestamps := func(
- times map[types.ValidatorID]time.Time, b *types.Block) {
-
- // Update timestamps with the block's proposer time.
- // TODO (mission): make epslon configurable.
- times[b.ProposerID] = b.Timestamps[b.ProposerID].Add(
- 1 * time.Millisecond)
-
- // Update timestamps from the block if it's later than
- // current cached ones.
- for vID, t := range b.Timestamps {
- cachedTime, exists := times[vID]
- if !exists {
- // This means the block contains timestamps from
- // removed validators.
- continue
- }
- if cachedTime.After(t) {
- continue
- }
- times[vID] = t
- }
- return
- }
- // Initial timestamps with current validator set.
- times := make(map[types.ValidatorID]time.Time)
- for vID := range rb.validators {
- times[vID] = time.Time{}
- }
acks := make(map[common.Hash]struct{})
for chainID := range rb.lattice {
// find height of the latest block for that validator.
@@ -420,15 +379,17 @@ func (rb *reliableBroadcast) prepareBlock(block *types.Block) {
continue
}
acks[curBlock.Hash] = struct{}{}
- accumulateTimestamps(times, curBlock)
if uint32(chainID) == block.Position.ChainID {
block.ParentHash = curBlock.Hash
+ if block.Timestamp.Before(curBlock.Timestamp) {
+ // TODO (mission): make epslon configurable.
+ block.Timestamp = curBlock.Timestamp.Add(1 * time.Millisecond)
+ }
if block.Position.Height == 0 {
block.Position.Height = curBlock.Position.Height + 1
}
}
}
- block.Timestamps = times
block.Acks = acks
return
}
diff --git a/core/reliable-broadcast_test.go b/core/reliable-broadcast_test.go
index 2c71bcb..eedb634 100644
--- a/core/reliable-broadcast_test.go
+++ b/core/reliable-broadcast_test.go
@@ -55,8 +55,8 @@ func (s *ReliableBroadcastTest) prepareGenesisBlock(
Position: types.Position{
Height: 0,
},
- Acks: make(map[common.Hash]struct{}),
- Timestamps: genTimestamps(validatorIDs),
+ Acks: make(map[common.Hash]struct{}),
+ Timestamp: time.Now().UTC(),
}
for i, vID := range validatorIDs {
if proposerID == vID {
@@ -64,24 +64,13 @@ func (s *ReliableBroadcastTest) prepareGenesisBlock(
break
}
}
- for _, vID := range validatorIDs {
- b.Timestamps[vID] = time.Time{}
- }
- b.Timestamps[proposerID] = time.Now().UTC()
+ b.Timestamp = time.Now().UTC()
var err error
b.Hash, err = hashBlock(b)
s.Require().Nil(err)
return
}
-func genTimestamps(vids []types.ValidatorID) map[types.ValidatorID]time.Time {
- ts := make(map[types.ValidatorID]time.Time)
- for _, vid := range vids {
- ts[vid] = time.Now().UTC()
- }
- return ts
-}
-
// genTestCase1 generates test case 1,
// 3
// |
@@ -115,7 +104,7 @@ func genTestCase1(s *ReliableBroadcastTest, rb *reliableBroadcast) []types.Valid
ProposerID: vids[0],
ParentHash: h,
Hash: common.NewRandomHash(),
- Timestamps: genTimestamps(vids),
+ Timestamp: time.Now().UTC(),
Position: types.Position{
ChainID: 0,
Height: 1,
@@ -139,7 +128,7 @@ func genTestCase1(s *ReliableBroadcastTest, rb *reliableBroadcast) []types.Valid
ChainID: 0,
Height: 2,
},
- Timestamps: genTimestamps(vids),
+ Timestamp: time.Now().UTC(),
Acks: map[common.Hash]struct{}{
h: struct{}{},
rb.lattice[1].blocks[0].Hash: struct{}{},
@@ -156,7 +145,7 @@ func genTestCase1(s *ReliableBroadcastTest, rb *reliableBroadcast) []types.Valid
ProposerID: vids[0],
ParentHash: h,
Hash: common.NewRandomHash(),
- Timestamps: genTimestamps(vids),
+ Timestamp: time.Now().UTC(),
Position: types.Position{
ChainID: 0,
Height: 3,
@@ -176,7 +165,7 @@ func genTestCase1(s *ReliableBroadcastTest, rb *reliableBroadcast) []types.Valid
ProposerID: vids[3],
ParentHash: h,
Hash: common.NewRandomHash(),
- Timestamps: genTimestamps(vids),
+ Timestamp: time.Now().UTC(),
Position: types.Position{
ChainID: 3,
Height: 1,
@@ -313,9 +302,7 @@ func (s *ReliableBroadcastTest) TestSanityCheck() {
Acks: map[common.Hash]struct{}{
h: struct{}{},
},
- Timestamps: map[types.ValidatorID]time.Time{
- vids[0]: time.Now().UTC(),
- },
+ Timestamp: time.Now().UTC(),
}
b.Hash, err = hashBlock(b)
s.Require().Nil(err)
@@ -356,7 +343,7 @@ func (s *ReliableBroadcastTest) TestSanityCheck() {
h: struct{}{},
common.NewRandomHash(): struct{}{},
},
- Timestamps: genTimestamps(vids),
+ Timestamp: time.Now().UTC(),
}
b.Hash, err = hashBlock(b)
s.Require().Nil(err)
@@ -415,7 +402,7 @@ func (s *ReliableBroadcastTest) TestStrongAck() {
ChainID: 1,
Height: 1,
},
- Timestamps: genTimestamps(vids),
+ Timestamp: time.Now().UTC(),
Acks: map[common.Hash]struct{}{
rb.lattice[0].blocks[2].Hash: struct{}{},
rb.lattice[1].blocks[0].Hash: struct{}{},
@@ -441,7 +428,7 @@ func (s *ReliableBroadcastTest) TestStrongAck() {
ChainID: 2,
Height: 1,
},
- Timestamps: genTimestamps(vids),
+ Timestamp: time.Now().UTC(),
Acks: map[common.Hash]struct{}{
rb.lattice[0].blocks[2].Hash: struct{}{},
rb.lattice[2].blocks[0].Hash: struct{}{},
@@ -471,7 +458,7 @@ func (s *ReliableBroadcastTest) TestExtractBlocks() {
ChainID: 1,
Height: 1,
},
- Timestamps: genTimestamps(vids),
+ Timestamp: time.Now().UTC(),
Acks: map[common.Hash]struct{}{
rb.lattice[0].blocks[2].Hash: struct{}{},
rb.lattice[1].blocks[0].Hash: struct{}{},
@@ -492,7 +479,7 @@ func (s *ReliableBroadcastTest) TestExtractBlocks() {
ChainID: 2,
Height: 1,
},
- Timestamps: genTimestamps(vids),
+ Timestamp: time.Now().UTC(),
Acks: map[common.Hash]struct{}{
rb.lattice[0].blocks[2].Hash: struct{}{},
rb.lattice[2].blocks[0].Hash: struct{}{},
@@ -556,8 +543,8 @@ func (s *ReliableBroadcastTest) TestRandomIntensiveAcking() {
ChainID: uint32(id),
Height: height,
},
- Timestamps: genTimestamps(vids),
- Acks: acks,
+ Timestamp: time.Now().UTC(),
+ Acks: acks,
}
var err error
b.Hash, err = hashBlock(b)
@@ -687,14 +674,8 @@ func (s *ReliableBroadcastTest) TestPrepareBlock() {
req.Contains(b11.Acks, b10.Hash)
req.Contains(b11.Acks, b20.Hash)
req.Contains(b11.Acks, b30.Hash)
- req.Equal(b11.Timestamps[validators[0]],
- b00.Timestamps[b00.ProposerID].Add(time.Millisecond))
- req.Equal(b11.Timestamps[validators[1]],
- b10.Timestamps[b10.ProposerID].Add(time.Millisecond))
- req.Equal(b11.Timestamps[validators[2]],
- b20.Timestamps[b20.ProposerID].Add(time.Millisecond))
- req.Equal(b11.Timestamps[validators[3]],
- b30.Timestamps[b30.ProposerID].Add(time.Millisecond))
+ req.Equal(b11.Timestamp,
+ b10.Timestamp.Add(time.Millisecond))
req.Equal(b11.ParentHash, b10.Hash)
req.Equal(b11.Position.Height, uint64(1))
s.Require().Nil(rb.processBlock(b11))
diff --git a/core/test/blocks-generator.go b/core/test/blocks-generator.go
index 18e4995..f3e914e 100644
--- a/core/test/blocks-generator.go
+++ b/core/test/blocks-generator.go
@@ -70,25 +70,33 @@ func (vs *validatorStatus) getAckedBlockHash(
// validatorSetStatus is a state holder for all validators
// during generating blocks.
type validatorSetStatus struct {
- status map[types.ValidatorID]*validatorStatus
- validatorIDs []types.ValidatorID
- randGen *rand.Rand
- hashBlock hashBlockFn
+ status map[types.ValidatorID]*validatorStatus
+ proposerChain map[types.ValidatorID]uint32
+ timestamps []time.Time
+ validatorIDs []types.ValidatorID
+ randGen *rand.Rand
+ hashBlock hashBlockFn
}
func newValidatorSetStatus(vIDs []types.ValidatorID, hashBlock hashBlockFn) *validatorSetStatus {
status := make(map[types.ValidatorID]*validatorStatus)
- for _, vID := range vIDs {
+ timestamps := make([]time.Time, 0, len(vIDs))
+ proposerChain := make(map[types.ValidatorID]uint32)
+ for i, vID := range vIDs {
status[vID] = &validatorStatus{
blocks: []*types.Block{},
lastAckingHeight: make(map[types.ValidatorID]uint64),
}
+ timestamps = append(timestamps, time.Now().UTC())
+ proposerChain[vID] = uint32(i)
}
return &validatorSetStatus{
- status: status,
- validatorIDs: vIDs,
- randGen: rand.New(rand.NewSource(time.Now().UnixNano())),
- hashBlock: hashBlock,
+ status: status,
+ proposerChain: proposerChain,
+ timestamps: timestamps,
+ validatorIDs: vIDs,
+ randGen: rand.New(rand.NewSource(time.Now().UnixNano())),
+ hashBlock: hashBlock,
}
}
@@ -150,20 +158,18 @@ func (vs *validatorSetStatus) proposeBlock(
if len(status.blocks) > 0 {
parentHash = status.blocks[len(status.blocks)-1].Hash
}
+ chainID := vs.proposerChain[proposerID]
+ vs.timestamps[chainID] = vs.timestamps[chainID].Add(time.Second)
- ts := map[types.ValidatorID]time.Time{}
- for vid := range vs.status {
- ts[vid] = time.Time{}
- }
newBlock := &types.Block{
ProposerID: proposerID,
ParentHash: parentHash,
Position: types.Position{
- Height: uint64(len(status.blocks)),
+ Height: uint64(len(status.blocks)),
+ ChainID: chainID,
},
- Acks: acks,
- Timestamps: ts,
- // TODO(mission.liao): Generate timestamp.
+ Acks: acks,
+ Timestamp: vs.timestamps[chainID],
}
for i, vID := range vs.validatorIDs {
if vID == proposerID {
diff --git a/core/types/block.go b/core/types/block.go
index f02b1a0..3585a97 100644
--- a/core/types/block.go
+++ b/core/types/block.go
@@ -52,24 +52,19 @@ func NewBlock() (b *Block) {
delete(b.Acks, k)
}
}
- if b.Timestamps != nil {
- for k := range b.Timestamps {
- delete(b.Timestamps, k)
- }
- }
return
}
// Block represents a single event broadcasted on the network.
type Block struct {
- ProposerID ValidatorID `json:"proposer_id"`
- ParentHash common.Hash `json:"parent_hash"`
- Hash common.Hash `json:"hash"`
- Position Position `json:"position"`
- Timestamps map[ValidatorID]time.Time `json:"timestamps"`
- Acks map[common.Hash]struct{} `json:"acks"`
- Payloads [][]byte `json:"payloads"`
- Signature crypto.Signature `json:"signature"`
+ ProposerID ValidatorID `json:"proposer_id"`
+ ParentHash common.Hash `json:"parent_hash"`
+ Hash common.Hash `json:"hash"`
+ Position Position `json:"position"`
+ Timestamp time.Time `json:"timestamps"`
+ Acks map[common.Hash]struct{} `json:"acks"`
+ Payloads [][]byte `json:"payloads"`
+ Signature crypto.Signature `json:"signature"`
CRSSignature crypto.Signature `json:"crs_signature"`
@@ -94,13 +89,7 @@ func (b *Block) Clone() (bcopy *Block) {
bcopy.CRSSignature = b.CRSSignature.Clone()
bcopy.Notary.Timestamp = b.Notary.Timestamp
bcopy.Notary.Height = b.Notary.Height
- if bcopy.Timestamps == nil {
- bcopy.Timestamps = make(
- map[ValidatorID]time.Time, len(b.Timestamps))
- }
- for k, v := range b.Timestamps {
- bcopy.Timestamps[k] = v
- }
+ bcopy.Timestamp = b.Timestamp
if bcopy.Acks == nil {
bcopy.Acks = make(map[common.Hash]struct{}, len(b.Acks))
}
diff --git a/core/utils.go b/core/utils.go
index 6b03f93..59aa7a4 100644
--- a/core/utils.go
+++ b/core/utils.go
@@ -25,7 +25,6 @@ import (
"time"
"github.com/dexon-foundation/dexon-consensus-core/common"
- "github.com/dexon-foundation/dexon-consensus-core/core/types"
)
var (
@@ -72,25 +71,25 @@ func interpoTime(t1 time.Time, t2 time.Time, sep int) []time.Time {
}
return timestamps
}
-func getMedianTime(block *types.Block) (t time.Time, err error) {
- timestamps := []time.Time{}
- for _, timestamp := range block.Timestamps {
- timestamps = append(timestamps, timestamp)
- }
+
+func getMedianTime(timestamps []time.Time) (t time.Time, err error) {
if len(timestamps) == 0 {
err = ErrEmptyTimestamps
return
}
- sort.Sort(common.ByTime(timestamps))
- if len(timestamps)%2 == 0 {
- t1 := timestamps[len(timestamps)/2-1]
- t2 := timestamps[len(timestamps)/2]
+ tscopy := make([]time.Time, 0, len(timestamps))
+ for _, ts := range timestamps {
+ tscopy = append(tscopy, ts)
+ }
+ sort.Sort(common.ByTime(tscopy))
+ if len(tscopy)%2 == 0 {
+ t1 := tscopy[len(tscopy)/2-1]
+ t2 := tscopy[len(tscopy)/2]
t = interpoTime(t1, t2, 1)[0]
} else {
- t = timestamps[len(timestamps)/2]
+ t = tscopy[len(tscopy)/2]
}
return
-
}
func removeFromSortedIntSlice(xs []int, x int) []int {
diff --git a/simulation/app.go b/simulation/app.go
index 4bf4aff..5a31273 100644
--- a/simulation/app.go
+++ b/simulation/app.go
@@ -124,7 +124,7 @@ func (a *simApp) TotalOrderingDeliver(blockHashes common.Hashes, early bool) {
for _, block := range blocks {
if block.ProposerID == a.ValidatorID {
confirmLatency = append(confirmLatency,
- now.Sub(block.Timestamps[a.ValidatorID]))
+ now.Sub(block.Timestamp))
}
for hash := range block.Acks {
for _, blockHash := range a.getAckedBlocks(hash) {