aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorMission Liao <mission.liao@dexon.org>2019-02-15 14:18:59 +0800
committerJimmy Hu <jimmy.hu@dexon.org>2019-02-19 10:48:50 +0800
commit4dbdc22e355cf1f6f0c39af1b2f3737b7527bc0c (patch)
tree625b7d34aa700d072ffb8e68dc89ed3936b76d29
parente4825619fb2499f5f534537c1a4d52d3e0bcacfe (diff)
downloaddexon-consensus-4dbdc22e355cf1f6f0c39af1b2f3737b7527bc0c.tar
dexon-consensus-4dbdc22e355cf1f6f0c39af1b2f3737b7527bc0c.tar.gz
dexon-consensus-4dbdc22e355cf1f6f0c39af1b2f3737b7527bc0c.tar.bz2
dexon-consensus-4dbdc22e355cf1f6f0c39af1b2f3737b7527bc0c.tar.lz
dexon-consensus-4dbdc22e355cf1f6f0c39af1b2f3737b7527bc0c.tar.xz
dexon-consensus-4dbdc22e355cf1f6f0c39af1b2f3737b7527bc0c.tar.zst
dexon-consensus-4dbdc22e355cf1f6f0c39af1b2f3737b7527bc0c.zip
big-bang: single chain (#446)
-rw-r--r--.circleci/config.yml3
-rw-r--r--GNUmakefile3
-rw-r--r--cmd/dexcon-simulation-with-scheduler/main.go139
-rw-r--r--common/utils.go10
-rw-r--r--core/agreement-mgr.go260
-rw-r--r--core/agreement.go10
-rw-r--r--core/blockchain.go617
-rw-r--r--core/blockchain_test.go462
-rw-r--r--core/blockpool.go79
-rw-r--r--core/blockpool_test.go83
-rw-r--r--core/compaction-chain.go291
-rw-r--r--core/compaction-chain_test.go343
-rw-r--r--core/consensus-timestamp.go162
-rw-r--r--core/consensus-timestamp_test.go221
-rw-r--r--core/consensus.go301
-rw-r--r--core/consensus_test.go322
-rw-r--r--core/db/level-db_test.go8
-rw-r--r--core/db/memory_test.go8
-rw-r--r--core/dkg-tsig-protocol.go2
-rw-r--r--core/interfaces.go3
-rw-r--r--core/lattice-data.go683
-rw-r--r--core/lattice-data_test.go707
-rw-r--r--core/lattice.go363
-rw-r--r--core/lattice_test.go262
-rw-r--r--core/negative-ack.go211
-rw-r--r--core/negative-ack_test.go228
-rw-r--r--core/nonblocking.go16
-rw-r--r--core/nonblocking_test.go23
-rw-r--r--core/syncer/consensus.go572
-rw-r--r--core/test/app.go152
-rw-r--r--core/test/app_test.go220
-rw-r--r--core/test/block-revealer.go214
-rw-r--r--core/test/block-revealer_test.go131
-rw-r--r--core/test/blocks-generator.go379
-rw-r--r--core/test/blocks-generator_test.go323
-rw-r--r--core/test/interface.go16
-rw-r--r--core/test/network.go31
-rw-r--r--core/test/scheduler-event.go79
-rw-r--r--core/test/scheduler.go215
-rw-r--r--core/test/scheduler_test.go176
-rw-r--r--core/test/stopper.go140
-rw-r--r--core/test/stopper_test.go182
-rw-r--r--core/total-ordering-syncer.go177
-rw-r--r--core/total-ordering-syncer_test.go312
-rw-r--r--core/total-ordering.go1321
-rw-r--r--core/total-ordering_test.go1453
-rw-r--r--core/types/block-randomness.go4
-rw-r--r--core/types/block.go32
-rw-r--r--core/types/block_test.go19
-rw-r--r--core/types/nodeset.go18
-rw-r--r--core/types/nodeset_test.go2
-rw-r--r--core/types/position.go17
-rw-r--r--core/types/position_test.go60
-rw-r--r--core/types/vote.go2
-rw-r--r--core/utils/nodeset-cache.go55
-rw-r--r--core/utils/nodeset-cache_test.go2
-rw-r--r--integration_test/consensus_test.go38
-rw-r--r--integration_test/node.go355
-rw-r--r--integration_test/stats.go192
-rw-r--r--integration_test/stats_test.go84
-rw-r--r--integration_test/utils.go140
-rw-r--r--integration_test/utils_test.go61
-rw-r--r--integration_test/with_scheduler_test.go150
-rw-r--r--simulation/app.go23
-rw-r--r--simulation/block-list.go64
-rw-r--r--simulation/marshaller.go8
-rw-r--r--simulation/peer-server.go53
-rw-r--r--simulation/verification.go260
68 files changed, 1689 insertions, 11863 deletions
diff --git a/.circleci/config.yml b/.circleci/config.yml
index b069ec3..08ff726 100644
--- a/.circleci/config.yml
+++ b/.circleci/config.yml
@@ -20,6 +20,8 @@ commands:
command: |
if [ "${CIRCLE_BRANCH}" == "master" ]; then
make test
+ elif [ "${CIRCLE_BRANCH}" == "single-chain" ]; then
+ make test
else
make test-short
fi
@@ -128,6 +130,7 @@ workflows:
branches:
only:
- master
+ - single-chain
test_and_build:
jobs:
diff --git a/GNUmakefile b/GNUmakefile
index 64926a1..425a569 100644
--- a/GNUmakefile
+++ b/GNUmakefile
@@ -61,8 +61,7 @@ endif
COMPONENTS = \
dexcon-simulation \
- dexcon-simulation-peer-server \
- dexcon-simulation-with-scheduler
+ dexcon-simulation-peer-server
.PHONY: clean default
diff --git a/cmd/dexcon-simulation-with-scheduler/main.go b/cmd/dexcon-simulation-with-scheduler/main.go
deleted file mode 100644
index d06e6e0..0000000
--- a/cmd/dexcon-simulation-with-scheduler/main.go
+++ /dev/null
@@ -1,139 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package main
-
-import (
- "flag"
- "log"
- "math"
- "math/rand"
- "net/http"
- _ "net/http/pprof"
- "os"
- "runtime"
- "runtime/pprof"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core"
- "github.com/dexon-foundation/dexon-consensus/core/test"
- integration "github.com/dexon-foundation/dexon-consensus/integration_test"
- "github.com/dexon-foundation/dexon-consensus/simulation/config"
-)
-
-var (
- configFile = flag.String("config", "", "path to simulation config file")
- cpuprofile = flag.String("cpuprofile", "", "write cpu profile to `file`")
- memprofile = flag.String("memprofile", "", "write memory profile to `file`")
-)
-
-func main() {
- flag.Parse()
- rand.Seed(time.Now().UnixNano())
- // Supports runtime pprof monitoring.
- go func() {
- log.Println(http.ListenAndServe("localhost:6060", nil))
- }()
- if *configFile == "" {
- log.Fatal("error: no configuration file specified")
- }
- if *cpuprofile != "" {
- f, err := os.Create(*cpuprofile)
- if err != nil {
- log.Fatal("could not create CPU profile: ", err)
- }
- if err := pprof.StartCPUProfile(f); err != nil {
- log.Fatal("could not start CPU profile: ", err)
- }
- defer pprof.StopCPUProfile()
- }
-
- cfg, err := config.Read(*configFile)
- if err != nil {
- log.Fatal("unable to read config: ", err)
- }
- // Setup latencies, nodes.
- networkLatency := &test.NormalLatencyModel{
- Sigma: cfg.Networking.Direct.Sigma,
- Mean: cfg.Networking.Direct.Mean,
- }
- proposingLatency := &test.NormalLatencyModel{
- Sigma: cfg.Node.Legacy.ProposeIntervalSigma,
- Mean: cfg.Node.Legacy.ProposeIntervalMean,
- }
- // Setup key pairs.
- prvKeys, pubKeys, err := test.NewKeys(int(cfg.Node.Num))
- if err != nil {
- log.Fatal("could not setup key pairs: ", err)
- }
- // Setup governance instance.
- gov, err := test.NewGovernance(
- test.NewState(
- pubKeys,
- time.Duration(cfg.Networking.Direct.Mean)*time.Millisecond,
- &common.NullLogger{},
- true,
- ), core.ConfigRoundShift)
- if err != nil {
- log.Fatal("could not setup governance: ", err)
- }
- // Setup nodes and other consensus related stuffs.
- nodes, err := integration.PrepareNodes(
- gov, prvKeys, uint32(cfg.Node.Num), networkLatency, proposingLatency)
- if err != nil {
- log.Fatal("could not setup nodes: ", err)
- }
- apps, dbs := integration.CollectAppAndDBFromNodes(nodes)
- blockPerNode := int(math.Ceil(
- float64(cfg.Node.MaxBlock) / float64(cfg.Node.Num)))
- sch := test.NewScheduler(
- test.NewStopByConfirmedBlocks(blockPerNode, apps, dbs))
- now := time.Now().UTC()
- for _, v := range nodes {
- if err := v.Bootstrap(sch, now); err != nil {
- panic(err)
- }
- }
- // Run the simulation.
- sch.Run(cfg.Scheduler.WorkerNum)
- if err = integration.VerifyApps(apps); err != nil {
- log.Fatal("consensus result is not incorrect: ", err)
- }
- // Prepare statistics.
- stats, err := integration.NewStats(sch.CloneExecutionHistory(), apps)
- if err != nil {
- log.Fatal("could not generate statistics: ", err)
- }
- if *memprofile != "" {
- f, err := os.Create(*memprofile)
- if err != nil {
- log.Fatal("could not create memory profile: ", err)
- }
- runtime.GC() // get up-to-date statistics
- if err := pprof.WriteHeapProfile(f); err != nil {
- log.Fatal("could not write memory profile: ", err)
- }
- // #nosec G104
- f.Close()
- }
-
- log.Printf("BPS: %v\n", stats.BPS)
- log.Printf("ExecutionTime: %v\n", stats.ExecutionTime)
- log.Printf("Prepare: %v\n", time.Duration(stats.All.PrepareExecLatency))
- log.Printf("Process: %v\n", time.Duration(stats.All.ProcessExecLatency))
-}
diff --git a/common/utils.go b/common/utils.go
index 63d25a3..e46b3e9 100644
--- a/common/utils.go
+++ b/common/utils.go
@@ -19,3 +19,13 @@ func NewRandomHash() Hash {
}
return x
}
+
+// GenerateRandomBytes generates bytes randomly.
+func GenerateRandomBytes() []byte {
+ randomness := make([]byte, 32)
+ _, err := rand.Read(randomness)
+ if err != nil {
+ panic(err)
+ }
+ return randomness
+}
diff --git a/core/agreement-mgr.go b/core/agreement-mgr.go
index c3692d8..5f5b9ae 100644
--- a/core/agreement-mgr.go
+++ b/core/agreement-mgr.go
@@ -33,6 +33,8 @@ import (
// Errors returned from BA modules
var (
ErrPreviousRoundIsNotFinished = errors.New("previous round is not finished")
+ ErrRoundOutOfRange = errors.New("round out of range")
+ ErrInvalidBlock = errors.New("invalid block")
)
const maxResultCache = 100
@@ -44,7 +46,7 @@ func genValidLeader(
if block.Timestamp.After(time.Now()) {
return false, nil
}
- if err := mgr.lattice.SanityCheck(block, true); err != nil {
+ if err := mgr.bcModule.sanityCheck(block); err != nil {
if err == ErrRetrySanityCheckLater {
return false, nil
}
@@ -64,7 +66,6 @@ func genValidLeader(
type agreementMgrConfig struct {
beginTime time.Time
- numChains uint32
roundInterval time.Duration
notarySetSize uint32
lambdaBA time.Duration
@@ -72,7 +73,6 @@ type agreementMgrConfig struct {
}
type baRoundSetting struct {
- chainID uint32
notarySet map[types.NodeID]struct{}
agr *agreement
recv *consensusBAReceiver
@@ -90,28 +90,17 @@ type agreementMgr struct {
logger common.Logger
cache *utils.NodeSetCache
signer *utils.Signer
- lattice *Lattice
+ bcModule *blockChain
ctx context.Context
lastEndTime time.Time
initRound uint64
configs []*agreementMgrConfig
- baModules []*agreement
+ baModule *agreement
processedBAResult map[types.Position]struct{}
- voteFilters []*utils.VoteFilter
+ voteFilter *utils.VoteFilter
waitGroup sync.WaitGroup
- pendingVotes map[uint64][]*types.Vote
- pendingBlocks map[uint64][]*types.Block
isRunning bool
-
- // This lock should be used when attempting to:
- // - add a new baModule.
- // - remove all baModules when stopping. In this case, the cleaner need
- // to wait for all routines runnning baModules finished.
- // - access a method of baModule.
- // - append a config from new round.
- // The routine running corresponding baModule, however, doesn't have to
- // acquire this lock.
- lock sync.RWMutex
+ lock sync.RWMutex
}
func newAgreementMgr(con *Consensus, initRound uint64,
@@ -125,7 +114,7 @@ func newAgreementMgr(con *Consensus, initRound uint64,
logger: con.logger,
cache: con.nodeSetCache,
signer: con.signer,
- lattice: con.lattice,
+ bcModule: con.bcModule,
ctx: con.ctx,
initRound: initRound,
lastEndTime: initRoundBeginTime,
@@ -133,6 +122,20 @@ func newAgreementMgr(con *Consensus, initRound uint64,
}
}
+func (mgr *agreementMgr) run() {
+ mgr.lock.Lock()
+ defer mgr.lock.Unlock()
+ if mgr.isRunning {
+ return
+ }
+ mgr.isRunning = true
+ mgr.waitGroup.Add(1)
+ go func() {
+ defer mgr.waitGroup.Done()
+ mgr.runBA(mgr.initRound)
+ }()
+}
+
func (mgr *agreementMgr) getConfig(round uint64) *agreementMgrConfig {
mgr.lock.RLock()
defer mgr.lock.RUnlock()
@@ -146,22 +149,6 @@ func (mgr *agreementMgr) getConfig(round uint64) *agreementMgrConfig {
return mgr.configs[roundIndex]
}
-func (mgr *agreementMgr) run() {
- mgr.lock.Lock()
- defer mgr.lock.Unlock()
- if mgr.isRunning {
- return
- }
- mgr.isRunning = true
- for i := uint32(0); i < uint32(len(mgr.baModules)); i++ {
- mgr.waitGroup.Add(1)
- go func(idx uint32) {
- defer mgr.waitGroup.Done()
- mgr.runBA(mgr.initRound, idx)
- }(i)
- }
-}
-
func (mgr *agreementMgr) appendConfig(
round uint64, config *types.Config, crs common.Hash) (err error) {
mgr.lock.Lock()
@@ -171,7 +158,6 @@ func (mgr *agreementMgr) appendConfig(
}
newConfig := &agreementMgrConfig{
beginTime: mgr.lastEndTime,
- numChains: config.NumChains,
roundInterval: config.RoundInterval,
notarySetSize: config.NotarySetSize,
lambdaBA: config.LambdaBA,
@@ -179,80 +165,61 @@ func (mgr *agreementMgr) appendConfig(
}
mgr.configs = append(mgr.configs, newConfig)
mgr.lastEndTime = mgr.lastEndTime.Add(config.RoundInterval)
- // Create baModule for newly added chain.
- for i := uint32(len(mgr.baModules)); i < newConfig.numChains; i++ {
- // Prepare modules.
- recv := &consensusBAReceiver{
- consensus: mgr.con,
- chainID: i,
- restartNotary: make(chan types.Position, 1),
- roundValue: &atomic.Value{},
- }
- recv.roundValue.Store(uint64(0))
- agrModule := newAgreement(
- mgr.con.ID,
- recv,
- newLeaderSelector(genValidLeader(mgr), mgr.logger),
- mgr.signer,
- mgr.logger)
- // Hacky way to initialize first notarySet.
- nodes, err := mgr.cache.GetNodeSet(round)
- if err != nil {
- return err
- }
- agrModule.notarySet = nodes.GetSubSet(
- int(config.NotarySetSize),
- types.NewNotarySetTarget(crs, i))
- // Hacky way to make agreement module self contained.
- recv.agreementModule = agrModule
- mgr.baModules = append(mgr.baModules, agrModule)
- mgr.voteFilters = append(mgr.voteFilters, utils.NewVoteFilter())
- if mgr.isRunning {
- mgr.waitGroup.Add(1)
- go func(idx uint32) {
- defer mgr.waitGroup.Done()
- mgr.runBA(round, idx)
- }(i)
- }
+ // Prepare modules.
+ if mgr.baModule != nil {
+ return nil
+ }
+ recv := &consensusBAReceiver{
+ consensus: mgr.con,
+ restartNotary: make(chan types.Position, 1),
+ roundValue: &atomic.Value{},
}
+ recv.roundValue.Store(uint64(0))
+ agrModule := newAgreement(
+ mgr.con.ID,
+ recv,
+ newLeaderSelector(genValidLeader(mgr), mgr.logger),
+ mgr.signer,
+ mgr.logger)
+ // Hacky way to initialize first notarySet.
+ nodes, err := mgr.cache.GetNodeSet(round)
+ if err != nil {
+ return err
+ }
+ agrModule.notarySet = nodes.GetSubSet(
+ int(config.NotarySetSize), types.NewNotarySetTarget(crs))
+ // Hacky way to make agreement module self contained.
+ recv.agreementModule = agrModule
+ mgr.baModule = agrModule
+ mgr.voteFilter = utils.NewVoteFilter()
return nil
}
-func (mgr *agreementMgr) processVote(v *types.Vote) error {
- mgr.lock.RLock()
- defer mgr.lock.RUnlock()
- if v.Position.ChainID >= uint32(len(mgr.baModules)) {
+func (mgr *agreementMgr) processVote(v *types.Vote) (err error) {
+ if v.Position.ChainID > 0 {
mgr.logger.Error("Process vote for unknown chain to BA",
- "position", &v.Position,
- "baChain", len(mgr.baModules),
- "baRound", len(mgr.configs),
+ "position", v.Position,
"initRound", mgr.initRound)
return utils.ErrInvalidChainID
}
- filter := mgr.voteFilters[v.Position.ChainID]
- if filter.Filter(v) {
+ if mgr.voteFilter.Filter(v) {
return nil
}
- v = v.Clone()
- err := mgr.baModules[v.Position.ChainID].processVote(v)
- if err == nil {
- mgr.baModules[v.Position.ChainID].updateFilter(filter)
+ if err = mgr.baModule.processVote(v); err == nil {
+ mgr.baModule.updateFilter(mgr.voteFilter)
}
- return err
+ return
}
func (mgr *agreementMgr) processBlock(b *types.Block) error {
- mgr.lock.RLock()
- defer mgr.lock.RUnlock()
- if b.Position.ChainID >= uint32(len(mgr.baModules)) {
+ if b.Position.ChainID > 0 {
mgr.logger.Error("Process block for unknown chain to BA",
- "position", &b.Position,
- "baChain", len(mgr.baModules),
+ "position", b.Position,
"baRound", len(mgr.configs),
"initRound", mgr.initRound)
return utils.ErrInvalidChainID
}
- return mgr.baModules[b.Position.ChainID].processBlock(b)
+ return mgr.baModule.processBlock(b)
}
func (mgr *agreementMgr) touchAgreementResult(
@@ -280,30 +247,26 @@ func (mgr *agreementMgr) untouchAgreementResult(
func (mgr *agreementMgr) processAgreementResult(
result *types.AgreementResult) error {
- mgr.lock.RLock()
- defer mgr.lock.RUnlock()
- if result.Position.ChainID >= uint32(len(mgr.baModules)) {
+ if result.Position.ChainID > 0 {
mgr.logger.Error("Process unknown result for unknown chain to BA",
- "position", &result.Position,
- "baChain", len(mgr.baModules),
+ "position", result.Position,
"baRound", len(mgr.configs),
"initRound", mgr.initRound)
return utils.ErrInvalidChainID
}
- agreement := mgr.baModules[result.Position.ChainID]
- aID := agreement.agreementID()
+ aID := mgr.baModule.agreementID()
if isStop(aID) {
return nil
}
- if result.Position == aID && !agreement.confirmed() {
- mgr.logger.Info("Syncing BA", "position", &result.Position)
+ if result.Position == aID && !mgr.baModule.confirmed() {
+ mgr.logger.Info("Syncing BA", "position", result.Position)
for key := range result.Votes {
- if err := agreement.processVote(&result.Votes[key]); err != nil {
+ if err := mgr.baModule.processVote(&result.Votes[key]); err != nil {
return err
}
}
- } else if result.Position.Newer(&aID) {
- mgr.logger.Info("Fast syncing BA", "position", &result.Position)
+ } else if result.Position.Newer(aID) {
+ mgr.logger.Info("Fast syncing BA", "position", result.Position)
nodes, err := mgr.cache.GetNodeSet(result.Position.Round)
if err != nil {
return err
@@ -316,9 +279,9 @@ func (mgr *agreementMgr) processAgreementResult(
nIDs := nodes.GetSubSet(
int(utils.GetConfigWithPanic(
mgr.gov, result.Position.Round, mgr.logger).NotarySetSize),
- types.NewNotarySetTarget(crs, result.Position.ChainID))
+ types.NewNotarySetTarget(crs))
for key := range result.Votes {
- if err := agreement.processVote(&result.Votes[key]); err != nil {
+ if err := mgr.baModule.processVote(&result.Votes[key]); err != nil {
return err
}
}
@@ -326,7 +289,7 @@ func (mgr *agreementMgr) processAgreementResult(
if err != nil {
return err
}
- agreement.restart(nIDs, result.Position, leader, crs)
+ mgr.baModule.restart(nIDs, result.Position, leader, crs)
}
return nil
}
@@ -336,30 +299,20 @@ func (mgr *agreementMgr) stop() {
func() {
mgr.lock.Lock()
defer mgr.lock.Unlock()
- for _, agr := range mgr.baModules {
- agr.stop()
- }
+ mgr.baModule.stop()
}()
// Block until all routines are done.
mgr.waitGroup.Wait()
}
-func (mgr *agreementMgr) runBA(initRound uint64, chainID uint32) {
- // Acquire agreement module.
- agr, recv := func() (*agreement, *consensusBAReceiver) {
- mgr.lock.RLock()
- defer mgr.lock.RUnlock()
- agr := mgr.baModules[chainID]
- return agr, agr.data.recv.(*consensusBAReceiver)
- }()
+func (mgr *agreementMgr) runBA(initRound uint64) {
// These are round based variables.
var (
currentRound uint64
nextRound = initRound
setting = baRoundSetting{
- chainID: chainID,
- agr: agr,
- recv: recv,
+ agr: mgr.baModule,
+ recv: mgr.baModule.data.recv.(*consensusBAReceiver),
}
roundBeginTime time.Time
roundEndTime time.Time
@@ -368,7 +321,7 @@ func (mgr *agreementMgr) runBA(initRound uint64, chainID uint32) {
// Check if this routine needs to awake in this round and prepare essential
// variables when yes.
- checkRound := func() (isNotary, isDisabled bool) {
+ checkRound := func() (isNotary bool) {
defer func() {
currentRound = nextRound
nextRound++
@@ -386,13 +339,8 @@ func (mgr *agreementMgr) runBA(initRound uint64, chainID uint32) {
// Set next checkpoint.
roundBeginTime = config.beginTime
roundEndTime = config.beginTime.Add(config.roundInterval)
- // Check if this chain handled by this routine included in this round.
- if chainID >= config.numChains {
- isDisabled = true
- return
- }
// Check if this node in notary set of this chain in this round.
- notarySet, err := mgr.cache.GetNotarySet(nextRound, chainID)
+ notarySet, err := mgr.cache.GetNotarySet(nextRound, 0)
if err != nil {
panic(err)
}
@@ -402,13 +350,11 @@ func (mgr *agreementMgr) runBA(initRound uint64, chainID uint32) {
if isNotary {
mgr.logger.Info("selected as notary set",
"ID", mgr.ID,
- "round", nextRound,
- "chainID", chainID)
+ "round", nextRound)
} else {
mgr.logger.Info("not selected as notary set",
"ID", mgr.ID,
- "round", nextRound,
- "chainID", chainID)
+ "round", nextRound)
}
// Setup ticker
if tickDuration != config.lambdaBA {
@@ -428,16 +374,7 @@ Loop:
default:
}
now := time.Now().UTC()
- var isDisabled bool
- setting.recv.isNotary, isDisabled = checkRound()
- if isDisabled {
- select {
- case <-mgr.ctx.Done():
- break Loop
- case <-time.After(roundEndTime.Sub(now)):
- continue Loop
- }
- }
+ setting.recv.isNotary = checkRound()
// Sleep until round begin. Here a biased round begin time would be
// used instead of the one in config. The reason it to disperse the load
// of fullnodes to verify confirmed blocks from each chain.
@@ -454,18 +391,17 @@ Loop:
<-setting.ticker.Tick()
}
// Run BA for this round.
- recv.roundValue.Store(currentRound)
- recv.changeNotaryTime = roundEndTime
- recv.restartNotary <- types.Position{
+ setting.recv.roundValue.Store(currentRound)
+ setting.recv.changeNotaryTime = roundEndTime
+ setting.recv.restartNotary <- types.Position{
Round: setting.recv.round(),
ChainID: math.MaxUint32,
}
- mgr.voteFilters[chainID] = utils.NewVoteFilter()
+ mgr.voteFilter = utils.NewVoteFilter()
if err := mgr.baRoutineForOneRound(&setting); err != nil {
mgr.logger.Error("BA routine failed",
"error", err,
- "nodeID", mgr.ID,
- "chain", chainID)
+ "nodeID", mgr.ID)
break Loop
}
}
@@ -485,13 +421,13 @@ func (mgr *agreementMgr) baRoutineForOneRound(
break
default:
}
- tipRound := mgr.lattice.TipRound(setting.chainID)
+ tipRound := mgr.bcModule.tipRound()
if tipRound > restartPos.Round {
// It's a vary rare that this go routine sleeps for entire round.
break
} else if tipRound != restartPos.Round {
- mgr.logger.Debug("Waiting lattice to change round...",
- "pos", &restartPos)
+ mgr.logger.Debug("Waiting blockChain to change round...",
+ "pos", restartPos)
} else {
break
}
@@ -501,7 +437,7 @@ func (mgr *agreementMgr) baRoutineForOneRound(
breakLoop = true
return
}
- if restartPos.Older(&oldPos) {
+ if restartPos.Older(oldPos) {
// The restartNotary event is triggered by 'BlockConfirmed'
// of some older block.
return
@@ -510,16 +446,7 @@ func (mgr *agreementMgr) baRoutineForOneRound(
var nextHeight uint64
var nextTime time.Time
for {
- nextHeight, nextTime, err =
- mgr.lattice.NextBlock(recv.round(), setting.chainID)
- if err != nil {
- mgr.logger.Debug("Error getting next height",
- "error", err,
- "round", recv.round(),
- "chainID", setting.chainID)
- err = nil
- nextHeight = restartPos.Height
- }
+ nextHeight, nextTime = mgr.bcModule.nextBlock()
if isStop(oldPos) && nextHeight == 0 {
break
}
@@ -529,14 +456,13 @@ func (mgr *agreementMgr) baRoutineForOneRound(
if nextHeight > restartPos.Height {
break
}
- mgr.logger.Debug("Lattice not ready!!!",
- "old", &oldPos, "restart", &restartPos, "next", nextHeight)
+ mgr.logger.Debug("BlockChain not ready!!!",
+ "old", oldPos, "restart", restartPos, "next", nextHeight)
time.Sleep(100 * time.Millisecond)
}
nextPos := types.Position{
- Round: recv.round(),
- ChainID: setting.chainID,
- Height: nextHeight,
+ Round: recv.round(),
+ Height: nextHeight,
}
oldPos = nextPos
var leader types.NodeID
@@ -591,7 +517,7 @@ Loop:
if agr.pullVotes() {
pos := agr.agreementID()
mgr.logger.Debug("Calling Network.PullVotes for syncing votes",
- "position", &pos)
+ "position", pos)
mgr.network.PullVotes(pos)
}
for i := 0; i < agr.clocks(); i++ {
diff --git a/core/agreement.go b/core/agreement.go
index b35100b..579cea8 100644
--- a/core/agreement.go
+++ b/core/agreement.go
@@ -162,7 +162,7 @@ func (a *agreement) restart(
defer a.lock.Unlock()
if !isStop(aID) {
oldAID := a.agreementID()
- if !isStop(oldAID) && !aID.Newer(&oldAID) {
+ if !isStop(oldAID) && !aID.Newer(oldAID) {
return false
}
}
@@ -209,7 +209,7 @@ func (a *agreement) restart(
defer a.lock.Unlock()
newPendingBlock := make([]pendingBlock, 0)
for _, pending := range a.pendingBlock {
- if aID.Newer(&pending.block.Position) {
+ if aID.Newer(pending.block.Position) {
continue
} else if pending.block.Position == aID {
replayBlock = append(replayBlock, pending.block)
@@ -226,7 +226,7 @@ func (a *agreement) restart(
defer a.lock.Unlock()
newPendingVote := make([]pendingVote, 0)
for _, pending := range a.pendingVote {
- if aID.Newer(&pending.vote.Position) {
+ if aID.Newer(pending.vote.Position) {
continue
} else if pending.vote.Position == aID {
replayVote = append(replayVote, pending.vote)
@@ -390,7 +390,7 @@ func (a *agreement) processVote(vote *types.Vote) error {
return nil
}
if vote.Position != aID {
- if aID.Newer(&vote.Position) {
+ if aID.Newer(vote.Position) {
return nil
}
a.pendingVote = append(a.pendingVote, pendingVote{
@@ -547,7 +547,7 @@ func (a *agreement) processBlock(block *types.Block) error {
if block.Position != aID {
// Agreement module has stopped.
if !isStop(aID) {
- if aID.Newer(&block.Position) {
+ if aID.Newer(block.Position) {
return nil
}
}
diff --git a/core/blockchain.go b/core/blockchain.go
new file mode 100644
index 0000000..bfa8552
--- /dev/null
+++ b/core/blockchain.go
@@ -0,0 +1,617 @@
+// Copyright 2019 The dexon-consensus Authors
+// This file is part of the dexon-consensus library.
+//
+// The dexon-consensus 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 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 library. If not, see
+// <http://www.gnu.org/licenses/>.
+
+package core
+
+import (
+ "errors"
+ "fmt"
+ "sort"
+ "sync"
+ "time"
+
+ "github.com/dexon-foundation/dexon-consensus/common"
+ "github.com/dexon-foundation/dexon-consensus/core/crypto"
+ "github.com/dexon-foundation/dexon-consensus/core/types"
+ "github.com/dexon-foundation/dexon-consensus/core/utils"
+)
+
+// Errors for sanity check error.
+var (
+ ErrBlockFromOlderPosition = errors.New("block from older position")
+ ErrNotGenesisBlock = errors.New("not a genesis block")
+ ErrIsGenesisBlock = errors.New("is a genesis block")
+ ErrIncorrectParentHash = errors.New("incorrect parent hash")
+ ErrInvalidBlockHeight = errors.New("invalid block height")
+ ErrInvalidRoundID = errors.New("invalid round id")
+ ErrNotFollowTipPosition = errors.New("not follow tip position")
+ ErrDuplicatedPendingBlock = errors.New("duplicated pending block")
+ ErrRetrySanityCheckLater = errors.New("retry sanity check later")
+ ErrRoundNotIncreasing = errors.New("round not increasing")
+ ErrRoundNotSwitch = errors.New("round not switch")
+ ErrIncorrectBlockRandomnessResult = errors.New(
+ "incorrect block randomness result")
+)
+
+type pendingBlockRecord struct {
+ position types.Position
+ block *types.Block
+}
+
+type pendingBlockRecords []pendingBlockRecord
+
+func (pb *pendingBlockRecords) insert(p pendingBlockRecord) error {
+ idx := sort.Search(len(*pb), func(i int) bool {
+ return !(*pb)[i].position.Older(p.position)
+ })
+ switch idx {
+ case len(*pb):
+ *pb = append(*pb, p)
+ default:
+ if (*pb)[idx].position.Equal(p.position) {
+ return ErrDuplicatedPendingBlock
+ }
+ // Insert the value to that index.
+ *pb = append((*pb), pendingBlockRecord{})
+ copy((*pb)[idx+1:], (*pb)[idx:])
+ (*pb)[idx] = p
+ }
+ return nil
+}
+
+func (pb pendingBlockRecords) searchByHeight(h uint64) (
+ pendingBlockRecord, bool) {
+ idx := sort.Search(len(pb), func(i int) bool {
+ return pb[i].position.Height >= h
+ })
+ if idx == len(pb) || pb[idx].position.Height != h {
+ return pendingBlockRecord{}, false
+ }
+ return pb[idx], true
+}
+
+func (pb pendingBlockRecords) searchByPosition(p types.Position) (
+ pendingBlockRecord, bool) {
+ idx := sort.Search(len(pb), func(i int) bool {
+ return !pb[i].block.Position.Older(p)
+ })
+ if idx == len(pb) || !pb[idx].position.Equal(p) {
+ return pendingBlockRecord{}, false
+ }
+ return pb[idx], true
+}
+
+type blockChainConfig struct {
+ roundBasedConfig
+
+ minBlockInterval time.Duration
+}
+
+func (c *blockChainConfig) fromConfig(round uint64, config *types.Config) {
+ c.minBlockInterval = config.MinBlockInterval
+ c.setupRoundBasedFields(round, config)
+}
+
+func newBlockChainConfig(prev blockChainConfig, config *types.Config) (
+ c blockChainConfig) {
+ c = blockChainConfig{}
+ c.fromConfig(prev.roundID+1, config)
+ c.setRoundBeginTime(prev.roundEndTime)
+ return
+}
+
+type tsigVerifierGetter interface {
+ UpdateAndGet(uint64) (TSigVerifier, bool, error)
+}
+
+type blockChain struct {
+ lock sync.RWMutex
+ ID types.NodeID
+ lastConfirmed *types.Block
+ lastDelivered *types.Block
+ signer *utils.Signer
+ vGetter tsigVerifierGetter
+ app Application
+ logger common.Logger
+ pendingRandomnesses map[types.Position]*types.BlockRandomnessResult
+ configs []blockChainConfig
+ pendingBlocks pendingBlockRecords
+ confirmedBlocks types.BlocksByPosition
+}
+
+func newBlockChain(nID types.NodeID, initBlock *types.Block,
+ initConfig blockChainConfig, app Application, vGetter tsigVerifierGetter,
+ signer *utils.Signer, logger common.Logger) *blockChain {
+ if initBlock != nil {
+ if initConfig.roundID != initBlock.Position.Round {
+ panic(fmt.Errorf("incompatible config/block %s %d",
+ initBlock, initConfig.roundID))
+ }
+ } else {
+ if initConfig.roundID != 0 {
+ panic(fmt.Errorf("genesis config should from round 0 %d",
+ initConfig.roundID))
+ }
+ }
+ return &blockChain{
+ ID: nID,
+ lastConfirmed: initBlock,
+ lastDelivered: initBlock,
+ signer: signer,
+ vGetter: vGetter,
+ app: app,
+ logger: logger,
+ configs: []blockChainConfig{initConfig},
+ pendingRandomnesses: make(
+ map[types.Position]*types.BlockRandomnessResult),
+ }
+}
+
+func (bc *blockChain) appendConfig(round uint64, config *types.Config) error {
+ expectedRound := uint64(len(bc.configs))
+ if bc.lastConfirmed != nil {
+ expectedRound += bc.lastConfirmed.Position.Round
+ }
+ if round != expectedRound {
+ return ErrRoundNotIncreasing
+ }
+ bc.configs = append(bc.configs, newBlockChainConfig(
+ bc.configs[len(bc.configs)-1], config))
+ return nil
+}
+
+func (bc *blockChain) proposeBlock(position types.Position,
+ proposeTime time.Time) (b *types.Block, err error) {
+ bc.lock.RLock()
+ defer bc.lock.RUnlock()
+ return bc.prepareBlock(position, proposeTime, false)
+}
+
+func (bc *blockChain) extractBlocks() (ret []*types.Block) {
+ bc.lock.Lock()
+ defer bc.lock.Unlock()
+ for len(bc.confirmedBlocks) > 0 {
+ c := bc.confirmedBlocks[0]
+ if c.Position.Round > 0 && len(c.Finalization.Randomness) == 0 {
+ break
+ }
+ c, bc.confirmedBlocks = bc.confirmedBlocks[0], bc.confirmedBlocks[1:]
+ // TODO(mission): remove these duplicated field if we fully converted
+ // to single chain.
+ c.Finalization.ParentHash = c.ParentHash
+ c.Finalization.Timestamp = c.Timestamp
+ // It's a workaround, the height for application is one-based.
+ c.Finalization.Height = c.Position.Height + 1
+ ret = append(ret, c)
+ bc.lastDelivered = c
+ }
+ return
+}
+
+func (bc *blockChain) sanityCheck(b *types.Block) error {
+ if b.Position.ChainID != 0 {
+ panic(fmt.Errorf("attempt to process block from non-zero chainID"))
+ }
+ if b.IsEmpty() {
+ panic(fmt.Errorf("pass empty block to sanity check: %s", b))
+ }
+ bc.lock.RLock()
+ defer bc.lock.RUnlock()
+ if bc.lastConfirmed == nil {
+ // It should be a genesis block.
+ if !b.IsGenesis() {
+ return ErrNotGenesisBlock
+ }
+ // TODO(mission): Do we have to check timestamp of genesis block?
+ return nil
+ }
+ if b.IsGenesis() {
+ return ErrIsGenesisBlock
+ }
+ if b.Position.Height != bc.lastConfirmed.Position.Height+1 {
+ if b.Position.Height > bc.lastConfirmed.Position.Height {
+ return ErrRetrySanityCheckLater
+ }
+ return ErrInvalidBlockHeight
+ }
+ tipConfig := bc.getTipConfig()
+ if tipConfig.isLastBlock(bc.lastConfirmed) {
+ if b.Position.Round != bc.lastConfirmed.Position.Round+1 {
+ return ErrRoundNotSwitch
+ }
+ } else {
+ if b.Position.Round != bc.lastConfirmed.Position.Round {
+ return ErrInvalidRoundID
+ }
+ }
+ if !b.ParentHash.Equal(bc.lastConfirmed.Hash) {
+ return ErrIncorrectParentHash
+ }
+ if err := utils.VerifyBlockSignature(b); err != nil {
+ return err
+ }
+ return nil
+}
+
+// addEmptyBlock is called when an empty block is confirmed by BA.
+func (bc *blockChain) addEmptyBlock(position types.Position) (
+ *types.Block, error) {
+ if position.ChainID != 0 {
+ panic(fmt.Errorf("attempt to process block from non-zero chainID"))
+ }
+ bc.lock.Lock()
+ defer bc.lock.Unlock()
+ add := func() *types.Block {
+ emptyB, err := bc.prepareBlock(position, time.Time{}, true)
+ if err != nil || emptyB == nil {
+ // This helper is expected to be called when an empty block is ready
+ // to be confirmed.
+ panic(err)
+ }
+ bc.confirmBlock(emptyB)
+ bc.checkIfBlocksConfirmed()
+ return emptyB
+ }
+ if bc.lastConfirmed != nil {
+ if !position.Newer(bc.lastConfirmed.Position) {
+ bc.logger.Warn("Dropping empty block: older than tip",
+ "position", &position,
+ "last-confirmed", bc.lastConfirmed)
+ return nil, ErrBlockFromOlderPosition
+ }
+ if bc.lastConfirmed.Position.Height+1 == position.Height {
+ return add(), nil
+ }
+ } else if position.Height == 0 && position.Round == 0 {
+ return add(), nil
+ }
+ bc.addPendingBlockRecord(pendingBlockRecord{position, nil})
+ return nil, nil
+}
+
+// addBlock should be called when the block is confirmed by BA, we won't perform
+// sanity check against this block, it's ok to add block with skipping height.
+func (bc *blockChain) addBlock(b *types.Block) error {
+ if b.Position.ChainID != 0 {
+ panic(fmt.Errorf("attempt to process block from non-zero chainID"))
+ }
+ bc.lock.Lock()
+ defer bc.lock.Unlock()
+ confirmed := false
+ if bc.lastConfirmed != nil {
+ if !b.Position.Newer(bc.lastConfirmed.Position) {
+ bc.logger.Warn("Dropping block: older than tip",
+ "block", b, "last-confirmed", bc.lastConfirmed)
+ return nil
+ }
+ if bc.lastConfirmed.Position.Height+1 == b.Position.Height {
+ confirmed = true
+ }
+ } else if b.IsGenesis() {
+ confirmed = true
+ }
+ if !confirmed {
+ bc.addPendingBlockRecord(pendingBlockRecord{b.Position, b})
+ } else {
+ bc.confirmBlock(b)
+ bc.checkIfBlocksConfirmed()
+ }
+ return nil
+}
+
+func (bc *blockChain) addRandomness(r *types.BlockRandomnessResult) error {
+ if r.Position.ChainID != 0 {
+ panic(fmt.Errorf("attempt to process block from non-zero chainID"))
+ }
+ if func() bool {
+ bc.lock.RLock()
+ defer bc.lock.RUnlock()
+ if bc.lastDelivered != nil &&
+ bc.lastDelivered.Position.Newer(r.Position) {
+ return true
+ }
+ _, exists := bc.pendingRandomnesses[r.Position]
+ if exists {
+ return true
+ }
+ b := bc.findPendingBlock(r.Position)
+ return b != nil && len(b.Finalization.Randomness) > 0
+ }() {
+ return nil
+ }
+ ok, err := bc.verifyRandomness(r.BlockHash, r.Position.Round, r.Randomness)
+ if err != nil {
+ return err
+ }
+ if !ok {
+ return ErrIncorrectBlockRandomnessResult
+ }
+ bc.lock.Lock()
+ defer bc.lock.Unlock()
+ if b := bc.findPendingBlock(r.Position); b != nil {
+ if !r.BlockHash.Equal(b.Hash) {
+ panic(fmt.Errorf("mismathed randomness: %s %s", b, r))
+ }
+ b.Finalization.Randomness = r.Randomness
+ } else {
+ bc.pendingRandomnesses[r.Position] = r
+ }
+ return nil
+}
+
+// TODO(mission): remove this method after removing the strong binding between
+// BA and blockchain.
+func (bc *blockChain) tipRound() uint64 {
+ bc.lock.RLock()
+ defer bc.lock.RUnlock()
+ if bc.lastConfirmed == nil {
+ return 0
+ }
+ offset := uint64(0)
+ if bc.lastConfirmed.Timestamp.After(bc.getTipConfig().roundEndTime) {
+ offset++
+ }
+ return bc.lastConfirmed.Position.Round + offset
+}
+
+// TODO(mission): the pulling should be done inside of blockchain, then we don't
+// have to expose this method.
+func (bc *blockChain) confirmed(h uint64) bool {
+ bc.lock.RLock()
+ defer bc.lock.RUnlock()
+ if bc.lastConfirmed != nil && bc.lastConfirmed.Position.Height >= h {
+ return true
+ }
+ r, found := bc.pendingBlocks.searchByHeight(h)
+ if !found {
+ return false
+ }
+ return r.block != nil
+}
+
+// TODO(mission): this method can be removed after refining the relation between
+// BA and block storage.
+func (bc *blockChain) nextBlock() (uint64, time.Time) {
+ bc.lock.RLock()
+ defer bc.lock.RUnlock()
+ // It's ok to access tip config directly without checking the existence of
+ // lastConfirmed block in the scenario of "nextBlock" method.
+ tip, config := bc.lastConfirmed, bc.configs[0]
+ if tip == nil {
+ return 0, config.roundBeginTime
+ }
+ return tip.Position.Height + 1, tip.Timestamp.Add(config.minBlockInterval)
+}
+
+func (bc *blockChain) pendingBlocksWithoutRandomness() (hashes common.Hashes) {
+ bc.lock.RLock()
+ defer bc.lock.RUnlock()
+ for _, b := range bc.confirmedBlocks {
+ if b.Position.Round == 0 || len(b.Finalization.Randomness) > 0 {
+ continue
+ }
+ hashes = append(hashes, b.Hash)
+ }
+ for _, r := range bc.pendingBlocks {
+ if r.position.Round == 0 {
+ continue
+ }
+ if r.block != nil && len(r.block.Finalization.Randomness) == 0 {
+ hashes = append(hashes, r.block.Hash)
+ }
+ }
+ return
+}
+
+func (bc *blockChain) lastDeliveredBlock() *types.Block {
+ bc.lock.RLock()
+ defer bc.lock.RUnlock()
+ return bc.lastDelivered
+}
+
+func (bc *blockChain) lastPendingBlock() *types.Block {
+ bc.lock.RLock()
+ defer bc.lock.RUnlock()
+ if len(bc.confirmedBlocks) == 0 {
+ return nil
+ }
+ return bc.confirmedBlocks[0]
+}
+
+func (bc *blockChain) processFinalizedBlock(b *types.Block) error {
+ return bc.addRandomness(&types.BlockRandomnessResult{
+ BlockHash: b.Hash,
+ Position: b.Position,
+ Randomness: b.Finalization.Randomness,
+ })
+}
+
+/////////////////////////////////////////////
+//
+// internal helpers
+//
+/////////////////////////////////////////////
+
+// findPendingBlock is a helper to find a block in either pending or confirmed
+// state by position.
+func (bc *blockChain) findPendingBlock(p types.Position) *types.Block {
+ if idx := sort.Search(len(bc.confirmedBlocks), func(i int) bool {
+ return !bc.confirmedBlocks[i].Position.Older(p)
+ }); idx != len(bc.confirmedBlocks) &&
+ bc.confirmedBlocks[idx].Position.Equal(p) {
+ return bc.confirmedBlocks[idx]
+ }
+ pendingRec, _ := bc.pendingBlocks.searchByPosition(p)
+ return pendingRec.block
+}
+
+func (bc *blockChain) addPendingBlockRecord(p pendingBlockRecord) {
+ if err := bc.pendingBlocks.insert(p); err != nil {
+ panic(err)
+ }
+ if p.block != nil {
+ bc.setRandomnessFromPending(p.block)
+ }
+}
+
+func (bc *blockChain) checkIfBlocksConfirmed() {
+ var err error
+ for len(bc.pendingBlocks) > 0 {
+ if bc.pendingBlocks[0].position.Height <
+ bc.lastConfirmed.Position.Height+1 {
+ panic(fmt.Errorf("unexpected case %s %s", bc.lastConfirmed,
+ bc.pendingBlocks[0].position))
+ }
+ if bc.pendingBlocks[0].position.Height >
+ bc.lastConfirmed.Position.Height+1 {
+ break
+ }
+ var pending pendingBlockRecord
+ pending, bc.pendingBlocks = bc.pendingBlocks[0], bc.pendingBlocks[1:]
+ nextTip := pending.block
+ if nextTip == nil {
+ if nextTip, err = bc.prepareBlock(
+ pending.position, time.Time{}, true); err != nil {
+ // It should not be error when prepare empty block for correct
+ // position.
+ panic(err)
+ }
+ }
+ bc.confirmBlock(nextTip)
+ }
+}
+
+func (bc *blockChain) purgeConfig() {
+ for bc.configs[0].roundID < bc.lastConfirmed.Position.Round {
+ bc.configs = bc.configs[1:]
+ }
+ if bc.configs[0].roundID != bc.lastConfirmed.Position.Round {
+ panic(fmt.Errorf("mismatched tip config: %d %d",
+ bc.configs[0].roundID, bc.lastConfirmed.Position.Round))
+ }
+}
+
+func (bc *blockChain) verifyRandomness(
+ blockHash common.Hash, round uint64, randomness []byte) (bool, error) {
+ if round == 0 {
+ return len(randomness) == 0, nil
+ }
+ v, ok, err := bc.vGetter.UpdateAndGet(round)
+ if err != nil {
+ return false, err
+ }
+ if !ok {
+ return false, ErrTSigNotReady
+ }
+ return v.VerifySignature(blockHash, crypto.Signature{
+ Type: "bls",
+ Signature: randomness}), nil
+}
+
+func (bc *blockChain) prepareBlock(position types.Position,
+ proposeTime time.Time, empty bool) (b *types.Block, err error) {
+ // TODO(mission): refine timestamp.
+ b = &types.Block{Position: position, Timestamp: proposeTime}
+ tip := bc.lastConfirmed
+ // Make sure we can propose a block at expected position for callers.
+ expectedPosition := types.Position{}
+ if tip == nil {
+ // The case for genesis block.
+ if !position.Equal(expectedPosition) {
+ b, err = nil, ErrNotGenesisBlock
+ } else if empty {
+ b.Timestamp = bc.configs[0].roundBeginTime
+ }
+ } else {
+ expectedPosition.Height = tip.Position.Height + 1
+ tipConfig := bc.getTipConfig()
+ if tipConfig.isLastBlock(tip) {
+ expectedPosition.Round = tip.Position.Round + 1
+ } else {
+ expectedPosition.Round = tip.Position.Round
+ }
+ if !expectedPosition.Equal(position) {
+ b, err = nil, ErrNotFollowTipPosition
+ return
+ }
+ b.ParentHash = tip.Hash
+ if !empty {
+ if b.Payload, err = bc.app.PreparePayload(b.Position); err != nil {
+ return
+ }
+ if b.Witness, err = bc.app.PrepareWitness(
+ tip.Witness.Height); err != nil {
+ return
+ }
+ if !b.Timestamp.After(tip.Timestamp) {
+ b.Timestamp = tip.Timestamp.Add(tipConfig.minBlockInterval)
+ }
+
+ } else {
+ b.Witness.Height = tip.Witness.Height
+ b.Witness.Data = make([]byte, len(tip.Witness.Data))
+ copy(b.Witness.Data, tip.Witness.Data)
+ b.Timestamp = tip.Timestamp.Add(tipConfig.minBlockInterval)
+ }
+ }
+ if empty {
+ if b.Hash, err = utils.HashBlock(b); err != nil {
+ b = nil
+ return
+ }
+ } else {
+ if err = bc.signer.SignBlock(b); err != nil {
+ return
+ }
+ }
+ return
+}
+
+func (bc *blockChain) getTipConfig() blockChainConfig {
+ if bc.lastConfirmed == nil {
+ panic(fmt.Errorf("attempting to access config without tip"))
+ }
+ if bc.lastConfirmed.Position.Round != bc.configs[0].roundID {
+ panic(fmt.Errorf("inconsist config and tip: %d %d",
+ bc.lastConfirmed.Position.Round, bc.configs[0].roundID))
+ }
+ return bc.configs[0]
+}
+
+func (bc *blockChain) confirmBlock(b *types.Block) {
+ if bc.lastConfirmed != nil &&
+ bc.lastConfirmed.Position.Height+1 != b.Position.Height {
+ panic(fmt.Errorf("confirmed blocks not continuous in height: %s %s",
+ bc.lastConfirmed, b))
+ }
+ bc.logger.Debug("Calling Application.BlockConfirmed", "block", b)
+ bc.app.BlockConfirmed(*b)
+ bc.lastConfirmed = b
+ bc.setRandomnessFromPending(b)
+ bc.confirmedBlocks = append(bc.confirmedBlocks, b)
+ bc.purgeConfig()
+}
+
+func (bc *blockChain) setRandomnessFromPending(b *types.Block) {
+ if r, exist := bc.pendingRandomnesses[b.Position]; exist {
+ if !r.BlockHash.Equal(b.Hash) {
+ panic(fmt.Errorf("mismathed randomness: %s %s", b, r))
+ }
+ b.Finalization.Randomness = r.Randomness
+ delete(bc.pendingRandomnesses, b.Position)
+ }
+}
diff --git a/core/blockchain_test.go b/core/blockchain_test.go
new file mode 100644
index 0000000..908b04f
--- /dev/null
+++ b/core/blockchain_test.go
@@ -0,0 +1,462 @@
+// Copyright 2019 The dexon-consensus Authors
+// This file is part of the dexon-consensus library.
+//
+// The dexon-consensus 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 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 library. If not, see
+// <http://www.gnu.org/licenses/>.
+
+package core
+
+import (
+ "fmt"
+ "testing"
+ "time"
+
+ "github.com/dexon-foundation/dexon-consensus/common"
+ "github.com/dexon-foundation/dexon-consensus/core/crypto"
+ "github.com/dexon-foundation/dexon-consensus/core/test"
+ "github.com/dexon-foundation/dexon-consensus/core/types"
+ "github.com/dexon-foundation/dexon-consensus/core/utils"
+ "github.com/stretchr/testify/suite"
+)
+
+type testTSigVerifier struct{}
+
+func (v *testTSigVerifier) VerifySignature(hash common.Hash,
+ sig crypto.Signature) bool {
+ return true
+}
+
+type testTSigVerifierGetter struct{}
+
+func (t *testTSigVerifierGetter) UpdateAndGet(round uint64) (
+ TSigVerifier, bool, error) {
+ return &testTSigVerifier{}, true, nil
+}
+
+type BlockChainTestSuite struct {
+ suite.Suite
+
+ nID types.NodeID
+ signer *utils.Signer
+ dMoment time.Time
+ blockInterval time.Duration
+ roundInterval time.Duration
+}
+
+func (s *BlockChainTestSuite) SetupSuite() {
+ prvKeys, pubKeys, err := test.NewKeys(1)
+ s.Require().NoError(err)
+ s.nID = types.NewNodeID(pubKeys[0])
+ s.signer = utils.NewSigner(prvKeys[0])
+ s.dMoment = time.Now().UTC()
+ s.blockInterval = 1 * time.Millisecond
+ s.roundInterval = 10 * time.Second
+}
+
+func (s *BlockChainTestSuite) newBlocks(c uint64, initBlock *types.Block) (
+ blocks []*types.Block) {
+ parentHash := common.Hash{}
+ baseHeight := uint64(0)
+ t := s.dMoment
+ initRound := uint64(0)
+ if initBlock != nil {
+ parentHash = initBlock.Hash
+ t = initBlock.Timestamp.Add(s.blockInterval)
+ initRound = initBlock.Position.Round
+ baseHeight = initBlock.Position.Height + 1
+ }
+ for i := uint64(0); i < uint64(c); i++ {
+ b := &types.Block{
+ ParentHash: parentHash,
+ Position: types.Position{Round: initRound, Height: baseHeight + i},
+ Timestamp: t,
+ }
+ s.Require().NoError(s.signer.SignBlock(b))
+ blocks = append(blocks, b)
+ parentHash = b.Hash
+ t = t.Add(s.blockInterval)
+ }
+ return
+}
+
+func (s *BlockChainTestSuite) newEmptyBlock(parent *types.Block,
+ blockInterval time.Duration) *types.Block {
+ emptyB := &types.Block{
+ ParentHash: parent.Hash,
+ Position: types.Position{
+ Round: parent.Position.Round,
+ Height: parent.Position.Height + 1,
+ },
+ Timestamp: parent.Timestamp.Add(blockInterval),
+ }
+ var err error
+ emptyB.Hash, err = utils.HashBlock(emptyB)
+ s.Require().NoError(err)
+ return emptyB
+}
+
+func (s *BlockChainTestSuite) newBlock(parent *types.Block, round uint64,
+ blockInterval time.Duration) *types.Block {
+ b := &types.Block{
+ ParentHash: parent.Hash,
+ Position: types.Position{
+ Round: round,
+ Height: parent.Position.Height + 1,
+ },
+ Timestamp: parent.Timestamp.Add(blockInterval),
+ }
+ s.Require().NoError(s.signer.SignBlock(b))
+ return b
+}
+
+func (s *BlockChainTestSuite) newRandomnessFromBlock(
+ b *types.Block) *types.BlockRandomnessResult {
+ return &types.BlockRandomnessResult{
+ BlockHash: b.Hash,
+ Position: b.Position,
+ Randomness: common.GenerateRandomBytes(),
+ }
+}
+
+func (s *BlockChainTestSuite) newBlockChain(initB *types.Block) *blockChain {
+ initRound := uint64(0)
+ if initB != nil {
+ initRound = initB.Position.Round
+ }
+ initConfig := blockChainConfig{}
+ initConfig.fromConfig(initRound, &types.Config{
+ MinBlockInterval: s.blockInterval,
+ RoundInterval: s.roundInterval,
+ })
+ initConfig.setRoundBeginTime(s.dMoment)
+ return newBlockChain(s.nID, initB, initConfig, test.NewApp(0, nil),
+ &testTSigVerifierGetter{}, s.signer, &common.NullLogger{})
+}
+
+func (s *BlockChainTestSuite) newRoundOneInitBlock() *types.Block {
+ initBlock := &types.Block{
+ ParentHash: common.NewRandomHash(),
+ Position: types.Position{Round: 1},
+ Timestamp: s.dMoment,
+ }
+ s.Require().NoError(s.signer.SignBlock(initBlock))
+ return initBlock
+}
+
+func (s *BlockChainTestSuite) baseConcurrentAceessTest(initBlock *types.Block,
+ blocks []*types.Block, rands []*types.BlockRandomnessResult) {
+ var (
+ bc = s.newBlockChain(initBlock)
+ start = make(chan struct{})
+ newNotif = make(chan struct{}, 1)
+ delivered []*types.Block
+ )
+ add := func(v interface{}) {
+ <-start
+ switch val := v.(type) {
+ case *types.Block:
+ if err := bc.addBlock(val); err != nil {
+ // Never assertion in sub routine when testing.
+ panic(err)
+ }
+ case *types.BlockRandomnessResult:
+ if err := bc.addRandomness(val); err != nil {
+ // Never assertion in sub routine when testing.
+ panic(err)
+ }
+ default:
+ panic(fmt.Errorf("unknown type: %v", v))
+ }
+ select {
+ case newNotif <- struct{}{}:
+ default:
+ }
+ }
+ for _, b := range blocks {
+ go add(b)
+ }
+ for _, r := range rands {
+ go add(r)
+ }
+ close(start)
+ for {
+ select {
+ case <-newNotif:
+ delivered = append(delivered, bc.extractBlocks()...)
+ case <-time.After(100 * time.Millisecond):
+ delivered = append(delivered, bc.extractBlocks()...)
+ }
+ if len(delivered) == len(blocks) {
+ break
+ }
+ }
+ // Check result.
+ b := delivered[0]
+ s.Require().Equal(b.Position.Height, uint64(1))
+ s.Require().NotEmpty(b.Finalization.Randomness)
+ for _, bb := range delivered[1:] {
+ s.Require().Equal(b.Position.Height+1, bb.Position.Height)
+ s.Require().NotEmpty(b.Finalization.Randomness)
+ b = bb
+ }
+}
+
+func (s *BlockChainTestSuite) TestBasicUsage() {
+ initBlock := s.newRoundOneInitBlock()
+ bc := s.newBlockChain(initBlock)
+ // test scenario: block, empty block, randomness can be added in any order
+ // of position.
+ blocks := s.newBlocks(4, initBlock)
+ b0, b1, b2, b3 := blocks[0], blocks[1], blocks[2], blocks[3]
+ // generate block-5 after block-4, which is an empty block.
+ b4 := s.newEmptyBlock(b3, time.Millisecond)
+ b5 := &types.Block{
+ ParentHash: b4.Hash,
+ Position: types.Position{Round: 1, Height: b4.Position.Height + 1},
+ }
+ s.Require().NoError(s.signer.SignBlock(b5))
+ r0 := s.newRandomnessFromBlock(b0)
+ r1 := s.newRandomnessFromBlock(b1)
+ r2 := s.newRandomnessFromBlock(b2)
+ r3 := s.newRandomnessFromBlock(b3)
+ r4 := s.newRandomnessFromBlock(b4)
+ r5 := s.newRandomnessFromBlock(b5)
+ // add those datum in reversed order of position.
+ s.Require().NoError(bc.addRandomness(r4))
+ s.Require().NoError(bc.addRandomness(r3))
+ s.Require().NoError(bc.addRandomness(r2))
+ s.Require().NoError(bc.addRandomness(r1))
+ s.Require().NoError(bc.addRandomness(r0))
+ s.Require().NoError(bc.addBlock(b5))
+ emptyB, err := bc.addEmptyBlock(b4.Position)
+ s.Require().Nil(emptyB)
+ s.Require().NoError(err)
+ s.Require().NoError(bc.addBlock(b3))
+ s.Require().NoError(bc.addBlock(b2))
+ s.Require().NoError(bc.addBlock(b1))
+ s.Require().NoError(bc.addBlock(b0))
+ extracted := bc.extractBlocks()
+ s.Require().Len(extracted, 5)
+ s.Require().Equal(extracted[4].Hash, b4.Hash)
+ s.Require().NoError(bc.addRandomness(r5))
+ extracted = bc.extractBlocks()
+ s.Require().Len(extracted, 1)
+ s.Require().Equal(extracted[0].Hash, b5.Hash)
+}
+
+func (s *BlockChainTestSuite) TestConcurrentAccess() {
+ // Raise one go routine for each block and randomness. And let them try to
+ // add to blockChain at the same time. Make sure we can delivered them all.
+ var (
+ retry = 10
+ initBlock = s.newRoundOneInitBlock()
+ blocks = s.newBlocks(500, initBlock)
+ rands = []*types.BlockRandomnessResult{}
+ )
+ for _, b := range blocks {
+ rands = append(rands, s.newRandomnessFromBlock(b))
+ }
+ for i := 0; i < retry; i++ {
+ s.baseConcurrentAceessTest(initBlock, blocks, rands)
+ }
+}
+
+func (s *BlockChainTestSuite) TestSanityCheck() {
+ bc := s.newBlockChain(nil)
+ // Non-zero chainID is not allowed.
+ s.Require().Panics(func() {
+ bc.sanityCheck(&types.Block{Position: types.Position{ChainID: 1}})
+ })
+ // Empty block is not allowed.
+ s.Require().Panics(func() {
+ bc.sanityCheck(&types.Block{})
+ })
+ blocks := s.newBlocks(3, nil)
+ b0, b1, b2 := blocks[0], blocks[1], blocks[2]
+ // ErrNotGenesisBlock
+ s.Require().Equal(ErrNotGenesisBlock.Error(), bc.sanityCheck(b1).Error())
+ // Genesis block should pass sanity check.
+ s.Require().NoError(bc.sanityCheck(b0))
+ s.Require().NoError(bc.addBlock(b0))
+ // ErrIsGenesisBlock
+ s.Require().Equal(ErrIsGenesisBlock.Error(), bc.sanityCheck(b0).Error())
+ // ErrRetrySanityCheckLater
+ s.Require().Equal(
+ ErrRetrySanityCheckLater.Error(), bc.sanityCheck(b2).Error())
+ // ErrInvalidBlockHeight
+ s.Require().NoError(bc.addBlock(b1))
+ s.Require().NoError(bc.addBlock(b2))
+ s.Require().Equal(
+ ErrInvalidBlockHeight.Error(), bc.sanityCheck(b1).Error())
+ // ErrInvalidRoundID
+ // Should not switch round when tip is not the last block.
+ s.Require().Equal(
+ ErrInvalidRoundID.Error(),
+ bc.sanityCheck(s.newBlock(b2, 1, 1*time.Second)).Error())
+ b3 := s.newBlock(b2, 0, 100*time.Second)
+ s.Require().NoError(bc.addBlock(b3))
+ // Should switch round when tip is the last block.
+ s.Require().Equal(
+ ErrRoundNotSwitch.Error(),
+ bc.sanityCheck(s.newBlock(b3, 0, 1*time.Second)).Error())
+ // ErrIncorrectParentHash
+ b4 := &types.Block{
+ ParentHash: b2.Hash,
+ Position: types.Position{
+ Round: 1,
+ Height: 4,
+ },
+ Timestamp: b3.Timestamp.Add(1 * time.Second),
+ }
+ s.Require().NoError(s.signer.SignBlock(b4))
+ s.Require().Equal(
+ ErrIncorrectParentHash.Error(), bc.sanityCheck(b4).Error())
+ // There is no valid signature attached.
+ b4.ParentHash = b3.Hash
+ s.Require().Error(bc.sanityCheck(b4))
+ // OK case.
+ s.Require().NoError(s.signer.SignBlock(b4))
+ s.Require().NoError(bc.sanityCheck(b4))
+}
+
+func (s *BlockChainTestSuite) TestAppendConfig() {
+ bc := s.newBlockChain(nil)
+ s.Require().Equal(ErrRoundNotIncreasing.Error(),
+ bc.appendConfig(0, &types.Config{}).Error())
+ s.Require().Equal(ErrRoundNotIncreasing.Error(),
+ bc.appendConfig(2, &types.Config{}).Error())
+ s.Require().NoError(bc.appendConfig(1, &types.Config{}))
+}
+
+func (s *BlockChainTestSuite) TestConfirmed() {
+ bc := s.newBlockChain(nil)
+ blocks := s.newBlocks(3, nil)
+ // Add a confirmed block.
+ s.Require().NoError(bc.addBlock(blocks[0]))
+ // Add a pending block.
+ s.Require().NoError(bc.addBlock(blocks[2]))
+ s.Require().True(bc.confirmed(0))
+ s.Require().False(bc.confirmed(1))
+ s.Require().True(bc.confirmed(2))
+}
+
+func (s *BlockChainTestSuite) TestNextBlock() {
+ bc := s.newBlockChain(nil)
+ blocks := s.newBlocks(3, nil)
+ nextH, nextT := bc.nextBlock()
+ s.Require().Equal(nextH, uint64(0))
+ s.Require().Equal(nextT, bc.configs[0].roundBeginTime)
+ // Add one block.
+ s.Require().NoError(bc.addBlock(blocks[0]))
+ nextH, nextT = bc.nextBlock()
+ s.Require().Equal(nextH, uint64(1))
+ s.Require().Equal(
+ nextT, blocks[0].Timestamp.Add(bc.configs[0].minBlockInterval))
+ // Add one block, expected to be pending.
+ s.Require().NoError(bc.addBlock(blocks[2]))
+ nextH2, nextT2 := bc.nextBlock()
+ s.Require().Equal(nextH, nextH2)
+ s.Require().Equal(nextT, nextT2)
+}
+
+func (s *BlockChainTestSuite) TestPendingBlocksWithoutRandomness() {
+ initBlock := s.newRoundOneInitBlock()
+ bc := s.newBlockChain(initBlock)
+ blocks := s.newBlocks(4, initBlock)
+ s.Require().NoError(bc.addBlock(blocks[0]))
+ s.Require().NoError(bc.addBlock(blocks[1]))
+ s.Require().NoError(bc.addBlock(blocks[3]))
+ s.Require().Equal(bc.pendingBlocksWithoutRandomness(), common.Hashes{
+ blocks[0].Hash, blocks[1].Hash, blocks[3].Hash})
+ s.Require().NoError(bc.addRandomness(s.newRandomnessFromBlock(blocks[0])))
+ s.Require().Equal(bc.pendingBlocksWithoutRandomness(), common.Hashes{
+ blocks[1].Hash, blocks[3].Hash})
+}
+
+func (s *BlockChainTestSuite) TestLastXBlock() {
+ initBlock := s.newRoundOneInitBlock()
+ bc := s.newBlockChain(initBlock)
+ s.Require().Nil(bc.lastPendingBlock())
+ s.Require().True(bc.lastDeliveredBlock() == initBlock)
+ blocks := s.newBlocks(2, initBlock)
+ s.Require().NoError(bc.addBlock(blocks[0]))
+ s.Require().True(bc.lastPendingBlock() == blocks[0])
+ s.Require().True(bc.lastDeliveredBlock() == initBlock)
+ s.Require().NoError(bc.addRandomness(s.newRandomnessFromBlock(blocks[0])))
+ s.Require().Len(bc.extractBlocks(), 1)
+ s.Require().Nil(bc.lastPendingBlock())
+ s.Require().True(bc.lastDeliveredBlock() == blocks[0])
+ s.Require().NoError(bc.addBlock(blocks[1]))
+ s.Require().True(bc.lastPendingBlock() == blocks[1])
+ s.Require().True(bc.lastDeliveredBlock() == blocks[0])
+}
+
+func (s *BlockChainTestSuite) TestPendingBlockRecords() {
+ bs := s.newBlocks(5, nil)
+ ps := pendingBlockRecords{}
+ s.Require().NoError(ps.insert(pendingBlockRecord{bs[2].Position, bs[2]}))
+ s.Require().NoError(ps.insert(pendingBlockRecord{bs[1].Position, bs[1]}))
+ s.Require().NoError(ps.insert(pendingBlockRecord{bs[0].Position, bs[0]}))
+ s.Require().Equal(ErrDuplicatedPendingBlock.Error(),
+ ps.insert(pendingBlockRecord{bs[0].Position, nil}).Error())
+ s.Require().True(ps[0].position.Equal(bs[0].Position))
+ s.Require().True(ps[1].position.Equal(bs[1].Position))
+ s.Require().True(ps[2].position.Equal(bs[2].Position))
+ s.Require().NoError(ps.insert(pendingBlockRecord{bs[4].Position, bs[4]}))
+ // Here assume block3 is empty, since we didn't verify parent hash in
+ // pendingBlockRecords, it should be fine.
+ s.Require().NoError(ps.insert(pendingBlockRecord{bs[3].Position, nil}))
+ s.Require().True(ps[3].position.Equal(bs[3].Position))
+ s.Require().True(ps[4].position.Equal(bs[4].Position))
+}
+
+func (s *BlockChainTestSuite) TestFindPendingBlock() {
+ bc := s.newBlockChain(nil)
+ blocks := s.newBlocks(7, nil)
+ s.Require().NoError(bc.addBlock(blocks[6]))
+ s.Require().NoError(bc.addBlock(blocks[5]))
+ s.Require().NoError(bc.addBlock(blocks[3]))
+ s.Require().NoError(bc.addBlock(blocks[2]))
+ s.Require().NoError(bc.addBlock(blocks[1]))
+ s.Require().NoError(bc.addBlock(blocks[0]))
+ s.Require().True(bc.findPendingBlock(blocks[0].Position) == blocks[0])
+ s.Require().True(bc.findPendingBlock(blocks[1].Position) == blocks[1])
+ s.Require().True(bc.findPendingBlock(blocks[2].Position) == blocks[2])
+ s.Require().True(bc.findPendingBlock(blocks[3].Position) == blocks[3])
+ s.Require().Nil(bc.findPendingBlock(blocks[4].Position))
+ s.Require().True(bc.findPendingBlock(blocks[5].Position) == blocks[5])
+ s.Require().True(bc.findPendingBlock(blocks[6].Position) == blocks[6])
+}
+
+func (s *BlockChainTestSuite) TestAddEmptyBlockDirectly() {
+ bc := s.newBlockChain(nil)
+ blocks := s.newBlocks(1, nil)
+ s.Require().NoError(bc.addBlock(blocks[0]))
+ // Add an empty block after a normal block.
+ pos := types.Position{Height: 1}
+ emptyB, err := bc.addEmptyBlock(pos)
+ s.Require().NotNil(emptyB)
+ s.Require().True(emptyB.Position.Equal(pos))
+ s.Require().NoError(err)
+ // Add an empty block after an empty block.
+ pos = types.Position{Height: 2}
+ emptyB, err = bc.addEmptyBlock(pos)
+ s.Require().NotNil(emptyB)
+ s.Require().True(emptyB.Position.Equal(pos))
+ s.Require().NoError(err)
+}
+
+func TestBlockChain(t *testing.T) {
+ suite.Run(t, new(BlockChainTestSuite))
+}
diff --git a/core/blockpool.go b/core/blockpool.go
deleted file mode 100644
index 4e41aa7..0000000
--- a/core/blockpool.go
+++ /dev/null
@@ -1,79 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package core
-
-import (
- "container/heap"
-
- "github.com/dexon-foundation/dexon-consensus/core/types"
-)
-
-// blockPool is a heaped slice of blocks ([][]*types.Block), indexed by chainID,
-// and blocks in each is sorted by block's height.
-type blockPool []types.ByPosition
-
-func newBlockPool(chainNum uint32) (pool blockPool) {
- pool = make(blockPool, chainNum)
- for i := range pool {
- heap.Init(&pool[i])
- }
- return
-}
-
-func (p *blockPool) resize(num uint32) {
- if uint32(len(*p)) >= num {
- // Do nothing If the origin size is larger.
- return
- }
- newPool := make(blockPool, num)
- copy(newPool, *p)
- for i := uint32(len(*p)); i < num; i++ {
- newChain := types.ByPosition{}
- heap.Init(&newChain)
- newPool[i] = newChain
- }
- *p = newPool
-}
-
-// addBlock adds a block into pool and sorts them by height.
-func (p blockPool) addBlock(b *types.Block) {
- heap.Push(&p[b.Position.ChainID], b)
-}
-
-// purgeBlocks purges blocks of a specified chain with less-or-equal heights.
-// NOTE: "chainID" is not checked here, this should be ensured by the caller.
-func (p blockPool) purgeBlocks(chainID uint32, height uint64) {
- for len(p[chainID]) > 0 && p[chainID][0].Position.Height <= height {
- heap.Pop(&p[chainID])
- }
-}
-
-// tip returns block with the smallest height, nil if no existing block.
-func (p blockPool) tip(chainID uint32) *types.Block {
- if len(p[chainID]) == 0 {
- return nil
- }
- return p[chainID][0]
-}
-
-// removeTip removes block with lowest height of a specified chain.
-func (p blockPool) removeTip(chainID uint32) {
- if len(p[chainID]) > 0 {
- heap.Pop(&p[chainID])
- }
-}
diff --git a/core/blockpool_test.go b/core/blockpool_test.go
deleted file mode 100644
index ce16efa..0000000
--- a/core/blockpool_test.go
+++ /dev/null
@@ -1,83 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package core
-
-import (
- "testing"
-
- "github.com/dexon-foundation/dexon-consensus/core/types"
- "github.com/stretchr/testify/suite"
-)
-
-type BlockPoolTestSuite struct {
- suite.Suite
-}
-
-func (s *BlockPoolTestSuite) TestBasicUsage() {
- // This test case try this flow:
- // - add some blocks into pool.
- // - get tips, check if expected.
- // - get tips, should be identical to previous call.
- // - remove tips, and get tips again, check if expected.
- // - purge one chain, check if expected.
- var (
- req = s.Require()
- pool = newBlockPool(3)
- )
- addBlockWithPosition := func(chainID uint32, height uint64) {
- pool.addBlock(&types.Block{
- Position: types.Position{
- ChainID: chainID,
- Height: height,
- }})
- }
- chkPos := func(b *types.Block, chainID uint32, height uint64) {
- req.Equal(b.Position.ChainID, chainID)
- req.Equal(b.Position.Height, height)
- }
- addBlockWithPosition(0, 0)
- addBlockWithPosition(0, 1)
- addBlockWithPosition(0, 2)
- addBlockWithPosition(0, 3)
- addBlockWithPosition(2, 0)
- addBlockWithPosition(2, 1)
- addBlockWithPosition(2, 2)
- // Check each tip.
- chkPos(pool.tip(0), 0, 0)
- chkPos(pool.tip(2), 2, 0)
- req.Nil(pool.tip(1))
- // Remove tips of chain#0, #1.
- pool.removeTip(0)
- pool.removeTip(1)
- // Get tips of chain#0, #2 back to check.
- chkPos(pool.tip(0), 0, 1)
- chkPos(pool.tip(2), 2, 0) // Chain#2 is untouched.
- // Purge with height lower than lowest height.
- pool.purgeBlocks(0, 0)
- chkPos(pool.tip(0), 0, 1) // Chain#0 is not affected.
- // Purge with height in range.
- pool.purgeBlocks(0, 2)
- chkPos(pool.tip(0), 0, 3) // Height = 1, 2 are purged.
- // Purge with height higher than highest height.
- pool.purgeBlocks(0, 4)
- req.Nil(pool.tip(0)) // Whole chain is purged.
-}
-
-func TestBlockPool(t *testing.T) {
- suite.Run(t, new(BlockPoolTestSuite))
-}
diff --git a/core/compaction-chain.go b/core/compaction-chain.go
deleted file mode 100644
index d7c2f85..0000000
--- a/core/compaction-chain.go
+++ /dev/null
@@ -1,291 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package core
-
-import (
- "fmt"
- "sync"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core/crypto"
- "github.com/dexon-foundation/dexon-consensus/core/types"
- "github.com/dexon-foundation/dexon-consensus/core/utils"
-)
-
-// Errors for compaction chain module.
-var (
- ErrBlockNotRegistered = fmt.Errorf(
- "block not registered")
- ErrNotInitiazlied = fmt.Errorf(
- "not initialized")
- ErrTSigNotReady = fmt.Errorf(
- "tsig not ready")
- ErrIncorrectBlockRandomnessResult = fmt.Errorf(
- "incorrect block randomness result")
-)
-
-const maxPendingPeriod = 3 * time.Second
-const maxRandomnessCache = 100
-
-type pendingRandomnessResult struct {
- receivedTime time.Time
- randResult *types.BlockRandomnessResult
-}
-
-type finalizedBlockHeap = types.ByFinalizationHeight
-
-type compactionChain struct {
- gov Governance
- chainUnsynced uint32
- tsigVerifier *TSigVerifierCache
- blocks map[common.Hash]*types.Block
- blockRandomness map[common.Hash][]byte
- pendingRandomness map[common.Hash]pendingRandomnessResult
- processedRandomnessResult map[types.Position]struct{}
- pendingBlocks []*types.Block
- lock sync.RWMutex
- prevBlock *types.Block
-}
-
-func newCompactionChain(gov Governance) *compactionChain {
- return &compactionChain{
- gov: gov,
- tsigVerifier: NewTSigVerifierCache(gov, 7),
- blocks: make(map[common.Hash]*types.Block),
- blockRandomness: make(map[common.Hash][]byte),
- pendingRandomness: make(map[common.Hash]pendingRandomnessResult),
- processedRandomnessResult: make(map[types.Position]struct{}, maxRandomnessCache),
- }
-}
-
-// init the compaction chain module with a finalized block, or just an empty
-// block for bootstrap case.
-func (cc *compactionChain) init(initBlock *types.Block) {
- cc.lock.Lock()
- defer cc.lock.Unlock()
- cc.prevBlock = initBlock
- cc.pendingBlocks = []*types.Block{}
- // It's the bootstrap case, compactionChain would only deliver blocks until
- // tips of all chains are received.
- if initBlock.Finalization.Height == 0 {
- cc.chainUnsynced = utils.GetConfigWithPanic(cc.gov, 0, nil).NumChains
- }
-}
-
-func (cc *compactionChain) registerBlock(block *types.Block) {
- if cc.blockRegistered(block.Hash) {
- return
- }
- cc.lock.Lock()
- defer cc.lock.Unlock()
- cc.blocks[block.Hash] = block
- if rand, exist := cc.pendingRandomness[block.Hash]; exist {
- cc.blockRandomness[rand.randResult.BlockHash] = rand.randResult.Randomness
- delete(cc.pendingRandomness, block.Hash)
- }
-}
-
-func (cc *compactionChain) blockRegistered(hash common.Hash) bool {
- cc.lock.RLock()
- defer cc.lock.RUnlock()
- return cc.blockRegisteredNoLock(hash)
-}
-
-func (cc *compactionChain) blockRegisteredNoLock(
- hash common.Hash) (exist bool) {
- _, exist = cc.blocks[hash]
- return
-}
-
-func (cc *compactionChain) processBlock(block *types.Block) error {
- prevBlock := cc.lastDeliveredBlock()
- if prevBlock == nil {
- return ErrNotInitiazlied
- }
- cc.lock.Lock()
- defer cc.lock.Unlock()
- if prevBlock.Finalization.Height == 0 && block.Position.Height == 0 {
- cc.chainUnsynced--
- }
- cc.pendingBlocks = append(cc.pendingBlocks, block)
- return nil
-}
-
-func (cc *compactionChain) extractBlocks() []*types.Block {
- // Check if we're synced.
- if !func() bool {
- cc.lock.RLock()
- defer cc.lock.RUnlock()
- if len(cc.pendingBlocks) == 0 {
- return false
- }
- // Finalization.Height == 0 is syncing from bootstrap.
- if cc.prevBlock.Finalization.Height == 0 {
- return cc.chainUnsynced == 0
- }
- return true
- }() {
- return []*types.Block{}
- }
- deliveringBlocks := make([]*types.Block, 0)
- cc.lock.Lock()
- defer cc.lock.Unlock()
- var (
- block *types.Block
- prevBlock = cc.prevBlock
- )
- for len(cc.pendingBlocks) > 0 &&
- (len(cc.blockRandomness[cc.pendingBlocks[0].Hash]) != 0 ||
- cc.pendingBlocks[0].Position.Round == 0) {
- delete(cc.blocks, cc.pendingBlocks[0].Hash)
- block, cc.pendingBlocks = cc.pendingBlocks[0], cc.pendingBlocks[1:]
- block.Finalization.ParentHash = prevBlock.Hash
- block.Finalization.Height = prevBlock.Finalization.Height + 1
- if block.Position.Round != 0 {
- block.Finalization.Randomness = cc.blockRandomness[block.Hash]
- delete(cc.blockRandomness, block.Hash)
- }
- deliveringBlocks = append(deliveringBlocks, block)
- prevBlock = block
- }
- cc.prevBlock = prevBlock
- return deliveringBlocks
-}
-
-func (cc *compactionChain) verifyRandomness(
- blockHash common.Hash, round uint64, randomness []byte) (bool, error) {
- if round == 0 {
- return len(randomness) == 0, nil
- }
- // Randomness is not available at round 0.
- v, ok, err := cc.tsigVerifier.UpdateAndGet(round)
- if err != nil {
- return false, err
- }
- if !ok {
- return false, ErrTSigNotReady
- }
- return v.VerifySignature(blockHash, crypto.Signature{
- Type: "bls",
- Signature: randomness}), nil
-}
-
-func (cc *compactionChain) processFinalizedBlock(block *types.Block) error {
- if block.Finalization.Height <= cc.lastDeliveredBlock().Finalization.Height {
- return nil
- }
- // Block of round 0 should not have randomness.
- if block.Position.Round == 0 && len(block.Finalization.Randomness) != 0 {
- return nil
- }
- cc.lock.Lock()
- defer cc.lock.Unlock()
- // The randomness result is missed previously.
- if cc.blockRegisteredNoLock(block.Hash) {
- ok, err := cc.verifyRandomness(
- block.Hash, block.Position.Round, block.Finalization.Randomness)
- if err != nil {
- return err
- }
- if ok {
- cc.blockRandomness[block.Hash] = block.Finalization.Randomness
- }
- }
- return nil
-}
-
-func (cc *compactionChain) touchBlockRandomnessResult(
- rand *types.BlockRandomnessResult) (first bool) {
- // DO NOT LOCK THIS FUNCTION!!!!!!!! YOU WILL REGRET IT!!!!!
- if _, exist := cc.processedRandomnessResult[rand.Position]; !exist {
- first = true
- if len(cc.processedRandomnessResult) > maxRandomnessCache {
- for k := range cc.processedRandomnessResult {
- // Randomly drop one element.
- delete(cc.processedRandomnessResult, k)
- break
- }
- }
- cc.processedRandomnessResult[rand.Position] = struct{}{}
- }
- return
-}
-
-func (cc *compactionChain) processBlockRandomnessResult(
- rand *types.BlockRandomnessResult) error {
- ok, err := cc.verifyRandomness(
- rand.BlockHash, rand.Position.Round, rand.Randomness)
- if err != nil {
- return err
- }
- if !ok {
- return ErrIncorrectBlockRandomnessResult
- }
- cc.lock.Lock()
- defer cc.lock.Unlock()
- if !cc.blockRegisteredNoLock(rand.BlockHash) {
- cc.purgePending()
- cc.pendingRandomness[rand.BlockHash] = pendingRandomnessResult{
- receivedTime: time.Now(),
- randResult: rand,
- }
- return ErrBlockNotRegistered
- }
- cc.blockRandomness[rand.BlockHash] = rand.Randomness
- return nil
-}
-
-func (cc *compactionChain) purgePending() {
- now := time.Now()
- for key, rand := range cc.pendingRandomness {
- if now.After(rand.receivedTime.Add(maxPendingPeriod)) {
- delete(cc.pendingRandomness, key)
- }
- }
-}
-
-// lastDeliveredBlock returns the last delivered block, or the one used to
-// initialize this module.
-func (cc *compactionChain) lastDeliveredBlock() *types.Block {
- cc.lock.RLock()
- defer cc.lock.RUnlock()
- return cc.prevBlock
-}
-
-// lastPendingBlock returns the last pending block.
-func (cc *compactionChain) lastPendingBlock() *types.Block {
- cc.lock.RLock()
- defer cc.lock.RUnlock()
- if len(cc.pendingBlocks) > 0 {
- return cc.pendingBlocks[0]
- }
- return nil
-}
-
-func (cc *compactionChain) pendingBlocksWithoutRandomness() (
- hashes common.Hashes) {
- cc.lock.RLock()
- defer cc.lock.RUnlock()
- for _, block := range cc.pendingBlocks {
- if _, exist := cc.blockRandomness[block.Hash]; !exist {
- hashes = append(hashes, block.Hash)
- }
- }
- return
-}
diff --git a/core/compaction-chain_test.go b/core/compaction-chain_test.go
deleted file mode 100644
index ca88734..0000000
--- a/core/compaction-chain_test.go
+++ /dev/null
@@ -1,343 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package core
-
-import (
- "testing"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core/crypto"
- "github.com/dexon-foundation/dexon-consensus/core/test"
- "github.com/dexon-foundation/dexon-consensus/core/types"
- "github.com/stretchr/testify/suite"
-)
-
-type CompactionChainTestSuite struct {
- suite.Suite
-}
-
-func (s *CompactionChainTestSuite) SetupTest() {
-}
-
-type mockTSigVerifier struct {
- defaultRet bool
- ret map[common.Hash]bool
-}
-
-func newMockTSigVerifier(defaultRet bool) *mockTSigVerifier {
- return &mockTSigVerifier{
- defaultRet: defaultRet,
- ret: make(map[common.Hash]bool),
- }
-}
-
-func (m *mockTSigVerifier) VerifySignature(
- hash common.Hash, _ crypto.Signature) bool {
- if ret, exist := m.ret[hash]; exist {
- return ret
- }
- return m.defaultRet
-}
-
-func (s *CompactionChainTestSuite) newCompactionChain() (
- *compactionChain, *mockTSigVerifier) {
- _, pubKeys, err := test.NewKeys(4)
- s.Require().NoError(err)
- gov, err := test.NewGovernance(test.NewState(
- pubKeys, 100*time.Millisecond, &common.NullLogger{}, true), ConfigRoundShift)
- s.Require().NoError(err)
- cc := newCompactionChain(gov)
- cc.init(&types.Block{})
-
- mock := newMockTSigVerifier(true)
- for i := 0; i < cc.tsigVerifier.cacheSize; i++ {
- cc.tsigVerifier.verifier[uint64(i)] = mock
- }
-
- return cc, mock
-}
-
-func (s *CompactionChainTestSuite) TestProcessBlock() {
- cc, _ := s.newCompactionChain()
- now := time.Now().UTC()
- blocks := make([]*types.Block, 10)
- for idx := range blocks {
- blocks[idx] = &types.Block{
- Hash: common.NewRandomHash(),
- Finalization: types.FinalizationResult{
- Timestamp: now,
- },
- }
- now = now.Add(100 * time.Millisecond)
- }
- for _, block := range blocks {
- s.Require().NoError(cc.processBlock(block))
- }
- s.Len(cc.pendingBlocks, len(blocks))
-}
-
-func (s *CompactionChainTestSuite) TestExtractBlocks() {
- cc, _ := s.newCompactionChain()
- s.Require().Equal(uint32(4), cc.gov.Configuration(uint64(0)).NumChains)
- blocks := make([]*types.Block, 10)
- for idx := range blocks {
- blocks[idx] = &types.Block{
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Round: 1,
- ChainID: uint32(idx % 4),
- },
- }
- s.Require().False(cc.blockRegistered(blocks[idx].Hash))
- cc.registerBlock(blocks[idx])
- s.Require().True(cc.blockRegistered(blocks[idx].Hash))
- }
- // Randomness is ready for extract.
- for i := 0; i < 4; i++ {
- s.Require().NoError(cc.processBlock(blocks[i]))
- h := common.NewRandomHash()
- s.Require().NoError(cc.processBlockRandomnessResult(
- &types.BlockRandomnessResult{
- BlockHash: blocks[i].Hash,
- Position: blocks[i].Position,
- Randomness: h[:],
- }))
- }
- delivered := cc.extractBlocks()
- s.Require().Len(delivered, 4)
- s.Require().Equal(uint32(0), cc.chainUnsynced)
- // Randomness is not yet ready for extract.
- for i := 4; i < 6; i++ {
- s.Require().NoError(cc.processBlock(blocks[i]))
- }
- delivered = append(delivered, cc.extractBlocks()...)
- s.Require().Len(delivered, 4)
- // Make some randomness ready.
- for i := 4; i < 6; i++ {
- h := common.NewRandomHash()
- s.Require().NoError(cc.processBlockRandomnessResult(
- &types.BlockRandomnessResult{
- BlockHash: blocks[i].Hash,
- Position: blocks[i].Position,
- Randomness: h[:],
- }))
- }
- delivered = append(delivered, cc.extractBlocks()...)
- s.Require().Len(delivered, 6)
- // Later block's randomness is ready.
- for i := 6; i < 10; i++ {
- s.Require().NoError(cc.processBlock(blocks[i]))
- if i < 8 {
- continue
- }
- h := common.NewRandomHash()
- s.Require().NoError(cc.processBlockRandomnessResult(
- &types.BlockRandomnessResult{
- BlockHash: blocks[i].Hash,
- Position: blocks[i].Position,
- Randomness: h[:],
- }))
- }
- delivered = append(delivered, cc.extractBlocks()...)
- s.Require().Len(delivered, 6)
- // Prior block's randomness is ready.
- for i := 6; i < 8; i++ {
- h := common.NewRandomHash()
- s.Require().NoError(cc.processBlockRandomnessResult(
- &types.BlockRandomnessResult{
- BlockHash: blocks[i].Hash,
- Position: blocks[i].Position,
- Randomness: h[:],
- }))
- }
- delivered = append(delivered, cc.extractBlocks()...)
- s.Require().Len(delivered, 10)
- // The delivered order should be the same as processing order.
- for i, block := range delivered {
- if i > 1 {
- s.Equal(delivered[i-1].Finalization.Height+1,
- delivered[i].Finalization.Height)
- s.Equal(delivered[i-1].Hash,
- delivered[i].Finalization.ParentHash)
- }
- s.Equal(block.Hash, blocks[i].Hash)
- }
-}
-
-func (s *CompactionChainTestSuite) TestMissedRandomness() {
- // This test case makes sure a block's randomness field can be fulfilled by
- // calling:
- // - core.compactionChain.processBlockRandomnessResult
- // - core.compactionChain.processFinalizedBlock
- cc, _ := s.newCompactionChain()
- s.Require().Equal(uint32(4), cc.gov.Configuration(uint64(0)).NumChains)
- blocks := make([]*types.Block, 10)
- for idx := range blocks {
- blocks[idx] = &types.Block{
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Round: 1,
- Height: uint64(idx / 4),
- ChainID: uint32(idx % 4),
- },
- }
- s.Require().False(cc.blockRegistered(blocks[idx].Hash))
- cc.registerBlock(blocks[idx])
- s.Require().True(cc.blockRegistered(blocks[idx].Hash))
- }
- noRandBlocks := common.Hashes{}
- // Block#4, #5, contains randomness.
- for i := range blocks {
- s.Require().NoError(cc.processBlock(blocks[i]))
- if i >= 4 && i < 6 {
- h := common.NewRandomHash()
- s.Require().NoError(cc.processBlockRandomnessResult(
- &types.BlockRandomnessResult{
- BlockHash: blocks[i].Hash,
- Position: blocks[i].Position,
- Randomness: h[:],
- }))
- } else {
- noRandBlocks = append(noRandBlocks, blocks[i].Hash)
- }
- }
- s.Equal(noRandBlocks, cc.pendingBlocksWithoutRandomness())
- s.Require().Len(cc.extractBlocks(), 0)
- // Give compactionChain module randomnessResult via finalized block
- // #0, #1, #2, #3, #4.
- for i := range blocks {
- if i >= 4 {
- break
- }
- block := blocks[i].Clone()
- h := common.NewRandomHash()
- block.Finalization.Randomness = h[:]
- block.Finalization.Height = uint64(i + 1)
- cc.processFinalizedBlock(block)
- }
- // Block #0-3 has randomness result.
- noRandBlocks = noRandBlocks[4:]
- s.Equal(noRandBlocks, cc.pendingBlocksWithoutRandomness())
- delivered := cc.extractBlocks()
- s.Require().Len(delivered, 6)
- // Give compactionChain module randomnessResult#6-9.
- for i := 6; i < 10; i++ {
- h := common.NewRandomHash()
- s.Require().NoError(cc.processBlockRandomnessResult(
- &types.BlockRandomnessResult{
- BlockHash: blocks[i].Hash,
- Position: blocks[i].Position,
- Randomness: h[:],
- }))
- }
- s.Len(cc.pendingBlocksWithoutRandomness(), 0)
- delivered = append(delivered, cc.extractBlocks()...)
- s.Require().Len(delivered, 10)
- // The delivered order should be the same as processing order.
- for i, block := range delivered {
- if i > 1 {
- s.Equal(delivered[i-1].Finalization.Height+1,
- delivered[i].Finalization.Height)
- s.Equal(delivered[i-1].Hash,
- delivered[i].Finalization.ParentHash)
- }
- s.Equal(block.Hash, blocks[i].Hash)
- }
-}
-
-func (s *CompactionChainTestSuite) TestExtractBlocksRound0() {
- cc, _ := s.newCompactionChain()
- s.Require().Equal(uint32(4), cc.gov.Configuration(uint64(0)).NumChains)
- blocks := make([]*types.Block, 10)
- for idx := range blocks {
- blocks[idx] = &types.Block{
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Round: 0,
- },
- }
- s.Require().False(cc.blockRegistered(blocks[idx].Hash))
- cc.registerBlock(blocks[idx])
- s.Require().True(cc.blockRegistered(blocks[idx].Hash))
- }
- // Round 0 should be able to be extracted without randomness.
- for i := 0; i < 4; i++ {
- s.Require().NoError(cc.processBlock(blocks[i]))
- }
- delivered := cc.extractBlocks()
- s.Require().Len(delivered, 4)
- // Round 0 should be able to be extracted without randomness.
- for i := 4; i < 10; i++ {
- s.Require().NoError(cc.processBlock(blocks[i]))
- }
- delivered = append(delivered, cc.extractBlocks()...)
- s.Require().Len(delivered, 10)
- // The delivered order should be the same as processing order.
- for i, block := range delivered {
- s.Equal(block.Hash, blocks[i].Hash)
- }
-}
-
-func (s *CompactionChainTestSuite) TestBootstrapSync() {
- // This test case make sure compactionChain module would only deliver
- // blocks unless tips of each chain are received, when this module is
- // initialized with a block with finalizationHeight == 0.
- cc, _ := s.newCompactionChain()
- numChains := cc.gov.Configuration(uint64(0)).NumChains
- s.Require().Equal(uint32(4), numChains)
- now := time.Now().UTC()
- blocks := make([]*types.Block, 20)
- for idx := range blocks {
- blocks[idx] = &types.Block{
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: uint64(idx) / uint64(numChains),
- },
- Finalization: types.FinalizationResult{
- Timestamp: now,
- Height: uint64(idx + 1),
- },
- }
- now = now.Add(100 * time.Millisecond)
- }
- s.Require().NoError(cc.processBlock(blocks[1]))
- s.Len(cc.extractBlocks(), 0)
- s.Require().NoError(cc.processBlock(blocks[2]))
- s.Len(cc.extractBlocks(), 0)
- // Although genesis block is received, we can't deliver them until tip blocks
- // of each chain is received.
- s.Require().NoError(cc.processBlock(blocks[0]))
- s.Len(cc.extractBlocks(), 0)
- // Once we receive the tip of chain#3 then we can deliver all tips.
- s.Require().NoError(cc.processBlock(blocks[3]))
- confirmed := cc.extractBlocks()
- s.Require().Len(confirmed, 4)
- s.Equal(confirmed[0].Hash, blocks[1].Hash)
- s.Equal(blocks[1].Finalization.Height, uint64(1))
- s.Equal(confirmed[1].Hash, blocks[2].Hash)
- s.Equal(blocks[2].Finalization.Height, uint64(2))
- s.Equal(confirmed[2].Hash, blocks[0].Hash)
- s.Equal(blocks[0].Finalization.Height, uint64(3))
- s.Equal(confirmed[3].Hash, blocks[3].Hash)
- s.Equal(blocks[3].Finalization.Height, uint64(4))
-}
-
-func TestCompactionChain(t *testing.T) {
- suite.Run(t, new(CompactionChainTestSuite))
-}
diff --git a/core/consensus-timestamp.go b/core/consensus-timestamp.go
deleted file mode 100644
index d7ce8e2..0000000
--- a/core/consensus-timestamp.go
+++ /dev/null
@@ -1,162 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package core
-
-import (
- "errors"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/core/types"
-)
-
-// consensusTimestamp calculates a block's finalization timestamp. Notice that
-// the Finalization timestamps are increasing (but not strictly increasing).
-// Instance functions:
-// - processBlocks(blocks []*types.Block) error
-// called with blocks output from total ordering
-// - appendConfig(round uint64, config *types.Config) error
-// called when a new config is known
-// - synced() bool
-// called in sync mode
-type consensusTimestamp struct {
- timestampsOfChains []time.Time
-
- // Stores number of chains for rounds.
- numChainsOfRounds []uint32
- numChainsBase uint64
-
- // dMoment represents the genesis time.
- dMoment time.Time
-
- // lastTimestamp is the previous assigned consensus timestamp.
- lastTimestamp time.Time
-}
-
-var (
- // ErrTimestampNotIncrease for timestamp is not strictly increasing on one
- // chain.
- ErrTimestampNotIncrease = errors.New("timestamp is not increasing")
- // ErrNoRoundConfig for no round config found.
- ErrNoRoundConfig = errors.New("no round config found")
- // ErrConsensusTimestampRewind for rewinding timestamp.
- ErrConsensusTimestampRewind = errors.New("consensus timestamp rewind")
-)
-
-// newConsensusTimestamp creates consensus timestamp instance.
-func newConsensusTimestamp(
- dMoment time.Time, round uint64, numChains uint32) *consensusTimestamp {
-
- ts := make([]time.Time, numChains)
- for i := range ts {
- ts[i] = dMoment
- }
- return &consensusTimestamp{
- numChainsOfRounds: []uint32{numChains},
- numChainsBase: round,
- dMoment: dMoment,
- timestampsOfChains: ts,
- }
-}
-
-// appendConfig appends a configuration for upcoming round. When you append
-// a config for round R, next time you can only append the config for round R+1.
-func (ct *consensusTimestamp) appendConfig(
- round uint64, config *types.Config) error {
- if round != uint64(len(ct.numChainsOfRounds))+ct.numChainsBase {
- return ErrRoundNotIncreasing
- }
- // This segment is to handle the corner case for config checking logic in
- // processBlock method.
- if len(ct.numChainsOfRounds) == 1 {
- if ct.numChainsOfRounds[0] > config.NumChains {
- ct.resizeTimetamps(ct.numChainsOfRounds[0])
- } else {
- ct.resizeTimetamps(config.NumChains)
- }
- }
- ct.numChainsOfRounds = append(ct.numChainsOfRounds, config.NumChains)
- return nil
-}
-
-func (ct *consensusTimestamp) resizeTimetamps(numChain uint32) {
- l := uint32(len(ct.timestampsOfChains))
- if numChain > l {
- for i := l; i < numChain; i++ {
- ct.timestampsOfChains = append(ct.timestampsOfChains, ct.dMoment)
- }
- } else if numChain < l {
- ct.timestampsOfChains = ct.timestampsOfChains[:numChain]
- }
-}
-
-// ProcessBlocks is the entry function.
-func (ct *consensusTimestamp) processBlocks(blocks []*types.Block) (err error) {
- for _, block := range blocks {
- // Rounds might interleave within rounds if no configuration change
- // occurs. And it is limited to one round, that is, round r can only
- // interleave with r-1 and r+1.
- round := block.Position.Round
- if ct.numChainsBase == round || ct.numChainsBase+1 == round {
- // Normal case, no need to modify timestampsOfChains.
- } else if ct.numChainsBase+2 == round {
- // Resize timestampsOfChains if block from r+2 comes, because the interleave
- // of rounds must be less than 1. Resize the size to
- // max(numChainsOfRounds[r+1], numChainsOfRounds[r+2]).
- if len(ct.numChainsOfRounds) < 2 {
- return ErrNoRoundConfig
- }
- ct.numChainsBase++
- ct.numChainsOfRounds = ct.numChainsOfRounds[1:]
- if ct.numChainsOfRounds[0] > ct.numChainsOfRounds[1] {
- ct.resizeTimetamps(ct.numChainsOfRounds[0])
- } else {
- ct.resizeTimetamps(ct.numChainsOfRounds[1])
- }
- } else {
- // Error if round < base or round > base + 2.
- return ErrInvalidRoundID
- }
- ts := ct.timestampsOfChains[:ct.numChainsOfRounds[round-ct.numChainsBase]]
- if block.Finalization.Timestamp, err = getMedianTime(ts); err != nil {
- return
- }
- if block.Timestamp.Before(ct.timestampsOfChains[block.Position.ChainID]) {
- return ErrTimestampNotIncrease
- }
- ct.timestampsOfChains[block.Position.ChainID] = block.Timestamp
- // If the finalization timestamp is before the last timestamp, set it to
- // the last one. Notice that the finalization timestamps are increasing but
- // not strictly increasing.
- if block.Finalization.Timestamp.Before(ct.lastTimestamp) {
- block.Finalization.Timestamp = ct.lastTimestamp
- } else {
- ct.lastTimestamp = block.Finalization.Timestamp
- }
- }
- return
-}
-
-func (ct *consensusTimestamp) isSynced() bool {
- numChain := ct.numChainsOfRounds[0]
- for i := uint32(0); i < numChain; i++ {
- if ct.timestampsOfChains[i].Equal(ct.dMoment) {
- return false
- }
- }
- return true
-}
diff --git a/core/consensus-timestamp_test.go b/core/consensus-timestamp_test.go
deleted file mode 100644
index 9d199fe..0000000
--- a/core/consensus-timestamp_test.go
+++ /dev/null
@@ -1,221 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package core
-
-import (
- "math"
- "math/rand"
- "testing"
- "time"
-
- "github.com/stretchr/testify/suite"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core/types"
-)
-
-type ConsensusTimestampTest struct {
- suite.Suite
-}
-
-func (s *ConsensusTimestampTest) generateBlocksWithTimestamp(
- now time.Time,
- 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)
- }
- 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)] = now
- } 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.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 blocks
-}
-
-func (s *ConsensusTimestampTest) extractTimestamps(
- blocks []*types.Block) []time.Time {
- timestamps := make([]time.Time, 0, len(blocks))
- for _, block := range blocks {
- if block.IsGenesis() {
- continue
- }
- timestamps = append(timestamps, block.Finalization.Timestamp)
- }
- return timestamps
-}
-
-// 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}
- chainNum := 19
- sigma := 100 * time.Millisecond
- totalTimestamps := make([]time.Time, 0)
- now := time.Now().UTC()
- ct := newConsensusTimestamp(now, 0, uint32(chainNum))
- totalBlockNum := 0
- for _, blockNum := range blockNums {
- totalBlockNum += blockNum
- }
- totalChain := s.generateBlocksWithTimestamp(now,
- totalBlockNum, chainNum, 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(now, 0, uint32(chainNum))
- err := ct2.processBlocks(totalChain)
- s.Require().NoError(err)
- timestamps2 := s.extractTimestamps(totalChain)
- s.Equal(totalTimestamps, timestamps2)
-}
-
-func (s *ConsensusTimestampTest) TestTimestampIncrease() {
- chainNum := 19
- sigma := 100 * time.Millisecond
- now := time.Now().UTC()
- ct := newConsensusTimestamp(now, 0, uint32(chainNum))
- chain := s.generateBlocksWithTimestamp(
- now, 1000, chainNum, time.Second, sigma)
- err := ct.processBlocks(chain)
- s.Require().NoError(err)
- timestamps := s.extractTimestamps(chain)
- for i := 1; i < len(timestamps); i++ {
- s.False(timestamps[i].Before(timestamps[i-1]))
- }
- // Test if the processBlocks is stable.
- ct2 := newConsensusTimestamp(now, 0, uint32(chainNum))
- ct2.processBlocks(chain)
- s.Require().NoError(err)
- timestamps2 := s.extractTimestamps(chain)
- s.Equal(timestamps, timestamps2)
-}
-
-func (s *ConsensusTimestampTest) TestTimestampConfigChange() {
- chainNum := 19
- sigma := 100 * time.Millisecond
- now := time.Now().UTC()
- ct := newConsensusTimestamp(now, 20, uint32(chainNum))
- chain := s.generateBlocksWithTimestamp(now,
- 1000, chainNum, time.Second, sigma)
- blocks := make([]*types.Block, 0, 1000)
- ct.appendConfig(21, &types.Config{NumChains: uint32(16)})
- ct.appendConfig(22, &types.Config{NumChains: uint32(19)})
- // Blocks 0 to 299 is in round 20, blocks 300 to 599 is in round 21 and ignore
- // blocks which ChainID is 16 to 18, blocks 600 to 999 is in round 22.
- for i := 0; i < 1000; i++ {
- add := true
- if i < 300 {
- chain[i].Position.Round = 20
- } else if i < 600 {
- chain[i].Position.Round = 21
- add = chain[i].Position.ChainID < 16
- } else {
- chain[i].Position.Round = 22
- }
- if add {
- blocks = append(blocks, chain[i])
- }
- }
- err := ct.processBlocks(blocks)
- s.Require().NoError(err)
-}
-
-func (s *ConsensusTimestampTest) TestTimestampRoundInterleave() {
- chainNum := 9
- sigma := 100 * time.Millisecond
- now := time.Now().UTC()
- ct := newConsensusTimestamp(now, 0, uint32(chainNum))
- ct.appendConfig(1, &types.Config{NumChains: uint32(chainNum)})
- chain := s.generateBlocksWithTimestamp(now,
- 100, chainNum, time.Second, sigma)
- for i := 50; i < 100; i++ {
- chain[i].Position.Round = 1
- }
- chain[48].Position.Round = 1
- chain[49].Position.Round = 1
- chain[50].Position.Round = 0
- chain[51].Position.Round = 0
- err := ct.processBlocks(chain)
- s.Require().NoError(err)
-}
-
-func (s *ConsensusTimestampTest) TestNumChainsChangeAtSecondAppendedRound() {
- now := time.Now().UTC()
- ct := newConsensusTimestamp(now, 1, 4)
- s.Require().NoError(ct.appendConfig(2, &types.Config{NumChains: 5}))
- // We should be able to handle a block from the second appended round.
- s.Require().NoError(ct.processBlocks([]*types.Block{
- &types.Block{
- Position: types.Position{Round: 2},
- Timestamp: now.Add(1 * time.Second),
- }}))
-}
-
-func (s *ConsensusTimestampTest) TestTimestampSync() {
- chainNum := 19
- sigma := 100 * time.Millisecond
- now := time.Now().UTC()
- ct := newConsensusTimestamp(now, 0, uint32(chainNum))
- chain := s.generateBlocksWithTimestamp(now,
- 100, chainNum, time.Second, sigma)
- err := ct.processBlocks(chain[:chainNum-1])
- s.Require().NoError(err)
- s.Require().False(ct.isSynced())
- err = ct.processBlocks(chain[chainNum-1:])
- s.Require().NoError(err)
- s.Require().True(ct.isSynced())
-}
-
-func TestConsensusTimestamp(t *testing.T) {
- suite.Run(t, new(ConsensusTimestampTest))
-}
diff --git a/core/consensus.go b/core/consensus.go
index f465010..ebbcbd1 100644
--- a/core/consensus.go
+++ b/core/consensus.go
@@ -57,10 +57,9 @@ var (
// consensusBAReceiver implements agreementReceiver.
type consensusBAReceiver struct {
- // TODO(mission): consensus would be replaced by lattice and network.
+ // TODO(mission): consensus would be replaced by blockChain and network.
consensus *Consensus
agreementModule *agreement
- chainID uint32
changeNotaryTime time.Time
roundValue *atomic.Value
isNotary bool
@@ -96,9 +95,9 @@ func (recv *consensusBAReceiver) ProposeBlock() common.Hash {
if !recv.isNotary {
return common.Hash{}
}
- block := recv.consensus.proposeBlock(recv.chainID, recv.round())
- if block == nil {
- recv.consensus.logger.Error("unable to propose block")
+ block, err := recv.consensus.proposeBlock(recv.agreementModule.agreementID())
+ if err != nil || block == nil {
+ recv.consensus.logger.Error("unable to propose block", "error", err)
return types.NullBlockHash
}
go func() {
@@ -115,25 +114,29 @@ func (recv *consensusBAReceiver) ProposeBlock() common.Hash {
func (recv *consensusBAReceiver) ConfirmBlock(
hash common.Hash, votes map[types.NodeID]*types.Vote) {
- var block *types.Block
+ var (
+ block *types.Block
+ aID = recv.agreementModule.agreementID()
+ )
isEmptyBlockConfirmed := hash == common.Hash{}
if isEmptyBlockConfirmed {
- aID := recv.agreementModule.agreementID()
- recv.consensus.logger.Info("Empty block is confirmed",
- "position", &aID)
+ recv.consensus.logger.Info("Empty block is confirmed", "position", aID)
var err error
- block, err = recv.consensus.proposeEmptyBlock(recv.round(), recv.chainID)
+ block, err = recv.consensus.bcModule.addEmptyBlock(aID)
if err != nil {
- recv.consensus.logger.Error("Propose empty block failed", "error", err)
+ recv.consensus.logger.Error("Add position for empty failed",
+ "error", err)
return
}
+ if block == nil {
+ panic(fmt.Errorf("empty block should be proposed directly: %s", aID))
+ }
} else {
var exist bool
block, exist = recv.agreementModule.findBlockNoLock(hash)
if !exist {
recv.consensus.logger.Error("Unknown block confirmed",
- "hash", hash.String()[:6],
- "chainID", recv.chainID)
+ "hash", hash.String()[:6])
ch := make(chan *types.Block)
func() {
recv.consensus.lock.Lock()
@@ -155,8 +158,7 @@ func (recv *consensusBAReceiver) ConfirmBlock(
}
recv.consensus.logger.Info("Receive unknown block",
"hash", hash.String()[:6],
- "position", &block.Position,
- "chainID", recv.chainID)
+ "position", block.Position)
recv.agreementModule.addCandidateBlock(block)
recv.agreementModule.lock.Lock()
defer recv.agreementModule.lock.Unlock()
@@ -165,15 +167,14 @@ func (recv *consensusBAReceiver) ConfirmBlock(
return
}
}
- recv.consensus.ccModule.registerBlock(block)
if block.Position.Height != 0 &&
- !recv.consensus.lattice.Exist(block.ParentHash) {
+ !recv.consensus.bcModule.confirmed(block.Position.Height-1) {
go func(hash common.Hash) {
parentHash := hash
for {
recv.consensus.logger.Warn("Parent block not confirmed",
"parent-hash", parentHash.String()[:6],
- "cur-position", &block.Position)
+ "cur-position", block.Position)
ch := make(chan *types.Block)
if !func() bool {
recv.consensus.lock.Lock()
@@ -200,13 +201,12 @@ func (recv *consensusBAReceiver) ConfirmBlock(
}
recv.consensus.logger.Info("Receive parent block",
"parent-hash", block.ParentHash.String()[:6],
- "cur-position", &block.Position,
- "chainID", recv.chainID)
- recv.consensus.ccModule.registerBlock(block)
+ "cur-position", block.Position)
recv.consensus.processBlockChan <- block
parentHash = block.ParentHash
if block.Position.Height == 0 ||
- recv.consensus.lattice.Exist(parentHash) {
+ recv.consensus.bcModule.confirmed(
+ block.Position.Height-1) {
return
}
}
@@ -372,11 +372,6 @@ type Consensus struct {
dkgReady *sync.Cond
cfgModule *configurationChain
- // Dexon consensus v1's modules.
- lattice *Lattice
- ccModule *compactionChain
- toSyncer *totalOrderingSyncer
-
// Interfaces.
db db.Database
app Application
@@ -385,21 +380,21 @@ type Consensus struct {
network Network
// Misc.
- dMoment time.Time
- nodeSetCache *utils.NodeSetCache
- round uint64
- roundForNewConfig uint64
- lock sync.RWMutex
- ctx context.Context
- ctxCancel context.CancelFunc
- event *common.Event
- logger common.Logger
- nonFinalizedBlockDelivered bool
- resetRandomnessTicker chan struct{}
- resetDeliveryGuardTicker chan struct{}
- msgChan chan interface{}
- waitGroup sync.WaitGroup
- processBlockChan chan *types.Block
+ bcModule *blockChain
+ dMoment time.Time
+ nodeSetCache *utils.NodeSetCache
+ round uint64
+ roundForNewConfig uint64
+ lock sync.RWMutex
+ ctx context.Context
+ ctxCancel context.CancelFunc
+ event *common.Event
+ logger common.Logger
+ resetRandomnessTicker chan struct{}
+ resetDeliveryGuardTicker chan struct{}
+ msgChan chan interface{}
+ waitGroup sync.WaitGroup
+ processBlockChan chan *types.Block
// Context of Dummy receiver during switching from syncer.
dummyCancel context.CancelFunc
@@ -417,7 +412,7 @@ func NewConsensus(
prv crypto.PrivateKey,
logger common.Logger) *Consensus {
return newConsensusForRound(
- &types.Block{}, dMoment, app, gov, db, network, prv, logger, nil, true)
+ nil, dMoment, app, gov, db, network, prv, logger, true)
}
// NewConsensusForSimulation creates an instance of Consensus for simulation,
@@ -431,7 +426,7 @@ func NewConsensusForSimulation(
prv crypto.PrivateKey,
logger common.Logger) *Consensus {
return newConsensusForRound(
- &types.Block{}, dMoment, app, gov, db, network, prv, logger, nil, false)
+ nil, dMoment, app, gov, db, network, prv, logger, false)
}
// NewConsensusFromSyncer constructs an Consensus instance from information
@@ -451,14 +446,13 @@ func NewConsensusFromSyncer(
db db.Database,
networkModule Network,
prv crypto.PrivateKey,
- latticeModule *Lattice,
- confirmedBlocks [][]*types.Block,
+ confirmedBlocks []*types.Block,
randomnessResults []*types.BlockRandomnessResult,
cachedMessages []interface{},
logger common.Logger) (*Consensus, error) {
// Setup Consensus instance.
con := newConsensusForRound(initBlock, initRoundBeginTime, app, gov, db,
- networkModule, prv, logger, latticeModule, true)
+ networkModule, prv, logger, true)
// Launch a dummy receiver before we start receiving from network module.
con.dummyMsgBuffer = cachedMessages
con.dummyCancel, con.dummyFinished = utils.LaunchDummyReceiver(
@@ -467,29 +461,18 @@ func NewConsensusFromSyncer(
})
// Dump all BA-confirmed blocks to the consensus instance, make sure these
// added blocks forming a DAG.
- for {
- updated := false
- for idx, bs := range confirmedBlocks {
- for bIdx, b := range bs {
- // Only when its parent block is already added to lattice, we can
- // then add this block. If not, our pulling mechanism would stop at
- // the block we added, and lost its parent block forever.
- if !latticeModule.Exist(b.ParentHash) {
- logger.Debug("Skip discontinuous confirmed block",
- "from", b,
- "until", bs[len(bs)-1])
- confirmedBlocks[idx] = bs[bIdx:]
- break
- }
- con.ccModule.registerBlock(b)
- if err := con.processBlock(b); err != nil {
- return nil, err
- }
- }
- }
- if !updated {
+ refBlock := initBlock
+ for _, b := range confirmedBlocks {
+ // Only when its parent block is already added to lattice, we can
+ // then add this block. If not, our pulling mechanism would stop at
+ // the block we added, and lost its parent block forever.
+ if b.Position.Height != refBlock.Position.Height+1 {
break
}
+ if err := con.processBlock(b); err != nil {
+ return nil, err
+ }
+ refBlock = b
}
// Dump all randomness result to the consensus instance.
for _, r := range randomnessResults {
@@ -502,7 +485,7 @@ func NewConsensusFromSyncer(
return con, nil
}
-// newConsensus creates a Consensus instance.
+// newConsensusForRound creates a Consensus instance.
func newConsensusForRound(
initBlock *types.Block,
initRoundBeginTime time.Time,
@@ -512,9 +495,7 @@ func newConsensusForRound(
network Network,
prv crypto.PrivateKey,
logger common.Logger,
- latticeModule *Lattice,
usingNonBlocking bool) *Consensus {
-
// TODO(w): load latest blockHeight from DB, and use config at that height.
nodeSetCache := utils.NewNodeSetCache(gov)
// Setup signer module.
@@ -525,13 +506,11 @@ func newConsensusForRound(
debugApp = a
}
// Get configuration for bootstrap round.
- initRound := initBlock.Position.Round
- initConfig := utils.GetConfigWithPanic(gov, initRound, logger)
- // Init lattice.
- if latticeModule == nil {
- latticeModule = NewLattice(initRoundBeginTime, initRound, initConfig,
- signer, app, debugApp, db, logger)
+ initRound := uint64(0)
+ if initBlock != nil {
+ initRound = initBlock.Position.Round
}
+ initConfig := utils.GetConfigWithPanic(gov, initRound, logger)
// Init configuration chain.
ID := types.NewNodeID(prv.PublicKey())
recv := &consensusDKGReceiver{
@@ -548,11 +527,14 @@ func newConsensusForRound(
if usingNonBlocking {
appModule = newNonBlocking(app, debugApp)
}
+ bcConfig := blockChainConfig{}
+ bcConfig.fromConfig(initRound, initConfig)
+ bcConfig.setRoundBeginTime(initRoundBeginTime)
+ bcModule := newBlockChain(ID, initBlock, bcConfig, appModule,
+ NewTSigVerifierCache(gov, 7), signer, logger)
// Construct Consensus instance.
con := &Consensus{
ID: ID,
- ccModule: newCompactionChain(gov),
- lattice: latticeModule,
app: appModule,
debugApp: debugApp,
gov: gov,
@@ -561,6 +543,7 @@ func newConsensusForRound(
baConfirmedBlock: make(map[common.Hash]chan<- *types.Block),
dkgReady: sync.NewCond(&sync.Mutex{}),
cfgModule: cfgModule,
+ bcModule: bcModule,
dMoment: initRoundBeginTime,
nodeSetCache: nodeSetCache,
signer: signer,
@@ -581,37 +564,36 @@ func newConsensusForRound(
// prepare the Consensus instance to be ready for blocks after 'initBlock'.
// 'initBlock' could be either:
-// - an empty block
+// - nil
// - the last finalized block
-func (con *Consensus) prepare(initBlock *types.Block) error {
+func (con *Consensus) prepare(initBlock *types.Block) (err error) {
// The block past from full node should be delivered already or known by
// full node. We don't have to notify it.
- con.roundForNewConfig = initBlock.Position.Round + 1
- initRound := initBlock.Position.Round
+ initRound := uint64(0)
+ if initBlock != nil {
+ initRound = initBlock.Position.Round
+ }
+ con.roundForNewConfig = initRound + 1
initConfig := utils.GetConfigWithPanic(con.gov, initRound, con.logger)
// Setup context.
- con.ccModule.init(initBlock)
con.logger.Debug("Calling Governance.CRS", "round", initRound)
initCRS := con.gov.CRS(initRound)
if (initCRS == common.Hash{}) {
- return ErrCRSNotReady
+ err = ErrCRSNotReady
+ return
}
- if err := con.baMgr.appendConfig(initRound, initConfig, initCRS); err != nil {
- return err
+ if err = con.baMgr.appendConfig(initRound, initConfig, initCRS); err != nil {
+ return
}
- // Setup lattice module.
+ // Setup blockChain module.
initPlusOneCfg := utils.GetConfigWithPanic(con.gov, initRound+1, con.logger)
- if err := con.lattice.AppendConfig(initRound+1, initPlusOneCfg); err != nil {
- if err == ErrRoundNotIncreasing {
- err = nil
- } else {
- return err
- }
+ if err = con.bcModule.appendConfig(initRound+1, initPlusOneCfg); err != nil {
+ return
}
// Register events.
dkgSet, err := con.nodeSetCache.GetDKGSet(initRound)
if err != nil {
- return err
+ return
}
if _, exist := dkgSet[con.ID]; exist {
con.logger.Info("Selected as DKG set", "round", initRound)
@@ -626,7 +608,7 @@ func (con *Consensus) prepare(initBlock *types.Block) error {
}()
}
con.initialRound(con.dMoment, initRound, initConfig)
- return nil
+ return
}
// Run starts running DEXON Consensus.
@@ -847,7 +829,7 @@ func (con *Consensus) initialRound(
})
}(round + 1)
})
- // Prepare lattice module for next round and next "initialRound" routine.
+ // Prepare blockChain module for next round and next "initialRound" routine.
con.event.RegisterTime(startTime.Add(config.RoundInterval),
func(time.Time) {
// Change round.
@@ -917,7 +899,7 @@ MessageLoop:
ch, e := con.baConfirmedBlock[val.Hash]
return ch, e
}(); exist {
- if err := con.lattice.SanityCheck(val, false); err != nil {
+ if err := con.bcModule.sanityCheck(val); err != nil {
if err == ErrRetrySanityCheckLater {
err = nil
} else {
@@ -965,7 +947,7 @@ MessageLoop:
if err := con.ProcessBlockRandomnessResult(val, true); err != nil {
con.logger.Error("Failed to process block randomness result",
"hash", val.BlockHash.String()[:6],
- "position", &val.Position,
+ "position", val.Position,
"error", err)
}
case *typesDKG.PrivateShare:
@@ -983,34 +965,6 @@ MessageLoop:
}
}
-func (con *Consensus) proposeBlock(chainID uint32, round uint64) *types.Block {
- block := &types.Block{
- Position: types.Position{
- ChainID: chainID,
- Round: round,
- },
- }
- if err := con.prepareBlock(block, time.Now().UTC()); err != nil {
- con.logger.Error("Failed to prepare block", "error", err)
- return nil
- }
- return block
-}
-
-func (con *Consensus) proposeEmptyBlock(
- round uint64, chainID uint32) (*types.Block, error) {
- block := &types.Block{
- Position: types.Position{
- Round: round,
- ChainID: chainID,
- },
- }
- if err := con.lattice.PrepareEmptyBlock(block); err != nil {
- return nil, err
- }
- return block, nil
-}
-
// ProcessVote is the entry point to submit ont vote to a Consensus instance.
func (con *Consensus) ProcessVote(vote *types.Vote) (err error) {
v := vote.Clone()
@@ -1024,14 +978,11 @@ func (con *Consensus) ProcessAgreementResult(
if !con.baMgr.touchAgreementResult(rand) {
return nil
}
-
// Sanity Check.
if err := VerifyAgreementResult(rand, con.nodeSetCache); err != nil {
con.baMgr.untouchAgreementResult(rand)
return err
}
- con.lattice.AddShallowBlock(rand.BlockHash, rand.Position)
-
// Syncing BA Module.
if err := con.baMgr.processAgreementResult(rand); err != nil {
return err
@@ -1089,7 +1040,7 @@ func (con *Consensus) ProcessAgreementResult(
if err != nil {
if err != ErrTSigAlreadyRunning {
con.logger.Error("Failed to run TSIG",
- "position", &rand.Position,
+ "position", rand.Position,
"hash", rand.BlockHash.String()[:6],
"error", err)
}
@@ -1113,15 +1064,8 @@ func (con *Consensus) ProcessBlockRandomnessResult(
if rand.Position.Round == 0 {
return nil
}
- if !con.ccModule.touchBlockRandomnessResult(rand) {
- return nil
- }
- if err := con.ccModule.processBlockRandomnessResult(rand); err != nil {
- if err == ErrBlockNotRegistered {
- err = nil
- } else {
- return err
- }
+ if err := con.bcModule.addRandomness(rand); err != nil {
+ return err
}
if needBroadcast {
con.logger.Debug("Calling Network.BroadcastRandomnessResult",
@@ -1154,7 +1098,7 @@ func (con *Consensus) pullRandomness() {
case <-con.resetRandomnessTicker:
case <-time.After(1500 * time.Millisecond):
// TODO(jimmy): pulling period should be related to lambdaBA.
- hashes := con.ccModule.pendingBlocksWithoutRandomness()
+ hashes := con.bcModule.pendingBlocksWithoutRandomness()
if len(hashes) > 0 {
con.logger.Debug(
"Calling Network.PullRandomness", "blocks", hashes)
@@ -1196,7 +1140,8 @@ func (con *Consensus) deliverBlock(b *types.Block) {
case con.resetDeliveryGuardTicker <- struct{}{}:
default:
}
- if err := con.db.UpdateBlock(*b); err != nil {
+ // TODO(mission): do we need to put block when confirmed now?
+ if err := con.db.PutBlock(*b); err != nil {
panic(err)
}
if err := con.db.PutCompactionChainTipInfo(
@@ -1209,13 +1154,13 @@ func (con *Consensus) deliverBlock(b *types.Block) {
if b.Position.Round == con.roundForNewConfig {
// Get configuration for the round next to next round. Configuration
// for that round should be ready at this moment and is required for
- // lattice module. This logic is related to:
+ // blockChain module. This logic is related to:
// - roundShift
// - notifyGenesisRound
futureRound := con.roundForNewConfig + 1
futureConfig := utils.GetConfigWithPanic(con.gov, futureRound, con.logger)
con.logger.Debug("Append Config", "round", futureRound)
- if err := con.lattice.AppendConfig(
+ if err := con.bcModule.appendConfig(
futureRound, futureConfig); err != nil {
con.logger.Debug("Unable to append config",
"round", futureRound,
@@ -1238,14 +1183,14 @@ func (con *Consensus) deliverFinalizedBlocks() error {
}
func (con *Consensus) deliverFinalizedBlocksWithoutLock() (err error) {
- deliveredBlocks := con.ccModule.extractBlocks()
+ deliveredBlocks := con.bcModule.extractBlocks()
con.logger.Debug("Last blocks in compaction chain",
- "delivered", con.ccModule.lastDeliveredBlock(),
- "pending", con.ccModule.lastPendingBlock())
+ "delivered", con.bcModule.lastDeliveredBlock(),
+ "pending", con.bcModule.lastPendingBlock())
for _, b := range deliveredBlocks {
con.deliverBlock(b)
+ go con.event.NotifyTime(b.Finalization.Timestamp)
}
- err = con.lattice.PurgeBlocks(deliveredBlocks)
return
}
@@ -1271,34 +1216,14 @@ func (con *Consensus) processBlockLoop() {
// processBlock is the entry point to submit one block to a Consensus instance.
func (con *Consensus) processBlock(block *types.Block) (err error) {
+ // Block processed by blockChain can be out-of-order. But the output from
+ // blockChain (deliveredBlocks) cannot, thus we need to protect the part
+ // below with writer lock.
con.lock.Lock()
defer con.lock.Unlock()
- // Block processed by lattice can be out-of-order. But the output of lattice
- // (deliveredBlocks) cannot.
- deliveredBlocks, err := con.lattice.ProcessBlock(block)
- if err != nil {
+ if err = con.bcModule.addBlock(block); err != nil {
return
}
- // Pass delivered blocks to compaction chain.
- for _, b := range deliveredBlocks {
- if b.IsFinalized() {
- if con.nonFinalizedBlockDelivered {
- panic(fmt.Errorf("attempting to skip finalized block: %s", b))
- }
- con.logger.Debug("skip delivery of finalized block",
- "block", b,
- "finalization-height", b.Finalization.Height)
- continue
- } else {
- // Mark that some non-finalized block delivered. After this flag
- // turned on, it's not allowed to deliver finalized blocks anymore.
- con.nonFinalizedBlockDelivered = true
- }
- if err = con.ccModule.processBlock(b); err != nil {
- return
- }
- go con.event.NotifyTime(b.Finalization.Timestamp)
- }
if err = con.deliverFinalizedBlocksWithoutLock(); err != nil {
return
}
@@ -1307,36 +1232,28 @@ func (con *Consensus) processBlock(block *types.Block) (err error) {
// processFinalizedBlock is the entry point for handling finalized blocks.
func (con *Consensus) processFinalizedBlock(block *types.Block) error {
- return con.ccModule.processFinalizedBlock(block)
+ return con.bcModule.processFinalizedBlock(block)
}
// PrepareBlock would setup header fields of block based on its ProposerID.
-func (con *Consensus) prepareBlock(b *types.Block,
- proposeTime time.Time) (err error) {
- if err = con.lattice.PrepareBlock(b, proposeTime); err != nil {
- return
+func (con *Consensus) proposeBlock(position types.Position) (
+ *types.Block, error) {
+ b, err := con.bcModule.proposeBlock(position, time.Now().UTC())
+ if err != nil {
+ return nil, err
}
con.logger.Debug("Calling Governance.CRS", "round", b.Position.Round)
crs := con.gov.CRS(b.Position.Round)
if crs.Equal(common.Hash{}) {
con.logger.Error("CRS for round is not ready, unable to prepare block",
"position", &b.Position)
- err = ErrCRSNotReady
- return
+ return nil, ErrCRSNotReady
}
- err = con.signer.SignCRS(b, crs)
- return
-}
-
-// PrepareGenesisBlock would setup header fields for genesis block.
-func (con *Consensus) PrepareGenesisBlock(b *types.Block,
- proposeTime time.Time) (err error) {
- if err = con.prepareBlock(b, proposeTime); err != nil {
- return
+ if err = con.signer.SignCRS(b, crs); err != nil {
+ return nil, err
}
- if len(b.Payload) != 0 {
- err = ErrGenesisBlockNotEmpty
- return
+ if b.IsGenesis() && len(b.Payload) != 0 {
+ return nil, ErrGenesisBlockNotEmpty
}
- return
+ return b, nil
}
diff --git a/core/consensus_test.go b/core/consensus_test.go
index ddaf731..67070d2 100644
--- a/core/consensus_test.go
+++ b/core/consensus_test.go
@@ -19,7 +19,6 @@ package core
import (
"encoding/json"
- "sort"
"testing"
"time"
@@ -186,20 +185,6 @@ func (s *ConsensusTestSuite) newNetworkConnection() *networkConnection {
}
}
-func (s *ConsensusTestSuite) prepareGenesisBlock(
- chainID uint32,
- con *Consensus) *types.Block {
-
- block := &types.Block{
- Position: types.Position{
- ChainID: chainID,
- },
- }
- err := con.PrepareGenesisBlock(block, time.Now().UTC())
- s.Require().NoError(err)
- return block
-}
-
func (s *ConsensusTestSuite) prepareConsensus(
dMoment time.Time,
gov *test.Governance,
@@ -214,7 +199,6 @@ func (s *ConsensusTestSuite) prepareConsensus(
network := conn.newNetwork(nID)
con := NewConsensus(
dMoment, app, gov, dbInst, network, prvKey, &common.NullLogger{})
- con.ccModule.init(&types.Block{})
conn.setCon(nID, con)
return app, con
}
@@ -229,310 +213,6 @@ func (s *ConsensusTestSuite) prepareAgreementMgrWithoutRunning(
}, common.NewRandomHash())
}
-func (s *ConsensusTestSuite) TestSimpleDeliverBlock() {
- // This test scenario:
- // o o o o <- this layer makes older blocks strongly acked.
- // |x|x|x| <- lots of acks.
- // o | o o <- this layer would be sent to total ordering.
- // |\|/|-|
- // | o | | <- the only block which is acked by all other blocks
- // |/|\|\| at the same height.
- // o o o o <- genesis blocks
- // 0 1 2 3 <- index of node ID
- //
- // - This test case only works for Total Ordering with K=0.
- // - Byzantine Agreement layer is not taken into consideration, every
- // block is passed to lattice module directly.
- var (
- req = s.Require()
- nodes []types.NodeID
- conn = s.newNetworkConnection()
- )
- prvKeys, pubKeys, err := test.NewKeys(4)
- s.Require().NoError(err)
- gov, err := test.NewGovernance(test.NewState(
- pubKeys, time.Second, &common.NullLogger{}, true), ConfigRoundShift)
- s.Require().NoError(err)
- minInterval := gov.Configuration(0).MinBlockInterval
- // Setup core.Consensus and test.App.
- objs := map[types.NodeID]*struct {
- app *test.App
- con *Consensus
- }{}
- dMoment := time.Now().UTC()
- for _, key := range prvKeys {
- nID := types.NewNodeID(key.PublicKey())
- app, con := s.prepareConsensus(dMoment, gov, key, conn)
- objs[nID] = &struct {
- app *test.App
- con *Consensus
- }{app, con}
- nodes = append(nodes, nID)
- }
- // It's a helper function to emit one block
- // to all core.Consensus objects.
- broadcast := func(b *types.Block) {
- h := common.NewRandomHash()
- b.Finalization.Randomness = h[:]
- for _, obj := range objs {
- copied := b.Clone()
- obj.con.ccModule.registerBlock(copied)
- req.Nil(obj.con.processBlock(copied))
- }
- }
- // Genesis blocks
- b00 := s.prepareGenesisBlock(0, objs[nodes[0]].con)
- b10 := s.prepareGenesisBlock(1, objs[nodes[1]].con)
- b20 := s.prepareGenesisBlock(2, objs[nodes[2]].con)
- b30 := s.prepareGenesisBlock(3, objs[nodes[3]].con)
- broadcast(b00)
- broadcast(b10)
- broadcast(b20)
- broadcast(b30)
- // Setup b11.
- b11 := &types.Block{
- Position: types.Position{
- ChainID: 1,
- },
- }
- req.NoError(
- objs[nodes[1]].con.prepareBlock(b11, b10.Timestamp.Add(minInterval)))
- req.Len(b11.Acks, 4)
- req.Contains(b11.Acks, b00.Hash)
- req.Contains(b11.Acks, b10.Hash)
- req.Contains(b11.Acks, b20.Hash)
- req.Contains(b11.Acks, b30.Hash)
- broadcast(b11)
- // Setup b01.
- b01 := &types.Block{
- Position: types.Position{
- ChainID: 0,
- },
- }
- req.NoError(
- objs[nodes[0]].con.prepareBlock(b01, b00.Timestamp.Add(minInterval)))
- req.Len(b01.Acks, 4)
- req.Contains(b01.Acks, b00.Hash)
- req.Contains(b01.Acks, b11.Hash)
- req.Contains(b01.Acks, b20.Hash)
- req.Contains(b01.Acks, b30.Hash)
- // Setup b21.
- b21 := &types.Block{
- Position: types.Position{
- ChainID: 2,
- },
- }
- req.NoError(
- objs[nodes[2]].con.prepareBlock(b21, b20.Timestamp.Add(minInterval)))
- req.Len(b21.Acks, 4)
- req.Contains(b21.Acks, b00.Hash)
- req.Contains(b21.Acks, b11.Hash)
- req.Contains(b21.Acks, b20.Hash)
- req.Contains(b21.Acks, b30.Hash)
- // Setup b31.
- b31 := &types.Block{
- Position: types.Position{
- ChainID: 3,
- },
- }
- req.NoError(
- objs[nodes[3]].con.prepareBlock(b31, b30.Timestamp.Add(minInterval)))
- req.Len(b31.Acks, 4)
- req.Contains(b31.Acks, b00.Hash)
- req.Contains(b31.Acks, b11.Hash)
- req.Contains(b31.Acks, b20.Hash)
- req.Contains(b31.Acks, b30.Hash)
- // Broadcast other height=1 blocks.
- broadcast(b01)
- broadcast(b21)
- broadcast(b31)
- // Setup height=2 blocks.
- // Setup b02.
- b02 := &types.Block{
- Position: types.Position{
- ChainID: 0,
- },
- }
- req.NoError(
- objs[nodes[0]].con.prepareBlock(b02, b01.Timestamp.Add(minInterval)))
- req.Len(b02.Acks, 3)
- req.Contains(b02.Acks, b01.Hash)
- req.Contains(b02.Acks, b21.Hash)
- req.Contains(b02.Acks, b31.Hash)
- // Setup b12.
- b12 := &types.Block{
- Position: types.Position{
- ChainID: 1,
- },
- }
- req.NoError(
- objs[nodes[1]].con.prepareBlock(b12, b11.Timestamp.Add(minInterval)))
- req.Len(b12.Acks, 4)
- req.Contains(b12.Acks, b01.Hash)
- req.Contains(b12.Acks, b11.Hash)
- req.Contains(b12.Acks, b21.Hash)
- req.Contains(b12.Acks, b31.Hash)
- // Setup b22.
- b22 := &types.Block{
- Position: types.Position{
- ChainID: 2,
- },
- }
- req.NoError(
- objs[nodes[2]].con.prepareBlock(b22, b21.Timestamp.Add(minInterval)))
- req.Len(b22.Acks, 3)
- req.Contains(b22.Acks, b01.Hash)
- req.Contains(b22.Acks, b21.Hash)
- req.Contains(b22.Acks, b31.Hash)
- // Setup b32.
- b32 := &types.Block{
- Position: types.Position{
- ChainID: 3,
- },
- }
- req.NoError(
- objs[nodes[3]].con.prepareBlock(b32, b31.Timestamp.Add(minInterval)))
- req.Len(b32.Acks, 3)
- req.Contains(b32.Acks, b01.Hash)
- req.Contains(b32.Acks, b21.Hash)
- req.Contains(b32.Acks, b31.Hash)
- // Broadcast blocks at height=2.
- broadcast(b02)
- broadcast(b12)
- broadcast(b22)
- broadcast(b32)
-
- // Verify the cached status of each app.
- verify := func(app *test.App) {
- req.Contains(app.Confirmed, b00.Hash)
- req.Contains(app.Confirmed, b10.Hash)
- req.Contains(app.Confirmed, b20.Hash)
- req.Contains(app.Confirmed, b30.Hash)
- req.Contains(app.Confirmed, b01.Hash)
- req.Contains(app.Confirmed, b11.Hash)
- req.Contains(app.Confirmed, b21.Hash)
- req.Contains(app.Confirmed, b31.Hash)
- // Genesis blocks are delivered by total ordering as a set.
- delivered0 := common.Hashes{b00.Hash, b10.Hash, b20.Hash, b30.Hash}
- sort.Sort(delivered0)
- req.Len(app.TotalOrdered, 4)
- req.Equal(app.TotalOrdered[0].BlockHashes, delivered0)
- req.Equal(app.TotalOrdered[0].Mode, TotalOrderingModeNormal)
- // b11 is the sencond set delivered by total ordering.
- delivered1 := common.Hashes{b11.Hash}
- sort.Sort(delivered1)
- req.Equal(app.TotalOrdered[1].BlockHashes, delivered1)
- req.Equal(app.TotalOrdered[1].Mode, TotalOrderingModeNormal)
- // b01, b21, b31 are the third set delivered by total ordering.
- delivered2 := common.Hashes{b01.Hash, b21.Hash, b31.Hash}
- sort.Sort(delivered2)
- req.Equal(app.TotalOrdered[2].BlockHashes, delivered2)
- req.Equal(app.TotalOrdered[2].Mode, TotalOrderingModeNormal)
- // b02, b12, b22, b32 are the fourth set delivered by total ordering.
- delivered3 := common.Hashes{b02.Hash, b12.Hash, b22.Hash, b32.Hash}
- sort.Sort(delivered3)
- req.Equal(app.TotalOrdered[3].BlockHashes, delivered3)
- req.Equal(app.TotalOrdered[3].Mode, TotalOrderingModeNormal)
- // Check generated timestamps.
- req.Contains(app.Delivered, b00.Hash)
- req.Contains(app.Delivered, b10.Hash)
- req.Contains(app.Delivered, b20.Hash)
- req.Contains(app.Delivered, b30.Hash)
- req.Contains(app.Delivered, b11.Hash)
- // 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.
- 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.NoError(err)
- req.Equal(t, app.Delivered[b11.Hash].Result.Timestamp)
- }
- for _, obj := range objs {
- if nb, ok := obj.con.app.(*nonBlocking); ok {
- nb.wait()
- }
- verify(obj.app)
- req.NoError(test.VerifyDB(obj.con.db))
- }
-}
-
-func (s *ConsensusTestSuite) TestPrepareBlock() {
- // This test case would test these steps:
- // - Add all genesis blocks into lattice.
- // - Make sure Consensus.prepareBlock would attempt to ack
- // all genesis blocks.
- // - Add the prepared block into lattice.
- // - Make sure Consensus.prepareBlock would only attempt to
- // ack the prepared block.
- var (
- req = s.Require()
- nodes []types.NodeID
- conn = s.newNetworkConnection()
- )
- prvKeys, pubKeys, err := test.NewKeys(4)
- s.Require().NoError(err)
- gov, err := test.NewGovernance(test.NewState(
- pubKeys, time.Second, &common.NullLogger{}, true), ConfigRoundShift)
- s.Require().NoError(err)
- dMoment := time.Now().UTC()
- // Setup core.Consensus and test.App.
- cons := map[types.NodeID]*Consensus{}
- for _, key := range prvKeys {
- _, con := s.prepareConsensus(dMoment, gov, key, conn)
- s.prepareAgreementMgrWithoutRunning(con, 4)
- nID := types.NewNodeID(key.PublicKey())
- cons[nID] = con
- nodes = append(nodes, nID)
- }
- b00 := s.prepareGenesisBlock(0, cons[nodes[0]])
- b10 := s.prepareGenesisBlock(1, cons[nodes[1]])
- b20 := s.prepareGenesisBlock(2, cons[nodes[2]])
- b30 := s.prepareGenesisBlock(3, cons[nodes[3]])
- for _, con := range cons {
- req.Nil(con.processBlock(b00))
- req.Nil(con.processBlock(b10))
- req.Nil(con.processBlock(b20))
- req.Nil(con.processBlock(b30))
- }
- b11 := &types.Block{
- Position: types.Position{ChainID: b10.Position.ChainID},
- }
- interval := gov.Configuration(0).MinBlockInterval
- req.Nil(cons[nodes[1]].prepareBlock(b11, b10.Timestamp.Add(interval)))
- for _, con := range cons {
- req.Nil(con.preProcessBlock(b11))
- req.Nil(con.processBlock(b11))
- }
- b12 := &types.Block{
- Position: types.Position{ChainID: b11.Position.ChainID},
- }
- req.Nil(cons[nodes[1]].prepareBlock(b12, b11.Timestamp.Add(interval)))
- req.Len(b12.Acks, 1)
- req.Contains(b12.Acks, b11.Hash)
-}
-
-func (s *ConsensusTestSuite) TestPrepareGenesisBlock() {
- conn := s.newNetworkConnection()
- prvKeys, pubKeys, err := test.NewKeys(4)
- s.Require().NoError(err)
- gov, err := test.NewGovernance(test.NewState(
- pubKeys, time.Second, &common.NullLogger{}, true), ConfigRoundShift)
- s.Require().NoError(err)
- prvKey := prvKeys[0]
- _, con := s.prepareConsensus(time.Now().UTC(), gov, prvKey, conn)
- s.prepareAgreementMgrWithoutRunning(con, 4)
- block := &types.Block{
- Position: types.Position{ChainID: 0},
- }
- s.Require().NoError(con.PrepareGenesisBlock(block, time.Now().UTC()))
- s.True(block.IsGenesis())
- s.NoError(con.preProcessBlock(block))
-}
-
func (s *ConsensusTestSuite) TestDKGCRS() {
n := 21
lambda := 200 * time.Millisecond
@@ -620,7 +300,7 @@ func (s *ConsensusTestSuite) TestSyncBA() {
// - the ticker is 1 lambdaa.
time.Sleep(5 * lambdaBA)
s.Require().NoError(con.ProcessAgreementResult(baResult))
- aID := con.baMgr.baModules[0].agreementID()
+ aID := con.baMgr.baModule.agreementID()
s.Equal(pos, aID)
// Negative cases are moved to TestVerifyAgreementResult in utils_test.go.
diff --git a/core/db/level-db_test.go b/core/db/level-db_test.go
index df971ee..40f9a60 100644
--- a/core/db/level-db_test.go
+++ b/core/db/level-db_test.go
@@ -148,10 +148,10 @@ func (s *LevelDBTestSuite) TestCompactionChainTipInfo() {
s.Require().Equal(height, uint64(1))
// Unable to put compaction chain tip info with lower height.
err = dbInst.PutCompactionChainTipInfo(hash, 0)
- s.Require().IsType(err, ErrInvalidCompactionChainTipHeight)
+ s.Require().Equal(err.Error(), ErrInvalidCompactionChainTipHeight.Error())
// Unable to put compaction chain tip info with height not incremental by 1.
err = dbInst.PutCompactionChainTipInfo(hash, 3)
- s.Require().IsType(err, ErrInvalidCompactionChainTipHeight)
+ s.Require().Equal(err.Error(), ErrInvalidCompactionChainTipHeight.Error())
// It's OK to put compaction chain tip info with height incremental by 1.
s.Require().NoError(dbInst.PutCompactionChainTipInfo(hash, 2))
}
@@ -173,12 +173,12 @@ func (s *LevelDBTestSuite) TestDKGPrivateKey() {
s.Require().False(exists)
// We should be unable to get it, too.
_, err = dbInst.GetDKGPrivateKey(1)
- s.Require().IsType(err, ErrDKGPrivateKeyDoesNotExist)
+ s.Require().Equal(err.Error(), ErrDKGPrivateKeyDoesNotExist.Error())
// Put it.
s.Require().NoError(dbInst.PutDKGPrivateKey(1, *p))
// Put it again, should not success.
err = dbInst.PutDKGPrivateKey(1, *p)
- s.Require().IsType(err, ErrDKGPrivateKeyExists)
+ s.Require().Equal(err.Error(), ErrDKGPrivateKeyExists.Error())
// Get it back.
tmpPrv, err := dbInst.GetDKGPrivateKey(1)
s.Require().NoError(err)
diff --git a/core/db/memory_test.go b/core/db/memory_test.go
index a1b5165..b210c8d 100644
--- a/core/db/memory_test.go
+++ b/core/db/memory_test.go
@@ -144,10 +144,10 @@ func (s *MemBackedDBTestSuite) TestCompactionChainTipInfo() {
s.Require().Equal(height, uint64(1))
// Unable to put compaction chain tip info with lower height.
err = dbInst.PutCompactionChainTipInfo(hash, 0)
- s.Require().IsType(err, ErrInvalidCompactionChainTipHeight)
+ s.Require().Equal(err.Error(), ErrInvalidCompactionChainTipHeight.Error())
// Unable to put compaction chain tip info with height not incremental by 1.
err = dbInst.PutCompactionChainTipInfo(hash, 3)
- s.Require().IsType(err, ErrInvalidCompactionChainTipHeight)
+ s.Require().Equal(err.Error(), ErrInvalidCompactionChainTipHeight.Error())
// It's OK to put compaction chain tip info with height incremental by 1.
s.Require().NoError(dbInst.PutCompactionChainTipInfo(hash, 2))
}
@@ -163,12 +163,12 @@ func (s *MemBackedDBTestSuite) TestDKGPrivateKey() {
s.Require().False(exists)
// We should be unable to get it, too.
_, err = dbInst.GetDKGPrivateKey(1)
- s.Require().IsType(err, ErrDKGPrivateKeyDoesNotExist)
+ s.Require().Equal(err.Error(), ErrDKGPrivateKeyDoesNotExist.Error())
// Put it.
s.Require().NoError(dbInst.PutDKGPrivateKey(1, *p))
// Put it again, should not success.
err = dbInst.PutDKGPrivateKey(1, *p)
- s.Require().IsType(err, ErrDKGPrivateKeyExists)
+ s.Require().Equal(err.Error(), ErrDKGPrivateKeyExists.Error())
// Get it back.
tmpPrv, err := dbInst.GetDKGPrivateKey(1)
s.Require().NoError(err)
diff --git a/core/dkg-tsig-protocol.go b/core/dkg-tsig-protocol.go
index 2028154..4f15a74 100644
--- a/core/dkg-tsig-protocol.go
+++ b/core/dkg-tsig-protocol.go
@@ -53,6 +53,8 @@ var (
"not enough of partial signatures")
ErrRoundAlreadyPurged = fmt.Errorf(
"cache of round already been purged")
+ ErrTSigNotReady = fmt.Errorf(
+ "tsig not ready")
)
type dkgReceiver interface {
diff --git a/core/interfaces.go b/core/interfaces.go
index 408343f..3879e36 100644
--- a/core/interfaces.go
+++ b/core/interfaces.go
@@ -51,9 +51,6 @@ type Application interface {
type Debug interface {
// BlockReceived is called when the block received in agreement.
BlockReceived(common.Hash)
- // TotalOrderingDelivered is called when the total ordering algorithm deliver
- // a set of block.
- TotalOrderingDelivered(common.Hashes, uint32)
// BlockReady is called when the block's randomness is ready.
BlockReady(common.Hash)
}
diff --git a/core/lattice-data.go b/core/lattice-data.go
deleted file mode 100644
index 0bbe890..0000000
--- a/core/lattice-data.go
+++ /dev/null
@@ -1,683 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package core
-
-import (
- "errors"
- "fmt"
- "sort"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core/db"
- "github.com/dexon-foundation/dexon-consensus/core/types"
- "github.com/dexon-foundation/dexon-consensus/core/utils"
-)
-
-// Errors for sanity check error.
-var (
- ErrDuplicatedAckOnOneChain = fmt.Errorf("duplicated ack on one chain")
- ErrInvalidProposerID = fmt.Errorf("invalid proposer id")
- ErrInvalidWitness = fmt.Errorf("invalid witness data")
- ErrInvalidBlock = fmt.Errorf("invalid block")
- ErrNotAckParent = fmt.Errorf("not ack parent")
- ErrDoubleAck = fmt.Errorf("double ack")
- ErrAcksNotSorted = fmt.Errorf("acks not sorted")
- ErrInvalidBlockHeight = fmt.Errorf("invalid block height")
- ErrAlreadyInLattice = fmt.Errorf("block already in lattice")
- ErrIncorrectBlockTime = fmt.Errorf("block timestamp is incorrect")
- ErrInvalidRoundID = fmt.Errorf("invalid round id")
- ErrUnknownRoundID = fmt.Errorf("unknown round id")
- ErrRoundOutOfRange = fmt.Errorf("round out of range")
- ErrRoundNotSwitch = fmt.Errorf("round not switch")
- ErrNotGenesisBlock = fmt.Errorf("not a genesis block")
- ErrUnexpectedGenesisBlock = fmt.Errorf("unexpected genesis block")
-)
-
-// ErrAckingBlockNotExists is for sanity check error.
-type ErrAckingBlockNotExists struct {
- hash common.Hash
-}
-
-func (e ErrAckingBlockNotExists) Error() string {
- return fmt.Sprintf("acking block %s not exists", e.hash.String()[:6])
-}
-
-// Errors for method usage
-var (
- ErrRoundNotIncreasing = errors.New("round not increasing")
- ErrPurgedBlockNotFound = errors.New("purged block not found")
- ErrPurgeNotDeliveredBlock = errors.New("not purge from head")
-)
-
-// latticeDataConfig is the configuration for latticeData for each round.
-type latticeDataConfig struct {
- roundBasedConfig
- // Number of chains between runs
- numChains uint32
- // Block interval specifies reasonable time difference between
- // parent/child blocks.
- minBlockTimeInterval time.Duration
-}
-
-// Initiate latticeDataConfig from types.Config.
-func (config *latticeDataConfig) fromConfig(roundID uint64, cfg *types.Config) {
- config.numChains = cfg.NumChains
- config.minBlockTimeInterval = cfg.MinBlockInterval
- config.setupRoundBasedFields(roundID, cfg)
-}
-
-// isValidBlockTime checks if timestamp of a block is valid according to a
-// reference time.
-func (config *latticeDataConfig) isValidBlockTime(
- b *types.Block, ref time.Time) bool {
- return !b.Timestamp.Before(ref.Add(config.minBlockTimeInterval))
-}
-
-// isValidGenesisBlockTime checks if a timestamp is valid for a genesis block.
-func (config *latticeDataConfig) isValidGenesisBlockTime(b *types.Block) bool {
- return !b.Timestamp.Before(config.roundBeginTime)
-}
-
-// newLatticeDataConfig constructs a latticeDataConfig instance.
-func newLatticeDataConfig(
- prev *latticeDataConfig, cur *types.Config) *latticeDataConfig {
- c := &latticeDataConfig{}
- c.fromConfig(prev.roundID+1, cur)
- c.setRoundBeginTime(prev.roundEndTime)
- return c
-}
-
-// latticeData is a module for storing lattice.
-type latticeData struct {
- // DB for getting blocks purged in memory.
- db db.Database
- // chains stores chains' blocks and other info.
- chains []*chainStatus
- // blockByHash stores blocks, indexed by block hash.
- blockByHash map[common.Hash]*types.Block
- // This stores configuration for each round.
- configs []*latticeDataConfig
- // shallowBlocks stores the hash of blocks that their body is not receive yet.
- shallowBlocks map[common.Hash]types.Position
-}
-
-// newLatticeData creates a new latticeData instance.
-func newLatticeData(
- db db.Database,
- dMoment time.Time,
- round uint64,
- config *types.Config) (data *latticeData) {
-
- genesisConfig := &latticeDataConfig{}
- genesisConfig.fromConfig(round, config)
- genesisConfig.setRoundBeginTime(dMoment)
- data = &latticeData{
- db: db,
- chains: make([]*chainStatus, genesisConfig.numChains),
- blockByHash: make(map[common.Hash]*types.Block),
- configs: []*latticeDataConfig{genesisConfig},
- shallowBlocks: make(map[common.Hash]types.Position),
- }
- for i := range data.chains {
- data.chains[i] = &chainStatus{
- ID: uint32(i),
- blocks: []*types.Block{},
- lastAckPos: make([]*types.Position, genesisConfig.numChains),
- }
- }
- return
-}
-
-func (data *latticeData) addShallowBlock(hash common.Hash, pos types.Position) {
- // We don't care other errors here. This `if` is to prevent being spammed by
- // very old blocks.
- if _, err := data.findBlock(hash); err != db.ErrBlockDoesNotExist {
- return
- }
- data.shallowBlocks[hash] = pos
-}
-
-func (data *latticeData) checkAckingRelations(
- b *types.Block, allowShallow bool) error {
- acksByChainID := make(map[uint32]struct{}, len(data.chains))
- for _, hash := range b.Acks {
- bAck, err := data.findBlock(hash)
- if err != nil {
- if err == db.ErrBlockDoesNotExist {
- err = &ErrAckingBlockNotExists{hash}
- if allowShallow {
- if pos, exist := data.shallowBlocks[hash]; exist {
- bAck = &types.Block{
- Position: pos,
- }
- err = nil
- }
- }
- }
- if err != nil {
- return err
- }
- }
- // Check if it acks blocks from old rounds, the allowed round difference
- // is 1.
- if DiffUint64(bAck.Position.Round, b.Position.Round) > 1 {
- return ErrRoundOutOfRange
- }
- // Check if it acks older blocks than blocks on the same chain.
- lastAckPos :=
- data.chains[bAck.Position.ChainID].lastAckPos[b.Position.ChainID]
- if lastAckPos != nil && !bAck.Position.Newer(lastAckPos) {
- return ErrDoubleAck
- }
- // Check if it acks two blocks on the same chain. This would need
- // to check after we replace map with slice for acks.
- if _, acked := acksByChainID[bAck.Position.ChainID]; acked {
- return ErrDuplicatedAckOnOneChain
- }
- acksByChainID[bAck.Position.ChainID] = struct{}{}
- }
- return nil
-}
-
-func (data *latticeData) sanityCheck(b *types.Block, allowShallow bool) error {
- // TODO(mission): Check if its proposer is in validator set, lattice has no
- // knowledge about node set.
- config := data.getConfig(b.Position.Round)
- if config == nil {
- return ErrInvalidRoundID
- }
- // Check if the chain id is valid.
- if b.Position.ChainID >= config.numChains {
- return utils.ErrInvalidChainID
- }
- // Make sure parent block is arrived.
- chain := data.chains[b.Position.ChainID]
- chainTip := chain.tip
- if chainTip == nil {
- if !b.ParentHash.Equal(common.Hash{}) {
- return &ErrAckingBlockNotExists{b.ParentHash}
- }
- if !b.IsGenesis() {
- return ErrNotGenesisBlock
- }
- if !config.isValidGenesisBlockTime(b) {
- return ErrIncorrectBlockTime
- }
- return data.checkAckingRelations(b, allowShallow)
- }
- // Check parent block if parent hash is specified.
- if !b.ParentHash.Equal(common.Hash{}) {
- if !b.ParentHash.Equal(chainTip.Hash) {
- return &ErrAckingBlockNotExists{b.ParentHash}
- }
- if !b.IsAcking(b.ParentHash) {
- return ErrNotAckParent
- }
- }
- chainTipConfig := data.getConfig(chainTip.Position.Round)
- // Round can't be rewinded.
- if chainTip.Position.Round > b.Position.Round {
- return ErrInvalidRoundID
- }
- checkTip := false
- if chainTip.Timestamp.After(chainTipConfig.roundEndTime) {
- // Round switching should happen when chainTip already pass
- // round end time of its round.
- if chainTip.Position.Round == b.Position.Round {
- return ErrRoundNotSwitch
- }
- // The round ID is continuous.
- if b.Position.Round-chainTip.Position.Round == 1 {
- checkTip = true
- } else {
- // This block should be genesis block of new round because round
- // ID is not continuous.
- if !b.IsGenesis() {
- return ErrNotGenesisBlock
- }
- if !config.isValidGenesisBlockTime(b) {
- return ErrIncorrectBlockTime
- }
- // TODO(mission): make sure rounds between chainTip and current block
- // don't expect blocks from this chain.
- }
- } else {
- if chainTip.Position.Round != b.Position.Round {
- // Round should not switch.
- return ErrInvalidRoundID
- }
- checkTip = true
- }
- // Validate the relation between chain tip when needed.
- if checkTip {
- if b.Position.Height != chainTip.Position.Height+1 {
- return ErrInvalidBlockHeight
- }
- if b.Witness.Height < chainTip.Witness.Height {
- return ErrInvalidWitness
- }
- if !config.isValidBlockTime(b, chainTip.Timestamp) {
- return ErrIncorrectBlockTime
- }
- // Chain tip should be acked.
- if !b.IsAcking(chainTip.Hash) {
- return ErrNotAckParent
- }
- }
- return data.checkAckingRelations(b, allowShallow)
-}
-
-// addBlock processes blocks. It does sanity check, inserts block into lattice
-// and deletes blocks which will not be used.
-func (data *latticeData) addBlock(
- block *types.Block) (deliverable []*types.Block, err error) {
- var (
- bAck *types.Block
- updated bool
- )
- if err = data.db.PutBlock(*block); err != nil {
- if err == db.ErrBlockExists {
- // If a node is crashed and restarted, we might encounter some
- // blocks that already confirmed but not delivered yet. Then
- // syncer might still try to add that block in this way.
- err = nil
- } else {
- return
- }
- }
- data.chains[block.Position.ChainID].addBlock(block)
- data.blockByHash[block.Hash] = block
- // Update lastAckPos.
- for _, ack := range block.Acks {
- if bAck, err = data.findBlock(ack); err != nil {
- return
- }
- data.chains[bAck.Position.ChainID].lastAckPos[block.Position.ChainID] =
- bAck.Position.Clone()
- }
- // Extract deliverable blocks to total ordering. A block is deliverable to
- // total ordering iff all its ackings blocks were delivered to total ordering.
- for {
- updated = false
- for _, status := range data.chains {
- if status.nextOutputIndex >= len(status.blocks) {
- continue
- }
- tip := status.blocks[status.nextOutputIndex]
- allAckingBlockDelivered := true
- for _, ack := range tip.Acks {
- if bAck, err = data.findBlock(ack); err != nil {
- if err == db.ErrBlockDoesNotExist {
- err = nil
- allAckingBlockDelivered = false
- break
- }
- return
- }
- // Check if this block is outputed or not.
- idx := data.chains[bAck.Position.ChainID].findBlock(&bAck.Position)
- var ok bool
- if idx == -1 {
- // Either the block is delivered or not added to chain yet.
- if out :=
- data.chains[bAck.Position.ChainID].lastOutputPosition; out != nil {
- ok = !out.Older(&bAck.Position)
- } else if ackTip :=
- data.chains[bAck.Position.ChainID].tip; ackTip != nil {
- ok = !ackTip.Position.Older(&bAck.Position)
- }
- } else {
- ok = idx < data.chains[bAck.Position.ChainID].nextOutputIndex
- }
- if ok {
- continue
- }
- // This acked block exists and not delivered yet.
- allAckingBlockDelivered = false
- }
- if allAckingBlockDelivered {
- status.lastOutputPosition = &tip.Position
- status.nextOutputIndex++
- deliverable = append(deliverable, tip)
- updated = true
- }
- }
- if !updated {
- break
- }
- }
- return
-}
-
-// addFinalizedBlock processes block for syncing internal data.
-func (data *latticeData) addFinalizedBlock(block *types.Block) (err error) {
- var bAck *types.Block
- chain := data.chains[block.Position.ChainID]
- if chain.tip != nil && chain.tip.Position.Height >= block.Position.Height {
- return
- }
- chain.nextOutputIndex = 0
- chain.blocks = []*types.Block{}
- chain.tip = block
- chain.lastOutputPosition = nil
- // Update lastAckPost.
- for _, ack := range block.Acks {
- if bAck, err = data.findBlock(ack); err != nil {
- return
- }
- data.chains[bAck.Position.ChainID].lastAckPos[block.Position.ChainID] =
- bAck.Position.Clone()
- }
- return
-}
-
-func (data *latticeData) tipRound(chainID uint32) uint64 {
- if tip := data.chains[chainID].tip; tip != nil {
- tipConfig := data.getConfig(tip.Position.Round)
- offset := uint64(0)
- if tip.Timestamp.After(tipConfig.roundEndTime) {
- offset++
- }
- return tip.Position.Round + offset
- }
- return uint64(0)
-
-}
-
-// isBindTip checks if a block's fields should follow up its parent block.
-func (data *latticeData) isBindTip(
- pos types.Position, tip *types.Block) (bindTip bool, err error) {
- if tip == nil {
- return
- }
- if pos.Round < tip.Position.Round {
- err = ErrInvalidRoundID
- return
- }
- tipConfig := data.getConfig(tip.Position.Round)
- if tip.Timestamp.After(tipConfig.roundEndTime) {
- if pos.Round == tip.Position.Round {
- err = ErrRoundNotSwitch
- return
- }
- if pos.Round == tip.Position.Round+1 {
- bindTip = true
- }
- } else {
- if pos.Round != tip.Position.Round {
- err = ErrInvalidRoundID
- return
- }
- bindTip = true
- }
- return
-}
-
-// prepareBlock setups fields of a block based on its ChainID and Round,
-// including:
-// - Acks
-// - Timestamp
-// - ParentHash and Height from parent block. If there is no valid parent block
-// (e.g. Newly added chain or bootstrap), these fields should be setup as
-// genesis block.
-func (data *latticeData) prepareBlock(b *types.Block) error {
- var (
- minTimestamp time.Time
- config *latticeDataConfig
- acks common.Hashes
- bindTip bool
- )
- if config = data.getConfig(b.Position.Round); config == nil {
- return ErrUnknownRoundID
- }
- // If chainID is illegal in this round, reject it.
- if b.Position.ChainID >= config.numChains {
- return utils.ErrInvalidChainID
- }
- // Reset fields to make sure we got these information from parent block.
- b.Position.Height = 0
- b.ParentHash = common.Hash{}
- // Decide valid timestamp range.
- chainTip := data.chains[b.Position.ChainID].tip
- if chainTip != nil {
- // TODO(mission): find a way to prevent us to assign a witness height
- // from Jurassic period.
- b.Witness.Height = chainTip.Witness.Height
- }
- bindTip, err := data.isBindTip(b.Position, chainTip)
- if err != nil {
- return err
- }
- // For blocks with continuous round ID, assign timestamp range based on
- // parent block and bound config.
- if bindTip {
- minTimestamp = chainTip.Timestamp.Add(config.minBlockTimeInterval)
- // When a chain is removed and added back, the reference block
- // of previous round can't be used as parent block.
- b.ParentHash = chainTip.Hash
- b.Position.Height = chainTip.Position.Height + 1
- } else {
- // Discontinuous round ID detected, another fresh start of
- // new round.
- minTimestamp = config.roundBeginTime
- }
- // Fix timestamp if the given one is invalid.
- if b.Timestamp.Before(minTimestamp) {
- b.Timestamp = minTimestamp
- }
- // Setup acks fields.
- for _, status := range data.chains {
- // Check if we can ack latest block on that chain.
- if status.tip == nil {
- continue
- }
- lastAckPos := status.lastAckPos[b.Position.ChainID]
- if lastAckPos != nil && !status.tip.Position.Newer(lastAckPos) {
- // The reference block is already acked.
- continue
- }
- if status.tip.Position.Round > b.Position.Round {
- // Avoid forward acking: acking some block from later rounds.
- continue
- }
- if b.Position.Round > status.tip.Position.Round+1 {
- // Can't ack block too old or too new to us.
- continue
- }
- acks = append(acks, status.tip.Hash)
- }
- b.Acks = common.NewSortedHashes(acks)
- return nil
-}
-
-// prepareEmptyBlock setups fields of a block based on its ChainID.
-// including:
-// - Acks only acking its parent
-// - Timestamp with parent.Timestamp + minBlockProposeInterval
-// - ParentHash and Height from parent block. If there is no valid parent block
-// (ex. Newly added chain or bootstrap), these fields would be setup as
-// genesis block.
-func (data *latticeData) prepareEmptyBlock(b *types.Block) (err error) {
- // emptyBlock has no proposer.
- b.ProposerID = types.NodeID{}
- // Reset fields to make sure we got these information from parent block.
- b.Position.Height = 0
- b.ParentHash = common.Hash{}
- b.Timestamp = time.Time{}
- // Decide valid timestamp range.
- config := data.getConfig(b.Position.Round)
- chainTip := data.chains[b.Position.ChainID].tip
- bindTip, err := data.isBindTip(b.Position, chainTip)
- if err != nil {
- return
- }
- if bindTip {
- b.ParentHash = chainTip.Hash
- b.Position.Height = chainTip.Position.Height + 1
- b.Timestamp = chainTip.Timestamp.Add(config.minBlockTimeInterval)
- b.Witness.Height = chainTip.Witness.Height
- b.Witness.Data = make([]byte, len(chainTip.Witness.Data))
- copy(b.Witness.Data, chainTip.Witness.Data)
- b.Acks = common.NewSortedHashes(common.Hashes{chainTip.Hash})
- } else {
- b.Timestamp = config.roundBeginTime
- }
- return
-}
-
-// TODO(mission): make more abstraction for this method.
-// nextBlock returns the next height and timestamp of a chain.
-func (data *latticeData) nextBlock(
- round uint64, chainID uint32) (uint64, time.Time, error) {
- chainTip := data.chains[chainID].tip
- bindTip, err := data.isBindTip(
- types.Position{Round: round, ChainID: chainID}, chainTip)
- if err != nil {
- return 0, time.Time{}, err
- }
- config := data.getConfig(round)
- if bindTip {
- return chainTip.Position.Height + 1,
- chainTip.Timestamp.Add(config.minBlockTimeInterval), nil
- }
- return 0, config.roundBeginTime, nil
-}
-
-// findBlock seeks blocks in memory or db.
-func (data *latticeData) findBlock(h common.Hash) (b *types.Block, err error) {
- if b = data.blockByHash[h]; b != nil {
- return
- }
- var tmpB types.Block
- if tmpB, err = data.db.GetBlock(h); err != nil {
- return
- }
- b = &tmpB
- return
-}
-
-// purgeBlocks purges blocks from cache.
-func (data *latticeData) purgeBlocks(blocks []*types.Block) error {
- for _, b := range blocks {
- if _, exists := data.blockByHash[b.Hash]; !exists {
- return ErrPurgedBlockNotFound
- }
- delete(data.blockByHash, b.Hash)
- // Blocks are purged in ascending order by position.
- if err := data.chains[b.Position.ChainID].purgeBlock(b); err != nil {
- return err
- }
- }
- return nil
-}
-
-// getConfig get configuration for lattice-data by round ID.
-func (data *latticeData) getConfig(round uint64) (config *latticeDataConfig) {
- r := data.configs[0].roundID
- if round < r || round >= r+uint64(len(data.configs)) {
- return
- }
- return data.configs[round-r]
-}
-
-// appendConfig appends a configuration for upcoming round. Rounds appended
-// should be consecutive.
-func (data *latticeData) appendConfig(
- round uint64, config *types.Config) (err error) {
- // Check if the round of config is increasing by 1.
- if round != uint64(len(data.configs))+data.configs[0].roundID {
- return ErrRoundNotIncreasing
- }
- // Set round beginning time.
- newConfig := newLatticeDataConfig(data.configs[len(data.configs)-1], config)
- data.configs = append(data.configs, newConfig)
- // Resize each slice if incoming config contains larger number of chains.
- if uint32(len(data.chains)) < newConfig.numChains {
- count := newConfig.numChains - uint32(len(data.chains))
- for _, status := range data.chains {
- status.lastAckPos = append(
- status.lastAckPos, make([]*types.Position, count)...)
- }
- for i := uint32(len(data.chains)); i < newConfig.numChains; i++ {
- data.chains = append(data.chains, &chainStatus{
- ID: i,
- blocks: []*types.Block{},
- lastAckPos: make([]*types.Position, newConfig.numChains),
- })
- }
- }
- return nil
-}
-
-type chainStatus struct {
- // ID keeps the chainID of this chain status.
- ID uint32
- // blocks stores blocks proposed for this chain, sorted by height.
- blocks []*types.Block
- // tip is the last block on this chain.
- tip *types.Block
- // lastAckPos caches last acking position from other chains. Nil means
- // not acked yet.
- lastAckPos []*types.Position
- // the index to be output next time.
- nextOutputIndex int
- // the position of output last time.
- lastOutputPosition *types.Position
-}
-
-// findBlock finds index of block in current pending blocks on this chain.
-// Return -1 if not found.
-func (s *chainStatus) findBlock(pos *types.Position) (idx int) {
- idx = sort.Search(len(s.blocks), func(i int) bool {
- return s.blocks[i].Position.Newer(pos) ||
- s.blocks[i].Position.Equal(pos)
- })
- if idx == len(s.blocks) {
- idx = -1
- } else if !s.blocks[idx].Position.Equal(pos) {
- idx = -1
- }
- return idx
-}
-
-// getBlock returns a pending block by giving its index from findBlock method.
-func (s *chainStatus) getBlock(idx int) (b *types.Block) {
- if idx < 0 || idx >= len(s.blocks) {
- return
- }
- b = s.blocks[idx]
- return
-}
-
-// addBlock adds a block to pending blocks on this chain.
-func (s *chainStatus) addBlock(b *types.Block) {
- s.blocks = append(s.blocks, b)
- s.tip = b
-}
-
-// purgeBlock purges a block from cache, make sure this block is already saved
-// in db.
-func (s *chainStatus) purgeBlock(b *types.Block) error {
- if b.Hash != s.blocks[0].Hash || s.nextOutputIndex <= 0 {
- return ErrPurgeNotDeliveredBlock
- }
- s.blocks = s.blocks[1:]
- s.nextOutputIndex--
- return nil
-}
diff --git a/core/lattice-data_test.go b/core/lattice-data_test.go
deleted file mode 100644
index 2e6d684..0000000
--- a/core/lattice-data_test.go
+++ /dev/null
@@ -1,707 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package core
-
-import (
- "math/rand"
- "sort"
- "testing"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core/db"
- "github.com/dexon-foundation/dexon-consensus/core/test"
- "github.com/dexon-foundation/dexon-consensus/core/types"
- "github.com/dexon-foundation/dexon-consensus/core/utils"
- "github.com/stretchr/testify/suite"
-)
-
-type LatticeDataTestSuite struct {
- suite.Suite
-}
-
-// genTestCase1 generates test case 1,
-// 3
-// |
-// 2
-// | \
-// 1 | 1
-// | | |
-// 0 0 0 0 (block height)
-// 0 1 2 3 (validator)
-func (s *LatticeDataTestSuite) genTestCase1() (
- data *latticeData, blocks map[uint32]map[uint64]*types.Block) {
- // Create new latticeData instance with 4 validators
- var (
- delivered []*types.Block
- chainNum uint32 = 4
- req = s.Require()
- now = time.Now().UTC()
- err error
- )
- // Setup stuffs.
- genesisConfig := &types.Config{
- RoundInterval: 500 * time.Second,
- NumChains: chainNum,
- MinBlockInterval: 2 * time.Nanosecond,
- }
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
- data = newLatticeData(dbInst, now, 0, genesisConfig)
- config := &types.Config{
- RoundInterval: 1000 * time.Second,
- NumChains: chainNum,
- MinBlockInterval: 2 * time.Nanosecond,
- }
- data.appendConfig(1, config)
- // Add genesis blocks.
- addBlock := func(b *types.Block) {
- s.hashBlock(b)
- delivered, err = data.addBlock(b)
- req.NoError(err)
- req.Len(delivered, 1)
- req.Equal(delivered[0].Hash, b.Hash)
- }
- // Genesis blocks are safe to be added to DAG, they acks no one.
- b00 := s.prepareGenesisBlock(0)
- addBlock(b00)
- b10 := s.prepareGenesisBlock(1)
- addBlock(b10)
- b20 := s.prepareGenesisBlock(2)
- addBlock(b20)
- b30 := s.prepareGenesisBlock(3)
- addBlock(b30)
- // Add block 0-1 which acks 0-0.
- b01 := &types.Block{
- ParentHash: b00.Hash,
- Hash: common.NewRandomHash(),
- Timestamp: time.Now().UTC(),
- Position: types.Position{
- ChainID: 0,
- Height: 1,
- },
- Acks: common.NewSortedHashes(common.Hashes{b00.Hash}),
- Witness: types.Witness{
- Height: 1,
- },
- }
- addBlock(b01)
- // Add block 0-2 which acks 0-1 and 1-0.
- b02 := &types.Block{
- ParentHash: b01.Hash,
- Position: types.Position{
- ChainID: 0,
- Height: 2,
- },
- Timestamp: time.Now().UTC(),
- Acks: common.NewSortedHashes(common.Hashes{
- b01.Hash,
- b10.Hash,
- }),
- Witness: types.Witness{
- Height: 2,
- },
- }
- addBlock(b02)
- // Add block 0-3 which acks 0-2.
- b03 := &types.Block{
- ParentHash: b02.Hash,
- Hash: common.NewRandomHash(),
- Timestamp: time.Now().UTC(),
- Position: types.Position{
- ChainID: 0,
- Height: 3,
- },
- Acks: common.NewSortedHashes(common.Hashes{b02.Hash}),
- Witness: types.Witness{
- Height: 3,
- },
- }
- addBlock(b03)
- // Add block 3-1 which acks 3-0.
- b31 := &types.Block{
- ParentHash: b30.Hash,
- Hash: common.NewRandomHash(),
- Timestamp: time.Now().UTC(),
- Position: types.Position{
- ChainID: 3,
- Height: 1,
- },
- Acks: common.NewSortedHashes(common.Hashes{b30.Hash}),
- Witness: types.Witness{
- Height: 1,
- },
- }
- addBlock(b31)
- // Return created blocks.
- blocks = map[uint32]map[uint64]*types.Block{
- 0: map[uint64]*types.Block{
- 0: b00,
- 1: b01,
- 2: b02,
- 3: b03,
- },
- 1: map[uint64]*types.Block{0: b10},
- 2: map[uint64]*types.Block{0: b20},
- 3: map[uint64]*types.Block{0: b30},
- }
- return
-}
-
-// hashBlock is a helper to hash a block and check if any error.
-func (s *LatticeDataTestSuite) hashBlock(b *types.Block) {
- var err error
- b.Hash, err = utils.HashBlock(b)
- s.Require().Nil(err)
-}
-
-func (s *LatticeDataTestSuite) prepareGenesisBlock(
- chainID uint32) (b *types.Block) {
-
- b = &types.Block{
- ParentHash: common.Hash{},
- Position: types.Position{
- ChainID: chainID,
- Height: 0,
- },
- Acks: common.NewSortedHashes(common.Hashes{}),
- Timestamp: time.Now().UTC(),
- }
- s.hashBlock(b)
- return
-}
-
-func (s *LatticeDataTestSuite) TestSanityCheck() {
- var (
- data, blocks = s.genTestCase1()
- req = s.Require()
- )
- check := func(expectedErr error, b *types.Block) {
- s.hashBlock(b)
- err := data.sanityCheck(b, false)
- req.NotNil(err)
- req.IsType(expectedErr, err)
- }
- // Non-genesis block with no ack, should get error.
- check(ErrNotAckParent, &types.Block{
- ParentHash: blocks[1][0].Hash,
- Position: types.Position{
- ChainID: 1,
- Height: 1,
- },
- Acks: common.NewSortedHashes(common.Hashes{}),
- Timestamp: time.Now().UTC(),
- })
- // Non-genesis block which acks its parent but the height is invalid.
- check(ErrInvalidBlockHeight, &types.Block{
- ParentHash: blocks[1][0].Hash,
- Position: types.Position{
- ChainID: 1,
- Height: 2,
- },
- Acks: common.NewSortedHashes(common.Hashes{blocks[1][0].Hash}),
- Timestamp: time.Now().UTC(),
- })
- // Invalid chain ID.
- check(utils.ErrInvalidChainID, &types.Block{
- ParentHash: blocks[1][0].Hash,
- Position: types.Position{
- ChainID: 100,
- Height: 1,
- },
- Acks: common.NewSortedHashes(common.Hashes{blocks[1][0].Hash}),
- Timestamp: time.Now().UTC(),
- })
- // Replicated ack.
- check(ErrDoubleAck, &types.Block{
- ParentHash: blocks[0][3].Hash,
- Position: types.Position{
- ChainID: 0,
- Height: 4,
- },
- Acks: common.NewSortedHashes(common.Hashes{
- blocks[0][3].Hash,
- blocks[1][0].Hash,
- }),
- Timestamp: time.Now().UTC(),
- Witness: types.Witness{
- Height: 4,
- },
- })
- // Acking block doesn't exists.
- check(&ErrAckingBlockNotExists{}, &types.Block{
- ParentHash: blocks[1][0].Hash,
- Position: types.Position{
- ChainID: 1,
- Height: 1,
- },
- Acks: common.NewSortedHashes(common.Hashes{
- blocks[1][0].Hash,
- common.NewRandomHash(),
- }),
- Timestamp: time.Now().UTC(),
- })
- // Parent block on different chain.
- check(&ErrAckingBlockNotExists{}, &types.Block{
- ParentHash: blocks[1][0].Hash,
- Position: types.Position{
- ChainID: 2,
- Height: 1,
- },
- Acks: common.NewSortedHashes(common.Hashes{
- blocks[1][0].Hash,
- blocks[2][0].Hash,
- }),
- Timestamp: time.Now().UTC(),
- })
- // Ack two blocks on the same chain.
- check(ErrDuplicatedAckOnOneChain, &types.Block{
- ParentHash: blocks[2][0].Hash,
- Position: types.Position{
- ChainID: 2,
- Height: 1,
- },
- Acks: common.NewSortedHashes(common.Hashes{
- blocks[2][0].Hash,
- blocks[0][0].Hash,
- blocks[0][1].Hash,
- }),
- Timestamp: time.Now().UTC(),
- })
- // Witness height decreases.
- check(ErrInvalidWitness, &types.Block{
- ParentHash: blocks[0][3].Hash,
- Position: types.Position{
- ChainID: 0,
- Height: 4,
- },
- Timestamp: time.Now().UTC(),
- Acks: common.NewSortedHashes(common.Hashes{
- blocks[0][3].Hash,
- }),
- Witness: types.Witness{
- Height: 2,
- },
- })
- // Add block 3-1 which acks 3-0, and violet reasonable block time interval.
- b := &types.Block{
- ParentHash: blocks[2][0].Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- ChainID: 2,
- Height: 1,
- },
- Acks: common.NewSortedHashes(common.Hashes{blocks[2][0].Hash}),
- Timestamp: time.Now().UTC(),
- }
- // Violet minimum block time interval.
- b.Timestamp = blocks[2][0].Timestamp.Add(1 * time.Nanosecond)
- check(ErrIncorrectBlockTime, b)
- // Add a normal block with timestamp pass round cutting time.
- b11 := &types.Block{
- ParentHash: blocks[1][0].Hash,
- Position: types.Position{
- ChainID: 1,
- Height: 1,
- },
- Acks: common.NewSortedHashes(common.Hashes{blocks[1][0].Hash}),
- Timestamp: time.Now().UTC().Add(500 * time.Second),
- }
- s.hashBlock(b11)
- req.NoError(data.sanityCheck(b11, false))
- _, err := data.addBlock(b11)
- req.NoError(err)
- // A block didn't perform round switching.
- b12 := &types.Block{
- ParentHash: b11.Hash,
- Position: types.Position{
- ChainID: 1,
- Height: 2,
- },
- Acks: common.NewSortedHashes(common.Hashes{b11.Hash}),
- Timestamp: time.Now().UTC().Add(501 * time.Second),
- }
- check(ErrRoundNotSwitch, b12)
- // A block with expected new round ID should be OK.
- b12.Position.Round = 1
- s.hashBlock(b12)
- req.NoError(data.sanityCheck(b12, false))
-}
-
-func (s *LatticeDataTestSuite) TestRandomlyGeneratedBlocks() {
- var (
- chainNum uint32 = 16
- repeat = 10
- delivered []*types.Block
- err error
- req = s.Require()
- datum []*latticeData
- genesisTime = time.Now().UTC()
- )
- if testing.Short() {
- chainNum = 7
- repeat = 3
- }
- // Setup configuration that no restriction on block interval and
- // round cutting.
- genesisConfig := &types.Config{
- RoundInterval: 1000 * time.Second,
- NumChains: chainNum,
- MinBlockInterval: 1 * time.Second,
- }
- // Prepare a randomly generated blocks.
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
- gen := test.NewBlocksGenerator(&test.BlocksGeneratorConfig{
- NumChains: genesisConfig.NumChains,
- MinBlockTimeInterval: genesisConfig.MinBlockInterval,
- }, nil)
- req.NoError(gen.Generate(
- 0,
- genesisTime,
- genesisTime.Add(genesisConfig.RoundInterval),
- dbInst))
- iter, err := dbInst.GetAllBlocks()
- req.NoError(err)
- // Setup a revealer that would reveal blocks randomly but still form
- // valid DAG without holes.
- revealer, err := test.NewRandomDAGBlockRevealer(iter)
- req.Nil(err)
-
- revealedHashesAsString := map[string]struct{}{}
- deliveredHashesAsString := map[string]struct{}{}
- for i := 0; i < repeat; i++ {
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
- data := newLatticeData(dbInst, genesisTime, 0, genesisConfig)
- deliveredHashes := common.Hashes{}
- revealedHashes := common.Hashes{}
- revealer.Reset()
- for {
- // Reveal next block.
- b, err := revealer.NextBlock()
- if err != nil {
- if err == db.ErrIterationFinished {
- err = nil
- break
- }
- }
- req.NoError(err)
- revealedHashes = append(revealedHashes, b.Hash)
- // Pass blocks to lattice.
- req.NoError(data.sanityCheck(&b, false))
- delivered, err = data.addBlock(&b)
- req.NoError(err)
- for _, b := range delivered {
- deliveredHashes = append(deliveredHashes, b.Hash)
- }
- }
- // To make it easier to check, sort hashes of
- // delivered blocks, and concatenate them into
- // a string.
- sort.Sort(deliveredHashes)
- asString := ""
- for _, h := range deliveredHashes {
- asString += h.String() + ","
- }
- deliveredHashesAsString[asString] = struct{}{}
- // Compose revealing hash sequense to string.
- asString = ""
- for _, h := range revealedHashes {
- asString += h.String() + ","
- }
- revealedHashesAsString[asString] = struct{}{}
- datum = append(datum, data)
- }
- // Make sure concatenated hashes of delivered blocks are identical.
- req.Len(deliveredHashesAsString, 1)
- for h := range deliveredHashesAsString {
- // Make sure at least some blocks are delivered.
- req.True(len(h) > 0)
- }
- // Make sure we test for more than 1 revealing sequence.
- req.True(len(revealedHashesAsString) > 1)
- // Make sure each latticeData instance have identical working set.
- req.True(len(datum) >= repeat)
- for i, bI := range datum {
- for j, bJ := range datum {
- if i == j {
- continue
- }
- // Check chain status of this pair.
- for chainID, statusI := range bI.chains {
- req.Equal(statusI.tip, bJ.chains[chainID].tip)
- req.Equal(len(statusI.blocks), len(bJ.chains[chainID].blocks))
- // Check lastAckPos.
- for x, pos := range statusI.lastAckPos {
- req.Equal(pos, bJ.chains[chainID].lastAckPos[x])
- }
- // Check blocks.
- if len(statusI.blocks) > 0 {
- req.Equal(statusI.blocks[0], bJ.chains[chainID].blocks[0])
- }
- }
- }
- }
-}
-
-func (s *LatticeDataTestSuite) TestPrepareBlock() {
- var (
- chainNum uint32 = 4
- req = s.Require()
- minInterval = 50 * time.Millisecond
- delivered []*types.Block
- err error
- )
- // Setup configuration that no restriction on block interval and
- // round cutting.
- genesisConfig := &types.Config{
- RoundInterval: 3000 * time.Second,
- NumChains: chainNum,
- MinBlockInterval: 1 * time.Second,
- }
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
- data := newLatticeData(dbInst, time.Now().UTC(), 0, genesisConfig)
- // Setup genesis blocks.
- b00 := s.prepareGenesisBlock(0)
- time.Sleep(minInterval)
- b10 := s.prepareGenesisBlock(1)
- time.Sleep(minInterval)
- b20 := s.prepareGenesisBlock(2)
- time.Sleep(minInterval)
- b30 := s.prepareGenesisBlock(3)
- // Submit these blocks to lattice.
- delivered, err = data.addBlock(b00)
- req.NoError(err)
- req.Len(delivered, 1)
- delivered, err = data.addBlock(b10)
- req.NoError(err)
- req.Len(delivered, 1)
- delivered, err = data.addBlock(b20)
- req.NoError(err)
- req.Len(delivered, 1)
- delivered, err = data.addBlock(b30)
- req.NoError(err)
- req.Len(delivered, 1)
- // We should be able to collect all 4 genesis blocks by calling
- // prepareBlock.
- b11 := &types.Block{
- Position: types.Position{
- ChainID: 1,
- },
- Timestamp: time.Now().UTC(),
- }
- req.NoError(data.prepareBlock(b11))
- s.hashBlock(b11)
- req.Contains(b11.Acks, b00.Hash)
- req.Contains(b11.Acks, b10.Hash)
- req.Contains(b11.Acks, b20.Hash)
- req.Contains(b11.Acks, b30.Hash)
- req.Equal(b11.ParentHash, b10.Hash)
- req.Equal(b11.Position.Height, uint64(1))
- delivered, err = data.addBlock(b11)
- req.Len(delivered, 1)
- req.NoError(err)
- // Propose/Process a block based on collected info.
- b12 := &types.Block{
- Position: types.Position{
- ChainID: 1,
- },
- Timestamp: time.Now().UTC(),
- }
- req.NoError(data.prepareBlock(b12))
- s.hashBlock(b12)
- // This time we only need to ack b11.
- req.Len(b12.Acks, 1)
- req.Contains(b12.Acks, b11.Hash)
- req.Equal(b12.ParentHash, b11.Hash)
- req.Equal(b12.Position.Height, uint64(2))
- // When calling with other validator ID, we should be able to
- // get 4 blocks to ack.
- b01 := &types.Block{
- Position: types.Position{
- ChainID: 0,
- },
- }
- req.NoError(data.prepareBlock(b01))
- s.hashBlock(b01)
- req.Len(b01.Acks, 4)
- req.Contains(b01.Acks, b00.Hash)
- req.Contains(b01.Acks, b11.Hash)
- req.Contains(b01.Acks, b20.Hash)
- req.Contains(b01.Acks, b30.Hash)
- req.Equal(b01.ParentHash, b00.Hash)
- req.Equal(b01.Position.Height, uint64(1))
-}
-
-func (s *LatticeDataTestSuite) TestNextBlock() {
- // Test 'NextBlock' method when lattice is ready.
- data, blocks := s.genTestCase1()
- h, ts, err := data.nextBlock(0, 0)
- s.Require().NoError(err)
- s.Require().Equal(uint64(4), h)
- // 2ns of minBlockTime is defined in genTestCase1().
- s.Require().Equal(blocks[0][3].Timestamp.Add(2*time.Nanosecond), ts)
- // Test 'NextHeight' method when lattice is empty.
- // Setup a configuration that no restriction on block interval and
- // round cutting.
- genesisConfig := &types.Config{
- RoundInterval: 1000 * time.Second,
- NumChains: 4,
- MinBlockInterval: 1 * time.Second,
- }
- now := time.Now().UTC()
- data = newLatticeData(nil, now, 0, genesisConfig)
- h, ts, err = data.nextBlock(0, 0)
- s.Require().NoError(err)
- s.Require().Equal(now, ts)
- s.Require().Equal(uint64(0), h)
-}
-
-func (s *LatticeDataTestSuite) TestPrepareEmptyBlock() {
- data, _ := s.genTestCase1()
- b := &types.Block{
- Position: types.Position{
- ChainID: 0,
- },
- }
- data.prepareEmptyBlock(b)
- s.True(b.IsEmpty())
- s.Equal(uint64(4), b.Position.Height)
-}
-
-func (s *LatticeDataTestSuite) TestNumChainsChange() {
- // This test case verify the behavior when NumChains
- // changes. We only reply on methods of latticeData
- // for test. It would run in this way:
- // - Several configs would be prepared in advance, scenario for NumChains
- // increasing and decreasing would be included.
- // - Blocks would be prepared from candidate chains.
- // - Once a block is detected as last block of that chain in that round,
- // that chain would be revmoved from candidate chains.
- // - Once candidate chains are empty, proceed to next round until the last
- // round.
- //
- // These scenarioes would be checked in this test:
- // - Each block generated successfully by latticeData.prepareBlock
- // should be no error when passing to latticeData.sanityCheck
- // and latticeData.addBlock.
- // - The delivered blocks should form a valid DAG.
- fixConfig := func(config *types.Config) *types.Config {
- config.MinBlockInterval = 10 * time.Second
- config.RoundInterval = 100 * time.Second
- return config
- }
- var (
- req = s.Require()
- maxChains = uint32(16)
- configs = []*types.Config{
- fixConfig(&types.Config{NumChains: 13}),
- fixConfig(&types.Config{NumChains: 10}),
- fixConfig(&types.Config{NumChains: maxChains}),
- fixConfig(&types.Config{NumChains: 7}),
- }
- randObj = rand.New(rand.NewSource(time.Now().UnixNano()))
- )
- // Setup db instance.
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
- // Set up latticeData instance.
- lattice := newLatticeData(dbInst, time.Now().UTC(), 0, configs[0])
- req.NoError(lattice.appendConfig(1, configs[1]))
- req.NoError(lattice.appendConfig(2, configs[2]))
- req.NoError(lattice.appendConfig(3, configs[3]))
- // Run until candidate chains are empty.
- var (
- delivered []*types.Block
- candidateChainIDs []uint32
- nextRound uint64
- )
- for {
- // Decide chainID.
- if len(candidateChainIDs) == 0 {
- // Proceed to next round.
- if nextRound >= uint64(len(configs)) {
- break
- }
- c := configs[nextRound]
- nextRound++
- for i := uint32(0); i < c.NumChains; i++ {
- candidateChainIDs = append(candidateChainIDs, i)
- }
- }
- chainID := candidateChainIDs[randObj.Intn(len(candidateChainIDs))]
- // Prepare blocks, see if we are legal to propose block at
- // this position.
- b := &types.Block{
- Position: types.Position{
- ChainID: chainID,
- Round: nextRound - 1,
- }}
- err = lattice.prepareBlock(b)
- if err == ErrRoundNotSwitch {
- // This round is done, remove this channel from candidate.
- for i := range candidateChainIDs {
- if candidateChainIDs[i] != chainID {
- continue
- }
- candidateChainIDs = append(
- candidateChainIDs[:i], candidateChainIDs[i+1:]...)
- break
- }
- continue
- }
- req.NoError(err)
- s.hashBlock(b)
- // Do the actual lattice usage.
- req.NoError(lattice.sanityCheck(b, false))
- d, err := lattice.addBlock(b)
- req.NoError(err)
- delivered = append(delivered, d...)
- }
- // verify delivered form a DAG.
- dag := map[common.Hash]struct{}{}
- for _, b := range delivered {
- for _, ack := range b.Acks {
- _, exists := dag[ack]
- req.True(exists)
- }
- dag[b.Hash] = struct{}{}
- }
-}
-
-func (s *LatticeDataTestSuite) TestAppendConfig() {
- var (
- req = s.Require()
- now = time.Now().UTC()
- round = uint64(5)
- cfg = &types.Config{NumChains: uint32(4)}
- )
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
- latticeData := newLatticeData(dbInst, now, round, cfg)
- err = latticeData.appendConfig(6, cfg)
- req.NoError(err)
- err = latticeData.appendConfig(10, cfg)
- req.Equal(err, ErrRoundNotIncreasing)
-}
-
-func TestLatticeData(t *testing.T) {
- suite.Run(t, new(LatticeDataTestSuite))
-}
diff --git a/core/lattice.go b/core/lattice.go
deleted file mode 100644
index de0e549..0000000
--- a/core/lattice.go
+++ /dev/null
@@ -1,363 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package core
-
-import (
- "fmt"
- "sync"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core/db"
- "github.com/dexon-foundation/dexon-consensus/core/types"
- "github.com/dexon-foundation/dexon-consensus/core/utils"
-)
-
-// Errors for sanity check error.
-var (
- ErrRetrySanityCheckLater = fmt.Errorf("retry sanity check later")
-)
-
-// Lattice represents a unit to produce a global ordering from multiple chains.
-type Lattice struct {
- lock sync.RWMutex
- signer *utils.Signer
- app Application
- debug Debug
- pool blockPool
- data *latticeData
- toModule *totalOrdering
- ctModule *consensusTimestamp
- logger common.Logger
-}
-
-// NewLattice constructs an Lattice instance.
-func NewLattice(
- dMoment time.Time,
- round uint64,
- cfg *types.Config,
- signer *utils.Signer,
- app Application,
- debug Debug,
- db db.Database,
- logger common.Logger) *Lattice {
-
- // Create genesis latticeDataConfig.
- return &Lattice{
- signer: signer,
- app: app,
- debug: debug,
- pool: newBlockPool(cfg.NumChains),
- data: newLatticeData(db, dMoment, round, cfg),
- toModule: newTotalOrdering(dMoment, round, cfg),
- ctModule: newConsensusTimestamp(dMoment, round, cfg.NumChains),
- logger: logger,
- }
-}
-
-// PrepareBlock setups block's fields based on current status.
-func (l *Lattice) PrepareBlock(
- b *types.Block, proposeTime time.Time) (err error) {
-
- l.lock.RLock()
- defer l.lock.RUnlock()
-
- b.Timestamp = proposeTime
- if err = l.data.prepareBlock(b); err != nil {
- return
- }
- l.logger.Debug("Calling Application.PreparePayload", "position", &b.Position)
- if b.Payload, err = l.app.PreparePayload(b.Position); err != nil {
- return
- }
- l.logger.Debug("Calling Application.PrepareWitness",
- "height", b.Witness.Height)
- if b.Witness, err = l.app.PrepareWitness(b.Witness.Height); err != nil {
- return
- }
- err = l.signer.SignBlock(b)
- return
-}
-
-// PrepareEmptyBlock setups block's fields based on current lattice status.
-func (l *Lattice) PrepareEmptyBlock(b *types.Block) (err error) {
- l.lock.RLock()
- defer l.lock.RUnlock()
- if err = l.data.prepareEmptyBlock(b); err != nil {
- return
- }
- b.Hash, err = utils.HashBlock(b)
- return
-}
-
-// AddShallowBlock adds a hash of a block that is confirmed by other nodes but
-// the content is not arrived yet.
-func (l *Lattice) AddShallowBlock(hash common.Hash, pos types.Position) {
- l.lock.Lock()
- defer l.lock.Unlock()
- l.data.addShallowBlock(hash, pos)
-}
-
-// SanityCheck checks the validity of a block.
-//
-// If any acking block of this block does not exist, Lattice caches this block
-// and retries when Lattice.ProcessBlock is called.
-func (l *Lattice) SanityCheck(b *types.Block, allowShallow bool) (err error) {
- if b.IsEmpty() {
- // Only need to verify block's hash.
- var hash common.Hash
- if hash, err = utils.HashBlock(b); err != nil {
- return
- }
- if b.Hash != hash {
- return ErrInvalidBlock
- }
- } else {
- // Verify block's signature.
- if err = utils.VerifyBlockSignature(b); err != nil {
- return
- }
- }
- // Make sure acks are sorted.
- for i := range b.Acks {
- if i == 0 {
- continue
- }
- if !b.Acks[i-1].Less(b.Acks[i]) {
- err = ErrAcksNotSorted
- return
- }
- }
- l.lock.RLock()
- defer l.lock.RUnlock()
- if err = l.data.sanityCheck(b, allowShallow); err != nil {
- if _, ok := err.(*ErrAckingBlockNotExists); ok {
- err = ErrRetrySanityCheckLater
- }
- return
- }
- return
-}
-
-// Exist checks if the block is known to lattice.
-func (l *Lattice) Exist(hash common.Hash) bool {
- l.lock.RLock()
- defer l.lock.RUnlock()
- _, err := l.data.findBlock(hash)
- return err == nil
-}
-
-// addBlockToLattice adds a block into lattice, and delivers blocks with the
-// acks already delivered.
-//
-// NOTE: input block should pass sanity check.
-func (l *Lattice) addBlockToLattice(
- input *types.Block) (outputBlocks []*types.Block, err error) {
-
- if tip := l.data.chains[input.Position.ChainID].tip; tip != nil {
- if !input.Position.Newer(&tip.Position) {
- l.logger.Warn("Dropping block: older than tip",
- "block", input, "tip", tip)
- return
- }
- }
- l.pool.addBlock(input)
- // Check tips in pool to check their validity for moving blocks from pool
- // to lattice.
- for {
- hasOutput := false
- for i := uint32(0); i < uint32(len(l.pool)); i++ {
- var tip *types.Block
- if tip = l.pool.tip(i); tip == nil {
- continue
- }
- err = l.data.sanityCheck(tip, false)
- if err == nil {
- var output []*types.Block
- if output, err = l.data.addBlock(tip); err != nil {
- // We should be able to add this block once sanity check
- // passed.
- l.logger.Error("Failed to add sanity-checked block",
- "block", tip, "error", err)
- panic(err)
- }
- delete(l.data.shallowBlocks, tip.Hash)
- hasOutput = true
- outputBlocks = append(outputBlocks, output...)
- l.pool.removeTip(i)
- continue
- }
- if _, ok := err.(*ErrAckingBlockNotExists); ok {
- l.logger.Debug("Pending block for lattice",
- "pending", tip,
- "err", err,
- "last", l.data.chains[tip.Position.ChainID].tip)
- err = nil
- continue
- } else {
- l.logger.Error("Unexpected sanity check error",
- "block", tip, "error", err)
- panic(err)
- }
- }
- if !hasOutput {
- break
- }
- }
-
- for _, b := range outputBlocks {
- l.logger.Debug("Calling Application.BlockConfirmed", "block", b)
- l.app.BlockConfirmed(*b.Clone())
- // Purge blocks in pool with the same chainID and lower height.
- l.pool.purgeBlocks(b.Position.ChainID, b.Position.Height)
- }
-
- return
-}
-
-// ProcessBlock adds a block into lattice, and deliver ordered blocks.
-// If any block pass sanity check after this block add into lattice, they
-// would be returned, too.
-//
-// NOTE: assume the block passed sanity check.
-func (l *Lattice) ProcessBlock(
- input *types.Block) (delivered []*types.Block, err error) {
- var (
- b *types.Block
- inLattice []*types.Block
- toDelivered []*types.Block
- deliveredMode uint32
- )
- l.lock.Lock()
- defer l.lock.Unlock()
- if inLattice, err = l.addBlockToLattice(input); err != nil {
- return
- }
- if len(inLattice) == 0 {
- return
- }
- for _, b = range inLattice {
- if err = l.toModule.addBlock(b); err != nil {
- // All errors from total ordering is serious, should panic.
- panic(err)
- }
- }
- for {
- toDelivered, deliveredMode, err = l.toModule.extractBlocks()
- if err != nil {
- panic(err)
- }
- if len(toDelivered) == 0 {
- break
- }
- if l.debug != nil {
- hashes := make(common.Hashes, len(toDelivered))
- for idx := range toDelivered {
- hashes[idx] = toDelivered[idx].Hash
- }
- l.debug.TotalOrderingDelivered(hashes, deliveredMode)
- }
- // Perform consensus timestamp module.
- if err = l.ctModule.processBlocks(toDelivered); err != nil {
- break
- }
- delivered = append(delivered, toDelivered...)
- }
- return
-}
-
-// NextBlock returns expected height and timestamp of incoming block for
-// specified chain and given round.
-func (l *Lattice) NextBlock(round uint64, chainID uint32) (
- uint64, time.Time, error) {
- l.lock.RLock()
- defer l.lock.RUnlock()
- return l.data.nextBlock(round, chainID)
-}
-
-// TipRound returns the round of the tip of given chain.
-func (l *Lattice) TipRound(chainID uint32) uint64 {
- l.lock.RLock()
- defer l.lock.RUnlock()
- return l.data.tipRound(chainID)
-}
-
-// PurgeBlocks purges blocks' cache in memory, this is called when the caller
-// makes sure those blocks are already saved in db.
-func (l *Lattice) PurgeBlocks(blocks []*types.Block) error {
- l.lock.Lock()
- defer l.lock.Unlock()
- return l.data.purgeBlocks(blocks)
-}
-
-// AppendConfig adds a new config for upcoming rounds. If a config of round r is
-// added, only config in round r + 1 is allowed next.
-func (l *Lattice) AppendConfig(round uint64, config *types.Config) (err error) {
- l.lock.Lock()
- defer l.lock.Unlock()
- l.pool.resize(config.NumChains)
- if err = l.data.appendConfig(round, config); err != nil {
- return
- }
- if err = l.toModule.appendConfig(round, config); err != nil {
- return
- }
- if err = l.ctModule.appendConfig(round, config); err != nil {
- return
- }
- return
-}
-
-// ProcessFinalizedBlock is used for syncing lattice data.
-func (l *Lattice) ProcessFinalizedBlock(
- b *types.Block) (delivered []*types.Block, err error) {
- var (
- toDelivered []*types.Block
- deliveredMode uint32
- )
- l.lock.Lock()
- defer l.lock.Unlock()
- // Syncing state for core.latticeData module.
- if err = l.data.addFinalizedBlock(b); err != nil {
- return
- }
- l.pool.purgeBlocks(b.Position.ChainID, b.Position.Height)
- // Syncing state for core.totalOrdering module.
- if err = l.toModule.addBlock(b); err != nil {
- return
- }
- for {
- toDelivered, deliveredMode, err = l.toModule.extractBlocks()
- if err != nil || len(toDelivered) == 0 {
- break
- }
- hashes := make(common.Hashes, len(toDelivered))
- for idx := range toDelivered {
- hashes[idx] = toDelivered[idx].Hash
- }
- if l.debug != nil {
- l.debug.TotalOrderingDelivered(hashes, deliveredMode)
- }
- // Sync core.consensusTimestamp module.
- if err = l.ctModule.processBlocks(toDelivered); err != nil {
- break
- }
- delivered = append(delivered, toDelivered...)
- }
- return
-}
diff --git a/core/lattice_test.go b/core/lattice_test.go
deleted file mode 100644
index b847fad..0000000
--- a/core/lattice_test.go
+++ /dev/null
@@ -1,262 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package core
-
-import (
- "math/rand"
- "testing"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core/crypto/ecdsa"
- "github.com/dexon-foundation/dexon-consensus/core/db"
- "github.com/dexon-foundation/dexon-consensus/core/test"
- "github.com/dexon-foundation/dexon-consensus/core/types"
- "github.com/dexon-foundation/dexon-consensus/core/utils"
- "github.com/stretchr/testify/suite"
-)
-
-// testLatticeMgr wraps compaction chain and lattice.
-type testLatticeMgr struct {
- lattice *Lattice
- ccModule *compactionChain
- app *test.App
- db db.Database
-}
-
-func (mgr *testLatticeMgr) prepareBlock(
- chainID uint32) (b *types.Block, err error) {
-
- b = &types.Block{
- Position: types.Position{
- ChainID: chainID,
- }}
- err = mgr.lattice.PrepareBlock(b, time.Now().UTC())
- return
-}
-
-// Process describes the usage of Lattice.ProcessBlock.
-func (mgr *testLatticeMgr) processBlock(b *types.Block) (err error) {
- var (
- delivered []*types.Block
- )
- if err = mgr.lattice.SanityCheck(b, false); err != nil {
- if err == ErrRetrySanityCheckLater {
- err = nil
- } else {
- return
- }
- }
- if err = mgr.db.PutBlock(*b); err != nil {
- if err != db.ErrBlockExists {
- return
- }
- err = nil
- }
- if delivered, err = mgr.lattice.ProcessBlock(b); err != nil {
- return
- }
- // Deliver blocks.
- for _, b = range delivered {
- if err = mgr.ccModule.processBlock(b); err != nil {
- return
- }
- }
- for _, b = range mgr.ccModule.extractBlocks() {
- if err = mgr.db.UpdateBlock(*b); err != nil {
- return
- }
- mgr.app.BlockDelivered(b.Hash, b.Position, b.Finalization)
- }
- if err = mgr.lattice.PurgeBlocks(delivered); err != nil {
- return
- }
- return
-}
-
-type LatticeTestSuite struct {
- suite.Suite
-}
-
-func (s *LatticeTestSuite) newTestLatticeMgr(
- cfg *types.Config, dMoment time.Time) *testLatticeMgr {
- var req = s.Require()
- // Setup private key.
- prvKey, err := ecdsa.NewPrivateKey()
- req.NoError(err)
- // Setup db.
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
- // Setup governance.
- logger := &common.NullLogger{}
- _, pubKeys, err := test.NewKeys(int(cfg.NotarySetSize))
- req.NoError(err)
- gov, err := test.NewGovernance(test.NewState(
- pubKeys, cfg.LambdaBA, logger, true), ConfigRoundShift)
- req.NoError(err)
- // Setup application.
- app := test.NewApp(0, gov)
- // Setup compaction chain.
- cc := newCompactionChain(gov)
- cc.init(&types.Block{})
- mock := newMockTSigVerifier(true)
- for i := 0; i < cc.tsigVerifier.cacheSize; i++ {
- cc.tsigVerifier.verifier[uint64(i)] = mock
- }
- // Setup lattice.
- return &testLatticeMgr{
- ccModule: cc,
- app: app,
- db: dbInst,
- lattice: NewLattice(
- dMoment,
- 0,
- cfg,
- utils.NewSigner(prvKey),
- app,
- app,
- dbInst,
- logger)}
-}
-
-func (s *LatticeTestSuite) TestBasicUsage() {
- // One Lattice prepare blocks on chains randomly selected each time
- // and process it. Those generated blocks and kept into a buffer, and
- // process by other Lattice instances with random order.
- var (
- blockNum = 100
- chainNum = uint32(19)
- otherLatticeNum = 20
- req = s.Require()
- err error
- cfg = types.Config{
- NumChains: chainNum,
- NotarySetSize: chainNum,
- PhiRatio: float32(2) / float32(3),
- K: 0,
- MinBlockInterval: 0,
- RoundInterval: time.Hour,
- }
- dMoment = time.Now().UTC()
- master = s.newTestLatticeMgr(&cfg, dMoment)
- apps = []*test.App{master.app}
- revealSeq = map[string]struct{}{}
- )
- // Master-lattice generates blocks.
- for i := uint32(0); i < chainNum; i++ {
- // Produced genesis blocks should be delivered before all other blocks,
- // or the consensus time would be wrong.
- b, err := master.prepareBlock(i)
- req.NotNil(b)
- req.NoError(err)
- // Ignore error "acking blocks don't exist".
- req.NoError(master.processBlock(b))
- }
- for i := 0; i < (blockNum - int(chainNum)); i++ {
- b, err := master.prepareBlock(uint32(rand.Intn(int(chainNum))))
- req.NotNil(b)
- req.NoError(err)
- // Ignore error "acking blocks don't exist".
- req.NoError(master.processBlock(b))
- }
- // Now we have some blocks, replay them on different lattices.
- iter, err := master.db.GetAllBlocks()
- req.NoError(err)
- revealer, err := test.NewRandomBlockRevealer(iter)
- req.NoError(err)
- for i := 0; i < otherLatticeNum; i++ {
- revealer.Reset()
- revealed := ""
- other := s.newTestLatticeMgr(&cfg, dMoment)
- for {
- b, err := revealer.NextBlock()
- if err != nil {
- if err == db.ErrIterationFinished {
- err = nil
- break
- }
- }
- req.NoError(err)
- req.NoError(other.processBlock(&b))
- revealed += b.Hash.String() + ","
- }
- revealSeq[revealed] = struct{}{}
- apps = append(apps, other.app)
- }
- // Make sure not only one revealing sequence.
- req.True(len(revealSeq) > 1)
- // Make sure nothing goes wrong.
- for i, app := range apps {
- err := app.Verify()
- req.NoError(err)
- for j, otherApp := range apps {
- if i >= j {
- continue
- }
- err := app.Compare(otherApp)
- s.NoError(err)
- }
- }
-}
-
-func (s *LatticeTestSuite) TestSanityCheck() {
- // This sanity check focuses on hash/signature part.
- var (
- chainNum = uint32(19)
- cfg = types.Config{
- NumChains: chainNum,
- PhiRatio: float32(2) / float32(3),
- K: 0,
- MinBlockInterval: 0,
- }
- lattice = s.newTestLatticeMgr(&cfg, time.Now().UTC()).lattice
- signer = lattice.signer // Steal signer module from lattice, :(
- req = s.Require()
- err error
- )
- // A block properly signed should pass sanity check.
- b := &types.Block{
- Position: types.Position{ChainID: 0},
- Timestamp: time.Now().UTC(),
- }
- req.NoError(signer.SignBlock(b))
- req.NoError(lattice.SanityCheck(b, false))
- // A block with incorrect signature should not pass sanity check.
- otherPrvKey, err := ecdsa.NewPrivateKey()
- req.NoError(err)
- b.Signature, err = otherPrvKey.Sign(common.NewRandomHash())
- req.Equal(lattice.SanityCheck(b, false), ErrIncorrectSignature)
- // A block with un-sorted acks should not pass sanity check.
- b.Acks = common.NewSortedHashes(common.Hashes{
- common.NewRandomHash(),
- common.NewRandomHash(),
- common.NewRandomHash(),
- common.NewRandomHash(),
- common.NewRandomHash(),
- })
- b.Acks[0], b.Acks[1] = b.Acks[1], b.Acks[0]
- req.NoError(signer.SignBlock(b))
- req.Equal(lattice.SanityCheck(b, false), ErrAcksNotSorted)
- // A block with incorrect hash should not pass sanity check.
- b.Hash = common.NewRandomHash()
- req.Equal(lattice.SanityCheck(b, false), ErrIncorrectHash)
-}
-
-func TestLattice(t *testing.T) {
- suite.Run(t, new(LatticeTestSuite))
-}
diff --git a/core/negative-ack.go b/core/negative-ack.go
deleted file mode 100644
index 4178629..0000000
--- a/core/negative-ack.go
+++ /dev/null
@@ -1,211 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package core
-
-import (
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/core/types"
-)
-
-type negativeAck struct {
- // owner is the ID of proposer itself, this is used when deciding
- // a node to be restricted or not.
- owner types.NodeID
-
- numOfNodes int
-
- // timeDelay and timeExpire are for nack timeout.
- timeDelay time.Duration
- timeExpire time.Duration
-
- // restricteds stores nodes which has been restricted and the time it's
- // restricted.
- restricteds map[types.NodeID]time.Time
-
- // lastVotes and lockedVotes store the votes for nack. lastVotes[nid1][nid2]
- // and lockedVotes[nid1][nid2] both mean that nid2 votes nid1. The difference
- // is lockedVotes works only when nid1 is restricted, so that the votes are
- // needed to be locked.
- lastVotes map[types.NodeID]map[types.NodeID]struct{}
- lockedVotes map[types.NodeID]map[types.NodeID]struct{}
-
- // timeDiffs is the cache for last time stamps. timeDiffs[nid1][nid2] means
- // the last updated timestamps nid1 sees nid2.
- timeDiffs map[types.NodeID]map[types.NodeID]map[types.NodeID]time.Time
-}
-
-// newNegativeAck creates a new negaticeAck instance.
-func newNegativeAck(nid types.NodeID) *negativeAck {
- n := &negativeAck{
- owner: nid,
- numOfNodes: 0,
- restricteds: make(map[types.NodeID]time.Time),
- lastVotes: make(map[types.NodeID]map[types.NodeID]struct{}),
- lockedVotes: make(map[types.NodeID]map[types.NodeID]struct{}),
- timeDiffs: make(map[types.NodeID]map[types.NodeID]map[types.NodeID]time.Time),
- }
- n.addNode(nid)
- return n
-}
-
-// processNewVote is called when a new "vote" occurs, that is, a node
-// sees that other 2f + 1 nodes think a node is slow. "nid" is the
-// node which propesed the block which the timestamps votes and "h" is
-// the node been voted to be nacked.
-func (n *negativeAck) processNewVote(
- nid types.NodeID,
- h types.NodeID,
-) []types.NodeID {
-
- nackeds := []types.NodeID{}
- if _, exist := n.restricteds[h]; exist {
- n.lockedVotes[h][nid] = struct{}{}
- if len(n.lockedVotes[h]) > 2*(n.numOfNodes-1)/3 {
- nackeds = append(nackeds, h)
- delete(n.restricteds, h)
- }
- } else {
- if n.owner == nid {
- n.restrict(h)
- } else {
- n.lastVotes[h][nid] = struct{}{}
- if len(n.lastVotes[h]) > (n.numOfNodes-1)/3 {
- n.restrict(h)
- }
- }
- }
- return nackeds
-}
-
-// processTimestamps process new timestamps of a block which is proposed by
-// node nid, and returns the nodes being nacked.
-func (n *negativeAck) processTimestamps(
- nid types.NodeID,
- ts map[types.NodeID]time.Time,
-) []types.NodeID {
-
- n.checkRestrictExpire()
-
- nackeds := []types.NodeID{}
- for h := range n.timeDiffs {
- if n.timeDiffs[nid][h][h].Equal(ts[h]) {
- votes := 0
- for hh := range n.timeDiffs {
- if ts[hh].Sub(n.timeDiffs[nid][h][hh]) >= n.timeDelay {
- votes++
- }
- }
- if votes > 2*((n.numOfNodes-1)/3) {
- n.lastVotes[h][nid] = struct{}{}
- nack := n.processNewVote(nid, h)
- for _, i := range nack {
- nackeds = append(nackeds, i)
- }
- } else {
- delete(n.lastVotes[h], nid)
- }
- } else {
- for hh := range n.timeDiffs {
- n.timeDiffs[nid][h][hh] = ts[hh]
- }
- delete(n.lastVotes[h], nid)
- }
- }
- return nackeds
-}
-
-func (n *negativeAck) checkRestrictExpire() {
- expired := []types.NodeID{}
- now := time.Now()
- for h, t := range n.restricteds {
- if now.Sub(t) >= n.timeExpire {
- expired = append(expired, h)
- }
- }
- for _, h := range expired {
- delete(n.restricteds, h)
- }
-}
-
-func (n *negativeAck) restrict(nid types.NodeID) {
- if _, exist := n.restricteds[nid]; !exist {
- n.restricteds[nid] = time.Now().UTC()
- n.lockedVotes[nid] = map[types.NodeID]struct{}{}
- for h := range n.lastVotes[nid] {
- n.lockedVotes[nid][h] = struct{}{}
- }
- }
-}
-
-func (n *negativeAck) getRestrictedNodes() map[types.NodeID]struct{} {
- n.checkRestrictExpire()
- ret := map[types.NodeID]struct{}{}
- for h := range n.restricteds {
- ret[h] = struct{}{}
- }
- return ret
-}
-
-func (n *negativeAck) setTimeDelay(t time.Duration) {
- n.timeDelay = t
-}
-
-func (n *negativeAck) setTimeExpire(t time.Duration) {
- n.timeExpire = t
-}
-
-func (n *negativeAck) addNode(nid types.NodeID) {
- n.numOfNodes++
- n.lastVotes[nid] = make(map[types.NodeID]struct{})
- n.lockedVotes[nid] = make(map[types.NodeID]struct{})
-
- newTimeDiff := make(map[types.NodeID]map[types.NodeID]time.Time)
- for h := range n.timeDiffs {
- newTimeDiff2 := make(map[types.NodeID]time.Time)
- for hh := range n.timeDiffs {
- newTimeDiff2[hh] = time.Time{}
- }
- newTimeDiff[h] = newTimeDiff2
- }
- n.timeDiffs[nid] = newTimeDiff
- for h := range n.timeDiffs {
- n.timeDiffs[h][nid] = make(map[types.NodeID]time.Time)
- }
-}
-
-func (n *negativeAck) deleteNode(nid types.NodeID) {
- n.numOfNodes--
-
- delete(n.timeDiffs, nid)
-
- for h := range n.lastVotes {
- delete(n.lastVotes[h], nid)
- }
- delete(n.lastVotes, nid)
- delete(n.lockedVotes, nid)
-
- for h := range n.timeDiffs {
- delete(n.timeDiffs[h], nid)
- for hh := range n.timeDiffs[h] {
- delete(n.timeDiffs[h][hh], nid)
- }
- }
-
- delete(n.restricteds, nid)
-}
diff --git a/core/negative-ack_test.go b/core/negative-ack_test.go
deleted file mode 100644
index 77302ad..0000000
--- a/core/negative-ack_test.go
+++ /dev/null
@@ -1,228 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package core
-
-import (
- "testing"
- "time"
-
- "github.com/stretchr/testify/suite"
-
- "github.com/dexon-foundation/dexon-consensus/core/test"
- "github.com/dexon-foundation/dexon-consensus/core/types"
-)
-
-var (
- baseTime = time.Now().UTC()
- timeDelay = 2 * time.Second
- timeExpire = 100 * time.Millisecond
-)
-
-type NegativeAckTest struct {
- suite.Suite
-}
-
-func (s *NegativeAckTest) SetupSuite() {
-
-}
-
-func (s *NegativeAckTest) SetupTest() {
-
-}
-
-func (s *NegativeAckTest) checkLastVotes(
- nids []types.NodeID,
- vs map[types.NodeID]map[types.NodeID]struct{},
- a [][]bool,
-) {
-
- for i := 0; i < len(nids); i++ {
- for j := 0; j < len(nids); j++ {
- _, exist := vs[nids[i]][nids[j]]
- s.Require().Equal(a[i][j], exist)
- }
- }
-}
-
-func (s *NegativeAckTest) checkTimeDiff(
- nids []types.NodeID,
- ts map[types.NodeID]map[types.NodeID]time.Time,
- a [][]int,
-) {
-
- for i := 0; i < len(nids); i++ {
- for j := 0; j < len(nids); j++ {
- s.Require().Equal(
- time.Duration(a[i][j])*timeDelay,
- ts[nids[i]][nids[j]].Sub(baseTime),
- )
- }
- }
-}
-
-func genTimestamp(nids []types.NodeID, a []int) map[types.NodeID]time.Time {
- ts := map[types.NodeID]time.Time{}
- for i := 0; i < len(nids); i++ {
- ts[nids[i]] = baseTime.Add(time.Duration(a[i]) * timeDelay)
- }
- return ts
-}
-
-func genTestNegativeAck(num int) (*negativeAck, []types.NodeID) {
- nids := test.GenerateRandomNodeIDs(num)
- n := newNegativeAck(nids[0])
- for i := 1; i < num; i++ {
- n.addNode(nids[i])
- }
- return n, nids
-}
-
-func (s *NegativeAckTest) TestProcessTimestamps() {
- n, nids := genTestNegativeAck(4)
- n.setTimeDelay(timeDelay)
- n.setTimeExpire(timeExpire)
-
- n.processTimestamps(nids[0], genTimestamp(nids, []int{1, 1, 1, 0}))
- s.checkTimeDiff(nids, n.timeDiffs[nids[0]], [][]int{
- {1, 1, 1, 0},
- {1, 1, 1, 0},
- {1, 1, 1, 0},
- {1, 1, 1, 0},
- })
- s.checkLastVotes(nids, n.lastVotes, [][]bool{
- {false, false, false, false},
- {false, false, false, false},
- {false, false, false, false},
- {false, false, false, false},
- })
-
- n.processTimestamps(nids[0], genTimestamp(nids, []int{3, 1, 2, 1}))
- s.checkTimeDiff(nids, n.timeDiffs[nids[0]], [][]int{
- {3, 1, 2, 1},
- {1, 1, 1, 0},
- {3, 1, 2, 1},
- {3, 1, 2, 1},
- })
- s.checkLastVotes(nids, n.lastVotes, [][]bool{
- {false, false, false, false},
- {true, false, false, false},
- {false, false, false, false},
- {false, false, false, false},
- })
-
- n.processTimestamps(nids[0], genTimestamp(nids, []int{5, 1, 2, 2}))
- s.checkTimeDiff(nids, n.timeDiffs[nids[0]], [][]int{
- {5, 1, 2, 2},
- {1, 1, 1, 0},
- {3, 1, 2, 1},
- {5, 1, 2, 2},
- })
- s.checkLastVotes(nids, n.lastVotes, [][]bool{
- {false, false, false, false},
- {true, false, false, false},
- {false, false, false, false},
- {false, false, false, false},
- })
-}
-
-func (s *NegativeAckTest) TestRestrictBySelf() {
- var exist bool
- n, nids := genTestNegativeAck(4)
- n.setTimeDelay(timeDelay)
- n.setTimeExpire(timeExpire)
-
- n.processTimestamps(nids[0], genTimestamp(nids, []int{1, 1, 1, 0}))
- _, exist = n.getRestrictedNodes()[nids[1]]
- s.Require().False(exist)
-
- n.processTimestamps(nids[0], genTimestamp(nids, []int{3, 1, 2, 1}))
- _, exist = n.getRestrictedNodes()[nids[1]]
- s.Require().True(exist)
-}
-
-func (s *NegativeAckTest) TestRestrictByVoting() {
- var nackeds []types.NodeID
- var exist bool
-
- n, nids := genTestNegativeAck(4)
- n.setTimeDelay(timeDelay)
- n.setTimeExpire(timeExpire)
-
- n.processTimestamps(nids[0], genTimestamp(nids, []int{1, 1, 1, 1}))
- n.processTimestamps(nids[0], genTimestamp(nids, []int{2, 2, 2, 2}))
-
- n.processTimestamps(nids[1], genTimestamp(nids, []int{1, 1, 1, 1}))
- n.processTimestamps(nids[2], genTimestamp(nids, []int{1, 1, 1, 1}))
- n.processTimestamps(nids[3], genTimestamp(nids, []int{1, 1, 1, 1}))
-
- nackeds = n.processTimestamps(nids[1], genTimestamp(nids, []int{1, 3, 3, 3}))
- _, exist = n.getRestrictedNodes()[nids[0]]
- s.Require().False(exist)
- s.Require().Equal(0, len(nackeds))
-
- nackeds = n.processTimestamps(nids[2], genTimestamp(nids, []int{1, 3, 3, 3}))
- _, exist = n.getRestrictedNodes()[nids[0]]
- s.Require().True(exist)
- s.Require().Equal(0, len(nackeds))
-
- nackeds = n.processTimestamps(nids[3], genTimestamp(nids, []int{1, 3, 3, 3}))
- _, exist = n.getRestrictedNodes()[nids[0]]
- s.Require().False(exist)
- s.Require().Equal(1, len(nackeds))
- s.Require().Equal(nids[0], nackeds[0])
-}
-
-func (s *NegativeAckTest) TestExpire() {
- var exist bool
-
- n, nids := genTestNegativeAck(4)
- n.setTimeDelay(timeDelay)
- n.setTimeExpire(timeExpire)
-
- n.processTimestamps(nids[0], genTimestamp(nids, []int{1, 1, 1, 1}))
- n.processTimestamps(nids[1], genTimestamp(nids, []int{1, 1, 1, 1}))
- n.processTimestamps(nids[2], genTimestamp(nids, []int{1, 1, 1, 1}))
- n.processTimestamps(nids[3], genTimestamp(nids, []int{1, 1, 1, 1}))
-
- n.processTimestamps(nids[1], genTimestamp(nids, []int{1, 3, 3, 3}))
- n.processTimestamps(nids[2], genTimestamp(nids, []int{1, 3, 3, 3}))
- _, exist = n.getRestrictedNodes()[nids[0]]
- s.Require().True(exist)
-
- time.Sleep(2 * timeExpire)
-
- n.processTimestamps(nids[0], genTimestamp(nids, []int{2, 2, 2, 2}))
-
- _, exist = n.getRestrictedNodes()[nids[0]]
- s.Require().False(exist)
-}
-
-func (s *NegativeAckTest) TestAddDeleteNode() {
- n, nids := genTestNegativeAck(10)
- s.Require().Equal(10, len(n.timeDiffs))
- s.Require().Equal(10, len(n.timeDiffs[nids[0]]))
-
- for _, nid := range nids {
- n.deleteNode(nid)
- }
- s.Require().Equal(0, len(n.timeDiffs))
-}
-
-func TestNegativeAck(t *testing.T) {
- suite.Run(t, new(NegativeAckTest))
-}
diff --git a/core/nonblocking.go b/core/nonblocking.go
index 56c42fe..095170b 100644
--- a/core/nonblocking.go
+++ b/core/nonblocking.go
@@ -29,11 +29,6 @@ type blockConfirmedEvent struct {
block *types.Block
}
-type totalOrderingDeliveredEvent struct {
- blockHashes common.Hashes
- mode uint32
-}
-
type blockDeliveredEvent struct {
blockHash common.Hash
blockPosition types.Position
@@ -91,8 +86,6 @@ func (nb *nonBlocking) run() {
switch e := event.(type) {
case blockConfirmedEvent:
nb.app.BlockConfirmed(*e.block)
- case totalOrderingDeliveredEvent:
- nb.debug.TotalOrderingDelivered(e.blockHashes, e.mode)
case blockDeliveredEvent:
nb.app.BlockDelivered(e.blockHash, e.blockPosition, *e.result)
default:
@@ -133,15 +126,6 @@ func (nb *nonBlocking) BlockConfirmed(block types.Block) {
nb.addEvent(blockConfirmedEvent{&block})
}
-// TotalOrderingDelivered is called when the total ordering algorithm deliver
-// a set of block.
-func (nb *nonBlocking) TotalOrderingDelivered(
- blockHashes common.Hashes, mode uint32) {
- if nb.debug != nil {
- nb.addEvent(totalOrderingDeliveredEvent{blockHashes, mode})
- }
-}
-
// BlockDelivered is called when a block is add to the compaction chain.
func (nb *nonBlocking) BlockDelivered(blockHash common.Hash,
blockPosition types.Position, result types.FinalizationResult) {
diff --git a/core/nonblocking_test.go b/core/nonblocking_test.go
index 542382c..110433a 100644
--- a/core/nonblocking_test.go
+++ b/core/nonblocking_test.go
@@ -29,18 +29,16 @@ import (
// slowApp is an Application instance slow things down in every method.
type slowApp struct {
- sleep time.Duration
- blockConfirmed map[common.Hash]struct{}
- totalOrderingDelivered map[common.Hash]struct{}
- blockDelivered map[common.Hash]struct{}
+ sleep time.Duration
+ blockConfirmed map[common.Hash]struct{}
+ blockDelivered map[common.Hash]struct{}
}
func newSlowApp(sleep time.Duration) *slowApp {
return &slowApp{
- sleep: sleep,
- blockConfirmed: make(map[common.Hash]struct{}),
- totalOrderingDelivered: make(map[common.Hash]struct{}),
- blockDelivered: make(map[common.Hash]struct{}),
+ sleep: sleep,
+ blockConfirmed: make(map[common.Hash]struct{}),
+ blockDelivered: make(map[common.Hash]struct{}),
}
}
@@ -61,13 +59,6 @@ func (app *slowApp) BlockConfirmed(block types.Block) {
app.blockConfirmed[block.Hash] = struct{}{}
}
-func (app *slowApp) TotalOrderingDelivered(blockHashes common.Hashes, mode uint32) {
- time.Sleep(app.sleep)
- for _, hash := range blockHashes {
- app.totalOrderingDelivered[hash] = struct{}{}
- }
-}
-
func (app *slowApp) BlockDelivered(blockHash common.Hash,
blockPosition types.Position, _ types.FinalizationResult) {
time.Sleep(app.sleep)
@@ -137,7 +128,6 @@ func (s *NonBlockingTestSuite) TestNonBlocking() {
nbModule.BlockDelivered(
hash, types.Position{}, types.FinalizationResult{})
}
- nbModule.TotalOrderingDelivered(hashes, TotalOrderingModeEarly)
// nonBlocking should be non-blocking.
s.True(shouldFinish.After(time.Now().UTC()))
@@ -145,7 +135,6 @@ func (s *NonBlockingTestSuite) TestNonBlocking() {
nbModule.wait()
for _, hash := range hashes {
s.Contains(app.blockConfirmed, hash)
- s.Contains(app.totalOrderingDelivered, hash)
s.Contains(app.blockDelivered, hash)
}
}
diff --git a/core/syncer/consensus.go b/core/syncer/consensus.go
index 75c1067..618d90e 100644
--- a/core/syncer/consensus.go
+++ b/core/syncer/consensus.go
@@ -63,19 +63,16 @@ type Consensus struct {
nodeSetCache *utils.NodeSetCache
tsigVerifier *core.TSigVerifierCache
- lattice *core.Lattice
- validatedChains map[uint32]struct{}
- finalizedBlockHashes common.Hashes
- latticeLastRound uint64
- randomnessResults map[common.Hash]*types.BlockRandomnessResult
- blocks []types.ByPosition
- agreements []*agreement
- configs []*types.Config
- roundBeginTimes []time.Time
- agreementRoundCut uint64
+ randomnessResults map[common.Hash]*types.BlockRandomnessResult
+ blocks types.BlocksByPosition
+ agreementModule *agreement
+ configs []*types.Config
+ roundBeginTimes []time.Time
+ agreementRoundCut uint64
// lock for accessing all fields.
lock sync.RWMutex
+ duringBuffering bool
moduleWaitGroup sync.WaitGroup
agreementWaitGroup sync.WaitGroup
pullChan chan common.Hash
@@ -100,16 +97,15 @@ func NewConsensus(
logger common.Logger) *Consensus {
con := &Consensus{
- dMoment: dMoment,
- app: app,
- gov: gov,
- db: db,
- network: network,
- nodeSetCache: utils.NewNodeSetCache(gov),
- tsigVerifier: core.NewTSigVerifierCache(gov, 7),
- prv: prv,
- logger: logger,
- validatedChains: make(map[uint32]struct{}),
+ dMoment: dMoment,
+ app: app,
+ gov: gov,
+ db: db,
+ network: network,
+ nodeSetCache: utils.NewNodeSetCache(gov),
+ tsigVerifier: core.NewTSigVerifierCache(gov, 7),
+ prv: prv,
+ logger: logger,
configs: []*types.Config{
utils.GetConfigWithPanic(gov, 0, logger),
},
@@ -119,294 +115,66 @@ func NewConsensus(
randomnessResults: make(map[common.Hash]*types.BlockRandomnessResult),
}
con.ctx, con.ctxCancel = context.WithCancel(context.Background())
+ con.agreementModule = newAgreement(
+ con.receiveChan, con.pullChan, con.nodeSetCache, con.logger)
+ con.agreementWaitGroup.Add(1)
+ go func() {
+ defer con.agreementWaitGroup.Done()
+ con.agreementModule.run()
+ }()
return con
}
-func (con *Consensus) initConsensusObj(initBlock *types.Block) {
- func() {
- con.lock.Lock()
- defer con.lock.Unlock()
- con.latticeLastRound = initBlock.Position.Round
- debugApp, _ := con.app.(core.Debug)
- con.lattice = core.NewLattice(
- con.roundBeginTimes[con.latticeLastRound],
- con.latticeLastRound,
- con.configs[con.latticeLastRound],
- utils.NewSigner(con.prv),
- con.app,
- debugApp,
- con.db,
- con.logger,
- )
- }()
+func (con *Consensus) assureBuffering() {
+ if func() bool {
+ con.lock.RLock()
+ defer con.lock.RUnlock()
+ return con.duringBuffering
+ }() {
+ return
+ }
+ con.lock.Lock()
+ defer con.lock.Unlock()
+ if con.duringBuffering {
+ return
+ }
+ con.duringBuffering = true
con.startAgreement()
con.startNetwork()
con.startCRSMonitor()
}
-func (con *Consensus) checkIfValidated() (validated bool) {
- con.lock.RLock()
- defer con.lock.RUnlock()
- var (
- round = con.blocks[0][0].Position.Round
- numChains = con.configs[round].NumChains
- validatedChainCount uint32
- )
- // Make sure we validate some block in all chains.
- for chainID := range con.validatedChains {
- if chainID < numChains {
- validatedChainCount++
- }
- }
- validated = validatedChainCount == numChains
- con.logger.Debug("syncer chain-validation status",
- "validated-chain", validatedChainCount,
- "round", round,
- "valid", validated)
- return
-}
-
func (con *Consensus) checkIfSynced(blocks []*types.Block) (synced bool) {
con.lock.RLock()
defer con.lock.RUnlock()
- var (
- round = con.blocks[0][0].Position.Round
- numChains = con.configs[round].NumChains
- compactionTips = make([]*types.Block, numChains)
- overlapCount = uint32(0)
- )
defer func() {
con.logger.Debug("syncer synced status",
- "overlap-count", overlapCount,
- "num-chain", numChains,
"last-block", blocks[len(blocks)-1],
- "synced", synced)
+ "synced", synced,
+ )
}()
- // Find tips (newset blocks) of each chain in compaction chain.
- b := blocks[len(blocks)-1]
- for tipCount := uint32(0); tipCount < numChains; {
- if compactionTips[b.Position.ChainID] == nil {
- // Check chainID for config change.
- if b.Position.ChainID < numChains {
- compactionTips[b.Position.ChainID] = b
- tipCount++
- }
- }
- if (b.Finalization.ParentHash == common.Hash{}) {
- return
- }
- b1, err := con.db.GetBlock(b.Finalization.ParentHash)
- if err != nil {
- panic(err)
- }
- b = &b1
- }
- // Check if chain tips of compaction chain and current cached confirmed
- // blocks are overlapped on each chain, numChains is decided by the round
- // of last block we seen on compaction chain.
- for chainID, b := range compactionTips {
- if len(con.blocks[chainID]) > 0 {
- if !b.Position.Older(&con.blocks[chainID][0].Position) {
- overlapCount++
- }
- }
+ if len(con.blocks) == 0 || len(blocks) == 0 {
+ return
}
- synced = overlapCount == numChains
+ synced = !blocks[len(blocks)-1].Position.Older(con.blocks[0].Position)
return
}
-// ensureAgreementOverlapRound ensures the oldest blocks in each chain in
-// con.blocks are all in the same round, for avoiding config change while
-// syncing.
-func (con *Consensus) ensureAgreementOverlapRound() bool {
+func (con *Consensus) buildAllEmptyBlocks() {
con.lock.Lock()
defer con.lock.Unlock()
- defer func() {
- con.logger.Debug("ensureAgreementOverlapRound returned",
- "round", con.agreementRoundCut)
- }()
- if con.agreementRoundCut > 0 {
- return true
- }
// Clean empty blocks on tips of chains.
- for idx, bs := range con.blocks {
- for len(bs) > 0 && con.isEmptyBlock(bs[0]) {
- bs = bs[1:]
- }
- con.blocks[idx] = bs
+ for len(con.blocks) > 0 && con.isEmptyBlock(con.blocks[0]) {
+ con.blocks = con.blocks[1:]
}
// Build empty blocks.
- for _, bs := range con.blocks {
- for i := range bs {
- if con.isEmptyBlock(bs[i]) {
- if bs[i-1].Position.Height == bs[i].Position.Height-1 {
- con.buildEmptyBlock(bs[i], bs[i-1])
- }
+ for i, b := range con.blocks {
+ if con.isEmptyBlock(b) {
+ if con.blocks[i-1].Position.Height+1 == b.Position.Height {
+ con.buildEmptyBlock(b, con.blocks[i-1])
}
}
}
- var tipRoundMap map[uint64]uint32
- for {
- tipRoundMap = make(map[uint64]uint32)
- for _, bs := range con.blocks {
- if len(bs) > 0 {
- tipRoundMap[bs[0].Position.Round]++
- }
- }
- if len(tipRoundMap) <= 1 {
- break
- }
- // Make all tips in same round.
- var maxRound uint64
- for r := range tipRoundMap {
- if r > maxRound {
- maxRound = r
- }
- }
- for idx, bs := range con.blocks {
- for len(bs) > 0 && bs[0].Position.Round < maxRound {
- bs = bs[1:]
- }
- con.blocks[idx] = bs
- }
- }
- if len(tipRoundMap) == 1 {
- var r uint64
- for r = range tipRoundMap {
- break
- }
- con.logger.Debug("check agreement round cut",
- "tip-round", r,
- "configs", len(con.configs))
- if tipRoundMap[r] == con.configs[r].NumChains {
- con.agreementRoundCut = r
- return true
- }
- }
- return false
-}
-
-func (con *Consensus) findLatticeSyncBlock(
- blocks []*types.Block) (*types.Block, error) {
- lastBlock := blocks[len(blocks)-1]
- round := lastBlock.Position.Round
- isConfigChanged := func(prev, cur *types.Config) bool {
- return prev.K != cur.K ||
- prev.NumChains != cur.NumChains ||
- prev.PhiRatio != cur.PhiRatio
- }
- for {
- // Find round r which r-1, r, r+1 are all in same total ordering config.
- for {
- sameAsPrevRound := round == 0 || !isConfigChanged(
- con.configs[round-1], con.configs[round])
- sameAsNextRound := !isConfigChanged(
- con.configs[round], con.configs[round+1])
- if sameAsPrevRound && sameAsNextRound {
- break
- }
- if round == 0 {
- // Unable to find a safe round, wait for new rounds.
- return nil, nil
- }
- round--
- }
- // Find the newset block which round is "round".
- for lastBlock.Position.Round != round {
- if (lastBlock.Finalization.ParentHash == common.Hash{}) {
- return nil, ErrGenesisBlockReached
- }
- b, err := con.db.GetBlock(lastBlock.Finalization.ParentHash)
- if err != nil {
- return nil, err
- }
- lastBlock = &b
- }
- // Find the deliver set by hash for two times. Blocks in a deliver set
- // returned by total ordering is sorted by hash. If a block's parent
- // hash is greater than its hash means there is a cut between deliver
- // sets.
- var curBlock, prevBlock *types.Block
- var deliverSetFirstBlock, deliverSetLastBlock *types.Block
- curBlock = lastBlock
- for {
- if (curBlock.Finalization.ParentHash == common.Hash{}) {
- return nil, ErrGenesisBlockReached
- }
- b, err := con.db.GetBlock(curBlock.Finalization.ParentHash)
- if err != nil {
- return nil, err
- }
- prevBlock = &b
- if !prevBlock.Hash.Less(curBlock.Hash) {
- break
- }
- curBlock = prevBlock
- }
- deliverSetLastBlock = prevBlock
- curBlock = prevBlock
- for {
- if (curBlock.Finalization.ParentHash == common.Hash{}) {
- break
- }
- b, err := con.db.GetBlock(curBlock.Finalization.ParentHash)
- if err != nil {
- return nil, err
- }
- prevBlock = &b
- if !prevBlock.Hash.Less(curBlock.Hash) {
- break
- }
- curBlock = prevBlock
- }
- deliverSetFirstBlock = curBlock
- // Check if all blocks from deliverSetFirstBlock to deliverSetLastBlock
- // are in the same round.
- ok := true
- curBlock = deliverSetLastBlock
- for {
- if curBlock.Position.Round != round {
- ok = false
- break
- }
- b, err := con.db.GetBlock(curBlock.Finalization.ParentHash)
- if err != nil {
- return nil, err
- }
- curBlock = &b
- if curBlock.Hash == deliverSetFirstBlock.Hash {
- break
- }
- }
- if ok {
- return deliverSetFirstBlock, nil
- }
- if round == 0 {
- return nil, nil
- }
- round--
- }
-}
-
-func (con *Consensus) processFinalizedBlock(block *types.Block) error {
- if con.lattice == nil {
- return nil
- }
- delivered, err := con.lattice.ProcessFinalizedBlock(block)
- if err != nil {
- return err
- }
- con.lock.Lock()
- defer con.lock.Unlock()
- con.finalizedBlockHashes = append(con.finalizedBlockHashes, block.Hash)
- for idx, b := range delivered {
- if con.finalizedBlockHashes[idx] != b.Hash {
- return ErrMismatchBlockHashSequence
- }
- con.validatedChains[b.Position.ChainID] = struct{}{}
- }
- con.finalizedBlockHashes = con.finalizedBlockHashes[len(delivered):]
- return nil
}
// SyncBlocks syncs blocks from compaction chain, latest is true if the caller
@@ -420,7 +188,8 @@ func (con *Consensus) SyncBlocks(
con.logger.Debug("SyncBlocks returned",
"synced", synced,
"error", err,
- "last-block", con.syncedLastBlock)
+ "last-block", con.syncedLastBlock,
+ )
}()
if con.syncedLastBlock != nil {
synced, err = true, ErrAlreadySynced
@@ -442,7 +211,8 @@ func (con *Consensus) SyncBlocks(
if blocks[0].Finalization.Height != tipHeight+1 {
con.logger.Error("mismatched finalization height",
"now", blocks[0].Finalization.Height,
- "expected", tipHeight+1)
+ "expected", tipHeight+1,
+ )
err = ErrInvalidSyncingFinalizationHeight
return
}
@@ -454,7 +224,6 @@ func (con *Consensus) SyncBlocks(
)
con.setupConfigs(blocks)
for _, b := range blocks {
- // TODO(haoping) remove this if lattice puts blocks into db.
if err = con.db.PutBlock(*b); err != nil {
// A block might be put into db when confirmed by BA, but not
// finalized yet.
@@ -469,60 +238,15 @@ func (con *Consensus) SyncBlocks(
b.Hash, b.Finalization.Height); err != nil {
return
}
- if err = con.processFinalizedBlock(b); err != nil {
- return
- }
- }
- if latest && con.lattice == nil {
- // New Lattice and find the deliver set of total ordering when "latest"
- // is true for first time. Deliver set is found by block hashes.
- var syncBlock *types.Block
- syncBlock, err = con.findLatticeSyncBlock(blocks)
- if err != nil {
- if err == ErrGenesisBlockReached {
- con.logger.Debug("SyncBlocks skip error", "error", err)
- err = nil
- }
- return
- }
- if syncBlock != nil {
- con.logger.Debug("deliver set found", "block", syncBlock)
- // New lattice with the round of syncBlock.
- con.initConsensusObj(syncBlock)
- con.setupConfigs(blocks)
- // Process blocks from syncBlock to blocks' last block.
- b := blocks[len(blocks)-1]
- blocksCount :=
- b.Finalization.Height - syncBlock.Finalization.Height + 1
- blocksToProcess := make([]*types.Block, blocksCount)
- for {
- blocksToProcess[blocksCount-1] = b
- blocksCount--
- if b.Hash == syncBlock.Hash {
- break
- }
- var b1 types.Block
- b1, err = con.db.GetBlock(b.Finalization.ParentHash)
- if err != nil {
- return
- }
- b = &b1
- }
- for _, b := range blocksToProcess {
- if err = con.processFinalizedBlock(b); err != nil {
- return
- }
- }
- }
}
- if latest && con.ensureAgreementOverlapRound() {
+ if latest {
+ con.assureBuffering()
+ con.buildAllEmptyBlocks()
// Check if compaction and agreements' blocks are overlapped. The
// overlapping of compaction chain and BA's oldest blocks means the
// syncing is done.
- if con.checkIfValidated() && con.checkIfSynced(blocks) {
- if err = con.Stop(); err != nil {
- return
- }
+ if con.checkIfSynced(blocks) {
+ con.stopBuffering()
con.dummyCancel, con.dummyFinished = utils.LaunchDummyReceiver(
context.Background(), con.network.ReceiveChan(),
func(msg interface{}) {
@@ -547,10 +271,6 @@ func (con *Consensus) GetSyncedConsensus() (*core.Consensus, error) {
}
// flush all blocks in con.blocks into core.Consensus, and build
// core.Consensus from syncer.
- confirmedBlocks := make([][]*types.Block, len(con.blocks))
- for i, bs := range con.blocks {
- confirmedBlocks[i] = []*types.Block(bs)
- }
randomnessResults := []*types.BlockRandomnessResult{}
for _, r := range con.randomnessResults {
randomnessResults = append(randomnessResults, r)
@@ -566,19 +286,31 @@ func (con *Consensus) GetSyncedConsensus() (*core.Consensus, error) {
con.db,
con.network,
con.prv,
- con.lattice,
- confirmedBlocks,
+ con.blocks,
randomnessResults,
con.dummyMsgBuffer,
con.logger)
return con.syncedConsensus, err
}
-// Stop the syncer.
+// stopBuffering stops the syncer buffering routines.
//
// This method is mainly for caller to stop the syncer before synced, the syncer
// would call this method automatically after being synced.
-func (con *Consensus) Stop() error {
+func (con *Consensus) stopBuffering() {
+ if func() bool {
+ con.lock.RLock()
+ defer con.lock.RUnlock()
+ return !con.duringBuffering
+ }() {
+ return
+ }
+ con.lock.Lock()
+ defer con.lock.Unlock()
+ if !con.duringBuffering {
+ return
+ }
+ con.duringBuffering = false
con.logger.Trace("syncer is about to stop")
// Stop network and CRS routines, wait until they are all stoped.
con.ctxCancel()
@@ -588,7 +320,7 @@ func (con *Consensus) Stop() error {
con.logger.Trace("stop syncer agreement modules")
con.stopAgreement()
con.logger.Trace("syncer stopped")
- return nil
+ return
}
// isEmptyBlock checks if a block is an empty block by both its hash and parent
@@ -607,41 +339,6 @@ func (con *Consensus) buildEmptyBlock(b *types.Block, parent *types.Block) {
b.Acks = common.NewSortedHashes(common.Hashes{parent.Hash})
}
-func (con *Consensus) setupConfigsUntilRound(round uint64) {
- curMaxNumChains := uint32(0)
- func() {
- con.lock.Lock()
- defer con.lock.Unlock()
- con.logger.Debug("syncer setupConfigs",
- "until-round", round,
- "length", len(con.configs),
- "lattice", con.latticeLastRound)
- for r := uint64(len(con.configs)); r <= round; r++ {
- cfg := utils.GetConfigWithPanic(con.gov, r, con.logger)
- con.configs = append(con.configs, cfg)
- con.roundBeginTimes = append(
- con.roundBeginTimes,
- con.roundBeginTimes[r-1].Add(con.configs[r-1].RoundInterval))
- if cfg.NumChains >= curMaxNumChains {
- curMaxNumChains = cfg.NumChains
- }
- }
- // Notify core.Lattice for new configs.
- if con.lattice != nil {
- for con.latticeLastRound+1 <= round {
- con.latticeLastRound++
- if err := con.lattice.AppendConfig(
- con.latticeLastRound,
- con.configs[con.latticeLastRound]); err != nil {
- panic(err)
- }
- }
- }
- }()
- con.resizeByNumChains(curMaxNumChains)
- con.logger.Trace("setupConfgis finished", "round", round)
-}
-
// setupConfigs is called by SyncBlocks with blocks from compaction chain. In
// the first time, setupConfigs setups from round 0.
func (con *Consensus) setupConfigs(blocks []*types.Block) {
@@ -661,25 +358,19 @@ func (con *Consensus) setupConfigs(blocks []*types.Block) {
con.setupConfigsUntilRound(maxRound + core.ConfigRoundShift - 1)
}
-// resizeByNumChains resizes fake lattice and agreement if numChains increases.
-// Notice the decreasing case is neglected.
-func (con *Consensus) resizeByNumChains(numChains uint32) {
+func (con *Consensus) setupConfigsUntilRound(round uint64) {
con.lock.Lock()
defer con.lock.Unlock()
- if numChains > uint32(len(con.blocks)) {
- for i := uint32(len(con.blocks)); i < numChains; i++ {
- // Resize the pool of blocks.
- con.blocks = append(con.blocks, types.ByPosition{})
- // Resize agreement modules.
- a := newAgreement(
- con.receiveChan, con.pullChan, con.nodeSetCache, con.logger)
- con.agreements = append(con.agreements, a)
- con.agreementWaitGroup.Add(1)
- go func() {
- defer con.agreementWaitGroup.Done()
- a.run()
- }()
- }
+ con.logger.Debug("syncer setupConfigs",
+ "until-round", round,
+ "length", len(con.configs),
+ )
+ for r := uint64(len(con.configs)); r <= round; r++ {
+ cfg := utils.GetConfigWithPanic(con.gov, r, con.logger)
+ con.configs = append(con.configs, cfg)
+ con.roundBeginTimes = append(
+ con.roundBeginTimes,
+ con.roundBeginTimes[r-1].Add(con.configs[r-1].RoundInterval))
}
}
@@ -693,17 +384,15 @@ func (con *Consensus) startAgreement() {
if !ok {
return
}
- chainID := b.Position.ChainID
func() {
con.lock.Lock()
defer con.lock.Unlock()
- // If round is cut in agreements, do not add blocks with
- // round less then cut round.
- if b.Position.Round < con.agreementRoundCut {
+ if len(con.blocks) > 0 &&
+ !b.Position.Newer(con.blocks[0].Position) {
return
}
- con.blocks[chainID] = append(con.blocks[chainID], b)
- sort.Sort(con.blocks[chainID])
+ con.blocks = append(con.blocks, b)
+ sort.Sort(con.blocks)
}()
case h, ok := <-con.pullChan:
if !ok {
@@ -721,18 +410,14 @@ func (con *Consensus) cacheRandomnessResult(r *types.BlockRandomnessResult) {
return
}
// We only have to cache randomness result after cutting round.
- if r.Position.Round < func() uint64 {
- con.lock.RLock()
- defer con.lock.RUnlock()
- return con.agreementRoundCut
- }() {
- return
- }
- if func() (exists bool) {
+ if func() bool {
con.lock.RLock()
defer con.lock.RUnlock()
- _, exists = con.randomnessResults[r.BlockHash]
- return
+ if len(con.blocks) > 0 && r.Position.Older(con.blocks[0].Position) {
+ return true
+ }
+ _, exists := con.randomnessResults[r.BlockHash]
+ return exists
}() {
return
}
@@ -740,8 +425,9 @@ func (con *Consensus) cacheRandomnessResult(r *types.BlockRandomnessResult) {
if err != nil {
con.logger.Error("Unable to get tsig verifier",
"hash", r.BlockHash.String()[:6],
- "position", &r.Position,
- "error", err)
+ "position", r.Position,
+ "error", err,
+ )
return
}
if !ok {
@@ -752,8 +438,9 @@ func (con *Consensus) cacheRandomnessResult(r *types.BlockRandomnessResult) {
Type: "bls",
Signature: r.Randomness}) {
con.logger.Info("Block randomness is not valid",
- "position", &r.Position,
- "hash", r.BlockHash.String()[:6])
+ "position", r.Position,
+ "hash", r.BlockHash.String()[:6],
+ )
return
}
con.lock.Lock()
@@ -785,18 +472,19 @@ func (con *Consensus) startNetwork() {
if func() bool {
con.lock.RLock()
defer con.lock.RUnlock()
- if pos.ChainID >= uint32(len(con.agreements)) {
+ if pos.ChainID > 0 {
// This error might be easily encountered when the
// "latest" parameter of SyncBlocks is turned on too
// early.
con.logger.Error(
"Unknown chainID message received (syncer)",
- "position", &pos)
+ "position", pos,
+ )
return false
}
return true
}() {
- con.agreements[pos.ChainID].inputChan <- val
+ con.agreementModule.inputChan <- val
}
case <-con.ctx.Done():
return
@@ -817,23 +505,25 @@ func (con *Consensus) startCRSMonitor() {
}
con.logger.Debug("CRS is ready", "round", round)
lastNotifiedRound = round
- con.lock.Lock()
- defer con.lock.Unlock()
- for idx, a := range con.agreements {
- loop:
- for {
- select {
- case <-con.ctx.Done():
- break loop
- case a.inputChan <- round:
- break loop
- case <-time.After(500 * time.Millisecond):
- con.logger.Debug(
- "agreement input channel is full when putting CRS",
- "chainID", idx,
- "round", round)
- }
+ for func() bool {
+ con.lock.RLock()
+ defer con.lock.RUnlock()
+ if !con.duringBuffering {
+ return false
+ }
+ select {
+ case <-con.ctx.Done():
+ return false
+ case con.agreementModule.inputChan <- round:
+ return false
+ case <-time.After(500 * time.Millisecond):
+ con.logger.Debug(
+ "agreement input channel is full when putting CRS",
+ "round", round,
+ )
+ return true
}
+ }() {
}
}
con.moduleWaitGroup.Add(1)
@@ -860,16 +550,10 @@ func (con *Consensus) startCRSMonitor() {
}
func (con *Consensus) stopAgreement() {
- func() {
- con.lock.Lock()
- defer con.lock.Unlock()
- for _, a := range con.agreements {
- if a.inputChan != nil {
- close(a.inputChan)
- a.inputChan = nil
- }
- }
- }()
+ if con.agreementModule.inputChan != nil {
+ close(con.agreementModule.inputChan)
+ con.agreementModule.inputChan = nil
+ }
con.agreementWaitGroup.Wait()
close(con.receiveChan)
close(con.pullChan)
diff --git a/core/test/app.go b/core/test/app.go
index 515ed23..1ce5b84 100644
--- a/core/test/app.go
+++ b/core/test/app.go
@@ -50,10 +50,6 @@ var (
// ErrDeliveredBlockNotConfirmed means some block delivered (confirmed) but
// not confirmed.
ErrDeliveredBlockNotConfirmed = fmt.Errorf("delivered block not confirmed")
- // ErrMismatchTotalOrderingAndDelivered mean the sequence of total ordering
- // and delivered are different.
- ErrMismatchTotalOrderingAndDelivered = fmt.Errorf(
- "mismatch total ordering and delivered sequence")
// ErrAckingBlockNotDelivered means the delivered sequence not forming a
// DAG.
ErrAckingBlockNotDelivered = fmt.Errorf("acking block not delivered")
@@ -70,26 +66,6 @@ var (
ErrParentBlockNotDelivered = fmt.Errorf("parent block not delivered")
)
-// This definition is copied from core package.
-const (
- // TotalOrderingModeError returns mode error.
- TotalOrderingModeError uint32 = iota
- // TotalOrderingModeNormal returns mode normal.
- TotalOrderingModeNormal
- // TotalOrderingModeEarly returns mode early.
- TotalOrderingModeEarly
- // TotalOrderingModeFlush returns mode flush.
- TotalOrderingModeFlush
-)
-
-// AppTotalOrderRecord caches information when this application received
-// a total-ordering deliver notification.
-type AppTotalOrderRecord struct {
- BlockHashes common.Hashes
- Mode uint32
- When time.Time
-}
-
// AppDeliveredRecord caches information when this application received
// a block delivered notification.
type AppDeliveredRecord struct {
@@ -103,9 +79,6 @@ type App struct {
Confirmed map[common.Hash]*types.Block
LastConfirmedHeights map[uint32]uint64
confirmedLock sync.RWMutex
- TotalOrdered []*AppTotalOrderRecord
- TotalOrderedByHash map[common.Hash]*AppTotalOrderRecord
- totalOrderedLock sync.RWMutex
Delivered map[common.Hash]*AppDeliveredRecord
DeliverSequence common.Hashes
deliveredLock sync.RWMutex
@@ -121,8 +94,6 @@ func NewApp(initRound uint64, gov *Governance) (app *App) {
app = &App{
Confirmed: make(map[common.Hash]*types.Block),
LastConfirmedHeights: make(map[uint32]uint64),
- TotalOrdered: []*AppTotalOrderRecord{},
- TotalOrderedByHash: make(map[common.Hash]*AppTotalOrderRecord),
Delivered: make(map[common.Hash]*AppDeliveredRecord),
DeliverSequence: common.Hashes{},
gov: gov,
@@ -223,25 +194,6 @@ func (app *App) BlockConfirmed(b types.Block) {
app.LastConfirmedHeights[b.Position.ChainID] = b.Position.Height
}
-// TotalOrderingDelivered implements Application interface.
-func (app *App) TotalOrderingDelivered(blockHashes common.Hashes, mode uint32) {
- app.totalOrderedLock.Lock()
- defer app.totalOrderedLock.Unlock()
-
- rec := &AppTotalOrderRecord{
- BlockHashes: blockHashes,
- Mode: mode,
- When: time.Now().UTC(),
- }
- app.TotalOrdered = append(app.TotalOrdered, rec)
- for _, h := range blockHashes {
- if _, exists := app.TotalOrderedByHash[h]; exists {
- panic(fmt.Errorf("deliver duplicated blocks from total ordering"))
- }
- app.TotalOrderedByHash[h] = rec
- }
-}
-
// BlockDelivered implements Application interface.
func (app *App) BlockDelivered(
blockHash common.Hash, pos types.Position, result types.FinalizationResult) {
@@ -307,30 +259,33 @@ func (app *App) GetLatestDeliveredPosition() types.Position {
// and return erros if not passed:
// - deliver sequence by comparing block hashes.
// - consensus timestamp of each block are equal.
-func (app *App) Compare(other *App) error {
- app.deliveredLock.RLock()
- defer app.deliveredLock.RUnlock()
- other.deliveredLock.RLock()
- defer other.deliveredLock.RUnlock()
-
- minLength := len(app.DeliverSequence)
- if minLength > len(other.DeliverSequence) {
- minLength = len(other.DeliverSequence)
- }
- if minLength == 0 {
- return ErrEmptyDeliverSequence
- }
- for idx, h := range app.DeliverSequence[:minLength] {
- hOther := other.DeliverSequence[idx]
- if hOther != h {
- return ErrMismatchBlockHashSequence
- }
- if app.Delivered[h].Result.Timestamp !=
- other.Delivered[h].Result.Timestamp {
- return ErrMismatchConsensusTime
- }
- }
- return nil
+func (app *App) Compare(other *App) (err error) {
+ app.WithLock(func(app *App) {
+ other.WithLock(func(other *App) {
+ minLength := len(app.DeliverSequence)
+ if minLength > len(other.DeliverSequence) {
+ minLength = len(other.DeliverSequence)
+ }
+ if minLength == 0 {
+ err = ErrEmptyDeliverSequence
+ return
+ }
+ // Here we assumes both Apps begin from the same height.
+ for idx, h := range app.DeliverSequence[:minLength] {
+ hOther := other.DeliverSequence[idx]
+ if hOther != h {
+ err = ErrMismatchBlockHashSequence
+ return
+ }
+ if app.Delivered[h].Result.Timestamp !=
+ other.Delivered[h].Result.Timestamp {
+ err = ErrMismatchConsensusTime
+ return
+ }
+ }
+ })
+ })
+ return
}
// Verify checks the integrity of date received by this App instance.
@@ -371,57 +326,6 @@ func (app *App) Verify() error {
}
expectHeight++
}
- // Check causality.
- revealedDAG := make(map[common.Hash]struct{})
- for _, toDeliver := range app.TotalOrdered {
- for _, h := range toDeliver.BlockHashes {
- b, exists := app.Confirmed[h]
- if !exists {
- return ErrDeliveredBlockNotConfirmed
- }
- for _, ack := range b.Acks {
- if _, ackingBlockExists := revealedDAG[ack]; !ackingBlockExists {
- return ErrAckingBlockNotDelivered
- }
- }
- if toDeliver.Mode == TotalOrderingModeFlush {
- // For blocks delivered by flushing, the acking relations would
- // exist in one deliver set, however, only later block would
- // ack previous block, not backward.
- revealedDAG[h] = struct{}{}
- }
- }
- // For blocks not delivered by flushing, the acking relations only exist
- // between deliver sets.
- if toDeliver.Mode != TotalOrderingModeFlush {
- for _, h := range toDeliver.BlockHashes {
- revealedDAG[h] = struct{}{}
- }
- }
- }
- // Make sure the order of delivered and total ordering are the same by
- // comparing the concated string.
- app.totalOrderedLock.RLock()
- defer app.totalOrderedLock.RUnlock()
-
- hashSequenceIdx := 0
-Loop:
- for _, rec := range app.TotalOrdered {
- for _, h := range rec.BlockHashes {
- if hashSequenceIdx >= len(app.DeliverSequence) {
- break Loop
- }
- if h != app.DeliverSequence[hashSequenceIdx] {
- return ErrMismatchTotalOrderingAndDelivered
- }
- hashSequenceIdx++
- }
- }
- if hashSequenceIdx != len(app.DeliverSequence) {
- // The count of delivered blocks should be larger than those delivered
- // by total ordering.
- return ErrMismatchTotalOrderingAndDelivered
- }
return nil
}
@@ -435,8 +339,6 @@ func (app *App) BlockReady(hash common.Hash) {}
func (app *App) WithLock(function func(*App)) {
app.confirmedLock.RLock()
defer app.confirmedLock.RUnlock()
- app.totalOrderedLock.RLock()
- defer app.totalOrderedLock.RUnlock()
app.deliveredLock.RLock()
defer app.deliveredLock.RUnlock()
app.lastPendingHeightLock.RLock()
diff --git a/core/test/app_test.go b/core/test/app_test.go
index 5ed562a..79518ea 100644
--- a/core/test/app_test.go
+++ b/core/test/app_test.go
@@ -23,52 +23,12 @@ import (
"time"
"github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core"
"github.com/dexon-foundation/dexon-consensus/core/types"
"github.com/stretchr/testify/suite"
)
type AppTestSuite struct {
suite.Suite
-
- to1, to2, to3 *AppTotalOrderRecord
-}
-
-func (s *AppTestSuite) SetupSuite() {
- s.to1 = &AppTotalOrderRecord{
- BlockHashes: common.Hashes{
- common.NewRandomHash(),
- common.NewRandomHash(),
- },
- Mode: core.TotalOrderingModeNormal,
- }
- s.to2 = &AppTotalOrderRecord{
- BlockHashes: common.Hashes{
- common.NewRandomHash(),
- common.NewRandomHash(),
- common.NewRandomHash(),
- },
- Mode: core.TotalOrderingModeNormal,
- }
- s.to3 = &AppTotalOrderRecord{
- BlockHashes: common.Hashes{
- common.NewRandomHash(),
- },
- Mode: core.TotalOrderingModeNormal,
- }
-}
-
-func (s *AppTestSuite) setupAppByTotalOrderDeliver(
- app *App, to *AppTotalOrderRecord) {
- for _, h := range to.BlockHashes {
- app.BlockConfirmed(types.Block{Hash: h})
- }
- app.TotalOrderingDelivered(to.BlockHashes, to.Mode)
- for _, h := range to.BlockHashes {
- // To make it simpler, use the index of hash sequence
- // as the time.
- s.deliverBlockWithTimeFromSequenceLength(app, h)
- }
}
func (s *AppTestSuite) deliverBlockWithTimeFromSequenceLength(
@@ -89,112 +49,94 @@ func (s *AppTestSuite) deliverBlock(
}
func (s *AppTestSuite) TestCompare() {
- req := s.Require()
-
+ var (
+ now = time.Now().UTC()
+ b0 = types.Block{Hash: common.Hash{}}
+ b1 = types.Block{
+ Hash: common.NewRandomHash(),
+ Position: types.Position{Height: 1},
+ }
+ )
+ // Prepare an OK App instance.
app1 := NewApp(0, nil)
- s.setupAppByTotalOrderDeliver(app1, s.to1)
- s.setupAppByTotalOrderDeliver(app1, s.to2)
- s.setupAppByTotalOrderDeliver(app1, s.to3)
- // An App with different deliver sequence.
+ app1.BlockConfirmed(b0)
+ app1.BlockConfirmed(b1)
+ app1.BlockDelivered(b0.Hash, b0.Position, types.FinalizationResult{
+ Height: 1,
+ Timestamp: now,
+ })
+ app1.BlockDelivered(b1.Hash, b1.Position, types.FinalizationResult{
+ Height: 2,
+ Timestamp: now.Add(1 * time.Second),
+ })
app2 := NewApp(0, nil)
- s.setupAppByTotalOrderDeliver(app2, s.to1)
- s.setupAppByTotalOrderDeliver(app2, s.to2)
- hash := common.NewRandomHash()
- app2.BlockConfirmed(types.Block{Hash: hash})
- app2.TotalOrderingDelivered(common.Hashes{hash}, core.TotalOrderingModeNormal)
- s.deliverBlockWithTimeFromSequenceLength(app2, hash)
- req.Equal(ErrMismatchBlockHashSequence, app1.Compare(app2))
- // An App with different consensus time for the same block.
- app3 := NewApp(0, nil)
- s.setupAppByTotalOrderDeliver(app3, s.to1)
- s.setupAppByTotalOrderDeliver(app3, s.to2)
- for _, h := range s.to3.BlockHashes {
- app3.BlockConfirmed(types.Block{Hash: h})
+ s.Require().Equal(ErrEmptyDeliverSequence.Error(),
+ app1.Compare(app2).Error())
+ app2.BlockConfirmed(b0)
+ app2.BlockDelivered(b0.Hash, b0.Position, types.FinalizationResult{
+ Height: 1,
+ Timestamp: now,
+ })
+ b1Bad := types.Block{
+ Hash: common.NewRandomHash(),
+ Position: types.Position{Height: 1},
}
- app3.TotalOrderingDelivered(s.to3.BlockHashes, s.to3.Mode)
- wrongTime := time.Time{}.Add(
- time.Duration(len(app3.DeliverSequence)) * time.Second)
- wrongTime = wrongTime.Add(1 * time.Second)
- s.deliverBlock(app3, s.to3.BlockHashes[0], wrongTime,
- uint64(len(app3.DeliverSequence)+1))
- req.Equal(ErrMismatchConsensusTime, app1.Compare(app3))
- req.Equal(ErrMismatchConsensusTime, app3.Compare(app1))
- // An App without any delivered blocks.
- app4 := NewApp(0, nil)
- req.Equal(ErrEmptyDeliverSequence, app4.Compare(app1))
- req.Equal(ErrEmptyDeliverSequence, app1.Compare(app4))
+ app2.BlockConfirmed(b1Bad)
+ app2.BlockDelivered(b1Bad.Hash, b1Bad.Position, types.FinalizationResult{
+ Height: 1,
+ Timestamp: now,
+ })
+ s.Require().Equal(ErrMismatchBlockHashSequence.Error(),
+ app1.Compare(app2).Error())
+ app2 = NewApp(0, nil)
+ app2.BlockConfirmed(b0)
+ app2.BlockDelivered(b0.Hash, b0.Position, types.FinalizationResult{
+ Height: 1,
+ Timestamp: now.Add(1 * time.Second),
+ })
+ s.Require().Equal(ErrMismatchConsensusTime.Error(),
+ app1.Compare(app2).Error())
}
func (s *AppTestSuite) TestVerify() {
- req := s.Require()
-
- // An OK App instance.
- app1 := NewApp(0, nil)
- s.setupAppByTotalOrderDeliver(app1, s.to1)
- s.setupAppByTotalOrderDeliver(app1, s.to2)
- s.setupAppByTotalOrderDeliver(app1, s.to3)
- req.NoError(app1.Verify())
- // A delivered block without strongly ack
- s.deliverBlock(app1, common.NewRandomHash(), time.Time{},
- uint64(len(app1.DeliverSequence)))
- req.Equal(ErrDeliveredBlockNotConfirmed, app1.Verify())
- // The consensus time is out of order.
- app2 := NewApp(0, nil)
- s.setupAppByTotalOrderDeliver(app2, s.to1)
- for _, h := range s.to2.BlockHashes {
- app2.BlockConfirmed(types.Block{Hash: h})
- }
- app2.TotalOrderingDelivered(s.to2.BlockHashes, s.to2.Mode)
- s.deliverBlock(app2, s.to2.BlockHashes[0], time.Time{},
- uint64(len(app2.DeliverSequence)+1))
- req.Equal(ErrConsensusTimestampOutOfOrder, app2.Verify())
- // A delivered block is not found in total ordering delivers.
- app3 := NewApp(0, nil)
- s.setupAppByTotalOrderDeliver(app3, s.to1)
- hash := common.NewRandomHash()
- app3.BlockConfirmed(types.Block{Hash: hash})
- s.deliverBlockWithTimeFromSequenceLength(app3, hash)
- req.Equal(ErrMismatchTotalOrderingAndDelivered, app3.Verify())
- // A delivered block is not found in total ordering delivers.
- app4 := NewApp(0, nil)
- s.setupAppByTotalOrderDeliver(app4, s.to1)
- for _, h := range s.to2.BlockHashes {
- app4.BlockConfirmed(types.Block{Hash: h})
- }
- app4.TotalOrderingDelivered(s.to2.BlockHashes, s.to2.Mode)
- hash = common.NewRandomHash()
- app4.BlockConfirmed(types.Block{Hash: hash})
- app4.TotalOrderingDelivered(common.Hashes{hash}, core.TotalOrderingModeNormal)
- s.deliverBlockWithTimeFromSequenceLength(app4, hash)
- // Witness ack on unknown block.
- app5 := NewApp(0, nil)
- s.setupAppByTotalOrderDeliver(app5, s.to1)
- // The conensus height is out of order.
- app6 := NewApp(0, nil)
- s.setupAppByTotalOrderDeliver(app6, s.to1)
- for _, h := range s.to2.BlockHashes {
- app6.BlockConfirmed(types.Block{Hash: h})
- }
- app6.TotalOrderingDelivered(s.to2.BlockHashes, s.to2.Mode)
- s.deliverBlock(app6, s.to2.BlockHashes[0], time.Time{}.Add(
- time.Duration(len(app6.DeliverSequence))*time.Second),
- uint64(len(app6.DeliverSequence)+2))
- req.Equal(ErrConsensusHeightOutOfOrder, app6.Verify())
- // Test the acking block doesn't delivered.
- app7 := NewApp(0, nil)
- // Patch a block's acks.
- b7 := &types.Block{
- Hash: common.NewRandomHash(),
- Acks: common.NewSortedHashes(common.Hashes{common.NewRandomHash()}),
- }
- app7.BlockConfirmed(*b7)
- app7.TotalOrderingDelivered(
- common.Hashes{b7.Hash}, core.TotalOrderingModeNormal)
- app7.BlockDelivered(b7.Hash, types.Position{}, types.FinalizationResult{
- Timestamp: time.Now(),
+ var (
+ now = time.Now().UTC()
+ b0 = types.Block{Hash: common.Hash{}}
+ b1 = types.Block{
+ Hash: common.NewRandomHash(),
+ Position: types.Position{Height: 1},
+ }
+ )
+ app := NewApp(0, nil)
+ s.Require().Equal(ErrEmptyDeliverSequence.Error(), app.Verify().Error())
+ app.BlockDelivered(b0.Hash, b0.Position, types.FinalizationResult{})
+ app.BlockDelivered(b1.Hash, b1.Position, types.FinalizationResult{Height: 1})
+ s.Require().Equal(
+ ErrDeliveredBlockNotConfirmed.Error(), app.Verify().Error())
+ app = NewApp(0, nil)
+ app.BlockConfirmed(b0)
+ app.BlockDelivered(b0.Hash, b0.Position, types.FinalizationResult{
+ Height: 1,
+ Timestamp: now,
+ })
+ app.BlockConfirmed(b1)
+ app.BlockDelivered(b1.Hash, b1.Position, types.FinalizationResult{
+ Height: 2,
+ Timestamp: now.Add(-1 * time.Second),
+ })
+ s.Require().Equal(ErrConsensusTimestampOutOfOrder.Error(),
+ app.Verify().Error())
+ app = NewApp(0, nil)
+ app.BlockConfirmed(b0)
+ app.BlockConfirmed(b1)
+ app.BlockDelivered(b0.Hash, b0.Position, types.FinalizationResult{
+ Height: 1,
+ Timestamp: now,
+ })
+ app.BlockDelivered(b1.Hash, b1.Position, types.FinalizationResult{
Height: 1,
+ Timestamp: now.Add(1 * time.Second),
})
- req.Equal(ErrAckingBlockNotDelivered, app7.Verify())
}
func (s *AppTestSuite) TestWitness() {
@@ -255,7 +197,7 @@ func (s *AppTestSuite) TestWitness() {
s.Require().Equal(app.LastPendingHeight, uint64(3))
// We can only prepare witness for what've delivered.
_, err := app.PrepareWitness(4)
- s.Require().IsType(err, ErrLowerPendingHeight)
+ s.Require().Equal(err.Error(), ErrLowerPendingHeight.Error())
// It should be ok to prepare for height that already delivered.
w, err := app.PrepareWitness(3)
s.Require().NoError(err)
diff --git a/core/test/block-revealer.go b/core/test/block-revealer.go
index ebd2e35..90b3d3e 100644
--- a/core/test/block-revealer.go
+++ b/core/test/block-revealer.go
@@ -19,9 +19,7 @@ package test
import (
"errors"
- "math/rand"
"sort"
- "time"
"github.com/dexon-foundation/dexon-consensus/common"
"github.com/dexon-foundation/dexon-consensus/core/db"
@@ -65,217 +63,11 @@ func loadAllBlocks(iter db.BlockIterator) (
return
}
-// RandomDAGBlockRevealer implements BlockRevealer interface, which would load
-// all blocks from db, and randomly pick one block to reveal if it still forms
-// a valid DAG in revealed blocks.
-type RandomDAGBlockRevealer struct {
- // blocksByChain group all blocks by chains and sorting
- // them by height.
- blocksByChain map[uint32][]*types.Block
- // tipIndexes store the height of next block from one chain
- // to check if is candidate.
- tipIndexes map[uint32]int
- // candidate are blocks that forms valid DAG with
- // current revealed blocks.
- candidates []*types.Block
- candidateChains map[uint32]struct{}
- // revealed stores block hashes of current revealed blocks.
- revealed map[common.Hash]struct{}
- randGen *rand.Rand
-}
-
-// NewRandomDAGBlockRevealer constructs RandomDAGBlockRevealer.
-func NewRandomDAGBlockRevealer(
- iter db.BlockIterator) (r *RandomDAGBlockRevealer, err error) {
-
- blocks, err := loadAllBlocks(iter)
- if err != nil {
- return
- }
-
- // Rearrange blocks by nodes and height.
- blocksByChain := make(map[uint32][]*types.Block)
- for _, block := range blocks {
- blocksByChain[block.Position.ChainID] =
- append(blocksByChain[block.Position.ChainID], block)
- }
- // Make sure blocks are sorted by block heights, from lower to higher.
- for chainID := range blocksByChain {
- sort.Sort(types.ByPosition(blocksByChain[chainID]))
- }
- r = &RandomDAGBlockRevealer{
- blocksByChain: blocksByChain,
- randGen: rand.New(rand.NewSource(time.Now().UnixNano())),
- candidateChains: make(map[uint32]struct{}),
- }
- // Make sure this revealer is ready to use.
- r.Reset()
- return
-}
-
-// pickCandidates is a helper function to pick candidates from current tips.
-func (r *RandomDAGBlockRevealer) pickCandidates() {
- for chainID, tip := range r.tipIndexes {
- if _, isPicked := r.candidateChains[chainID]; isPicked {
- continue
- }
- blocks, exists := r.blocksByChain[chainID]
- if !exists {
- continue
- }
- if tip >= len(blocks) {
- continue
- }
- block := blocks[tip]
- if isAllAckingBlockRevealed(block, r.revealed) {
- r.tipIndexes[chainID]++
- r.candidates = append(r.candidates, block)
- r.candidateChains[chainID] = struct{}{}
- }
- }
-}
-
-// NextBlock implement Revealer.Next method, which would reveal blocks
-// forming valid DAGs.
-func (r *RandomDAGBlockRevealer) NextBlock() (types.Block, error) {
- if len(r.candidates) == 0 {
- r.pickCandidates()
- if len(r.candidates) == 0 {
- return types.Block{}, db.ErrIterationFinished
- }
- }
-
- // Pick next block to be revealed.
- picked := r.randGen.Intn(len(r.candidates))
- block := r.candidates[picked]
- r.candidates =
- append(r.candidates[:picked], r.candidates[picked+1:]...)
- delete(r.candidateChains, block.Position.ChainID)
- r.revealed[block.Hash] = struct{}{}
- r.pickCandidates()
- return *block, nil
-}
-
-// Reset implement Revealer.Reset method, which would reset the revealing.
-func (r *RandomDAGBlockRevealer) Reset() {
- r.tipIndexes = make(map[uint32]int)
- for chainID := range r.blocksByChain {
- r.tipIndexes[chainID] = 0
- }
- r.revealed = make(map[common.Hash]struct{})
- r.candidates = []*types.Block{}
-}
-
-// RandomBlockRevealer implements BlockRevealer interface, which would load
-// all blocks from db, and randomly pick one block to reveal.
-type RandomBlockRevealer struct {
- blocks map[common.Hash]*types.Block
- remains common.Hashes
- randGen *rand.Rand
-}
-
-// NewRandomBlockRevealer constructs RandomBlockRevealer.
-func NewRandomBlockRevealer(
- iter db.BlockIterator) (r *RandomBlockRevealer, err error) {
-
- blocks, err := loadAllBlocks(iter)
- if err != nil {
- return
- }
- r = &RandomBlockRevealer{
- blocks: blocks,
- randGen: rand.New(rand.NewSource(time.Now().UnixNano())),
- }
- r.Reset()
- return
-}
-
-// NextBlock implements Revealer.NextBlock method, which would reveal blocks
-// randomly.
-func (r *RandomBlockRevealer) NextBlock() (types.Block, error) {
- if len(r.remains) == 0 {
- return types.Block{}, db.ErrIterationFinished
- }
-
- picked := r.randGen.Intn(len(r.remains))
- block := r.blocks[r.remains[picked]]
- r.remains =
- append(r.remains[:picked], r.remains[picked+1:]...)
- return *block, nil
-}
-
-// Reset implement Revealer.Reset method, which would reset revealing.
-func (r *RandomBlockRevealer) Reset() {
- hashes := common.Hashes{}
- for hash := range r.blocks {
- hashes = append(hashes, hash)
- }
- r.remains = hashes
-}
-
-// RandomTipBlockRevealer implements BlockRevealer interface, which would load
-// all blocks from db, and randomly pick one chain's tip to reveal.
-type RandomTipBlockRevealer struct {
- chainsBlock []map[uint64]*types.Block
- chainTip []uint64
- chainRevealSeq []uint32
- revealed int
- randGen *rand.Rand
-}
-
-// NewRandomTipBlockRevealer constructs RandomTipBlockRevealer.
-func NewRandomTipBlockRevealer(
- iter db.BlockIterator) (r *RandomTipBlockRevealer, err error) {
-
- blocks, err := loadAllBlocks(iter)
- if err != nil {
- return
- }
- r = &RandomTipBlockRevealer{
- randGen: rand.New(rand.NewSource(time.Now().UnixNano())),
- }
- for _, b := range blocks {
- for b.Position.ChainID >= uint32(len(r.chainsBlock)) {
- r.chainsBlock = append(r.chainsBlock, make(map[uint64]*types.Block))
- r.chainTip = append(r.chainTip, 0)
- }
- r.chainsBlock[b.Position.ChainID][b.Position.Height] = b
- r.chainRevealSeq = append(r.chainRevealSeq, b.Position.ChainID)
- }
- r.Reset()
- return
-}
-
-// NextBlock implements Revealer.Next method, which would reveal blocks randomly.
-func (r *RandomTipBlockRevealer) NextBlock() (types.Block, error) {
- if len(r.chainRevealSeq) == r.revealed {
- return types.Block{}, db.ErrIterationFinished
- }
-
- picked := r.chainRevealSeq[r.revealed]
- r.revealed++
- block := r.chainsBlock[picked][r.chainTip[picked]]
- r.chainTip[picked]++
- return *block, nil
-}
-
-// Reset implement Revealer.Reset method, which would reset revealing.
-func (r *RandomTipBlockRevealer) Reset() {
- r.revealed = 0
- r.randGen.Shuffle(len(r.chainRevealSeq), func(i, j int) {
- r.chainRevealSeq[i], r.chainRevealSeq[j] =
- r.chainRevealSeq[j], r.chainRevealSeq[i]
- })
- for i := range r.chainTip {
- r.chainTip[i] = 0
- }
-}
-
// CompactionChainBlockRevealer implements BlockRevealer interface, which would
// load all blocks from db, reveal them in the order of compaction chain,
// from the genesis block to the latest one.
type CompactionChainBlockRevealer struct {
- blocks types.ByFinalizationHeight
+ blocks types.BlocksByFinalizationHeight
nextRevealIndex int
}
@@ -290,14 +82,14 @@ func NewCompactionChainBlockRevealer(iter db.BlockIterator,
if startHeight == 0 {
startHeight = 1
}
- blocks := types.ByFinalizationHeight{}
+ blocks := types.BlocksByFinalizationHeight{}
for _, b := range blocksByHash {
if b.Finalization.Height < startHeight {
continue
}
blocks = append(blocks, b)
}
- sort.Sort(types.ByFinalizationHeight(blocks))
+ sort.Sort(types.BlocksByFinalizationHeight(blocks))
// Make sure the finalization height of blocks are incremental with step 1.
for idx, b := range blocks {
if idx == 0 {
diff --git a/core/test/block-revealer_test.go b/core/test/block-revealer_test.go
index 0e56eea..54432e8 100644
--- a/core/test/block-revealer_test.go
+++ b/core/test/block-revealer_test.go
@@ -19,7 +19,6 @@ package test
import (
"testing"
- "time"
"github.com/dexon-foundation/dexon-consensus/common"
"github.com/dexon-foundation/dexon-consensus/core/db"
@@ -29,132 +28,6 @@ import (
type BlockRevealerTestSuite struct {
suite.Suite
-
- db db.Database
- totalBlockCount int
-}
-
-func (s *BlockRevealerTestSuite) SetupSuite() {
- var (
- err error
- genesisTime = time.Now().UTC()
- )
- // Setup block database.
- s.db, err = db.NewMemBackedDB()
- s.Require().NoError(err)
-
- // Randomly generate blocks.
- config := &BlocksGeneratorConfig{
- NumChains: 19,
- MinBlockTimeInterval: 250 * time.Millisecond,
- }
- gen := NewBlocksGenerator(config, nil)
- s.Require().NoError(gen.Generate(
- 0,
- genesisTime,
- genesisTime.Add(30*time.Second),
- s.db))
- // Cache the count of total generated block.
- iter, err := s.db.GetAllBlocks()
- s.Require().NoError(err)
- blocks, err := loadAllBlocks(iter)
- s.Require().NoError(err)
- s.totalBlockCount = len(blocks)
-}
-
-func (s *BlockRevealerTestSuite) baseTest(
- revealer BlockRevealer,
- repeat int,
- checkFunc func(*types.Block, map[common.Hash]struct{})) {
-
- revealingSequence := map[string]struct{}{}
- for i := 0; i < repeat; i++ {
- revealed := map[common.Hash]struct{}{}
- sequence := ""
- for {
- b, err := revealer.NextBlock()
- if err != nil {
- if err == db.ErrIterationFinished {
- err = nil
- break
- }
- s.Require().NotNil(err)
- }
- checkFunc(&b, revealed)
- revealed[b.Hash] = struct{}{}
- sequence += b.Hash.String() + ","
- }
- s.Len(revealed, s.totalBlockCount)
- revealingSequence[sequence] = struct{}{}
- revealer.Reset()
- }
- // It should be reasonable to reveal at least two
- // different sequence.
- s.True(len(revealingSequence) > 1)
-
-}
-
-func (s *BlockRevealerTestSuite) TestRandomBlockReveal() {
- // This test case would make sure we could at least generate
- // two different revealing sequence when revealing more than
- // 10 times.
- iter, err := s.db.GetAllBlocks()
- s.Require().Nil(err)
- revealer, err := NewRandomBlockRevealer(iter)
- s.Require().Nil(err)
-
- checkFunc := func(b *types.Block, revealed map[common.Hash]struct{}) {
- // Make sure the revealer won't reveal the same block twice.
- _, alreadyRevealed := revealed[b.Hash]
- s.False(alreadyRevealed)
- }
- s.baseTest(revealer, 10, checkFunc)
-}
-
-func (s *BlockRevealerTestSuite) TestRandomDAGBlockReveal() {
- // This test case would make sure we could at least generate
- // two different revealing sequence when revealing more than
- // 10 times, and each of them would form valid DAGs during
- // revealing.
-
- iter, err := s.db.GetAllBlocks()
- s.Require().Nil(err)
- revealer, err := NewRandomDAGBlockRevealer(iter)
- s.Require().Nil(err)
-
- checkFunc := func(b *types.Block, revealed map[common.Hash]struct{}) {
- // Make sure this revealer won't reveal
- // the same block twice.
- _, alreadyRevealed := revealed[b.Hash]
- s.False(alreadyRevealed)
- // Make sure the newly revealed block would still
- // form a valid DAG after added to revealed blocks.
- s.True(isAllAckingBlockRevealed(b, revealed))
- }
- s.baseTest(revealer, 10, checkFunc)
-}
-
-func (s *BlockRevealerTestSuite) TestRandomTipBlockReveal() {
- // This test case would make sure we could at least generate
- // two different revealing sequence when revealing more than
- // 10 times.
- iter, err := s.db.GetAllBlocks()
- s.Require().Nil(err)
- revealer, err := NewRandomTipBlockRevealer(iter)
- s.Require().Nil(err)
-
- checkFunc := func(b *types.Block, revealed map[common.Hash]struct{}) {
- // Make sure the revealer won't reveal the same block twice.
- _, alreadyRevealed := revealed[b.Hash]
- s.False(alreadyRevealed)
- // Make sure the parent is already revealed.
- if b.Position.Height == 0 {
- return
- }
- _, alreadyRevealed = revealed[b.ParentHash]
- s.True(alreadyRevealed)
- }
- s.baseTest(revealer, 10, checkFunc)
}
func (s *BlockRevealerTestSuite) TestCompactionChainBlockReveal() {
@@ -186,7 +59,7 @@ func (s *BlockRevealerTestSuite) TestCompactionChainBlockReveal() {
// instance successfully.
r, err := NewCompactionChainBlockRevealer(iter, 0)
s.Require().Nil(r)
- s.Require().IsType(ErrNotValidCompactionChain, err)
+ s.Require().Equal(ErrNotValidCompactionChain.Error(), err.Error())
// Put a block to make the compaction chain complete.
s.Require().NoError(dbInst.PutBlock(*b2))
// We can construct that revealer now.
@@ -206,7 +79,7 @@ func (s *BlockRevealerTestSuite) TestCompactionChainBlockReveal() {
chk(3)
// Iteration should be finished
_, err = r.NextBlock()
- s.Require().IsType(db.ErrIterationFinished, err)
+ s.Require().Equal(db.ErrIterationFinished.Error(), err.Error())
// Test 'startHeight' parameter.
iter, err = dbInst.GetAllBlocks()
s.Require().NoError(err)
diff --git a/core/test/blocks-generator.go b/core/test/blocks-generator.go
deleted file mode 100644
index 5f1dbea..0000000
--- a/core/test/blocks-generator.go
+++ /dev/null
@@ -1,379 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package test
-
-import (
- "errors"
- "math"
- "math/rand"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core/crypto/ecdsa"
- "github.com/dexon-foundation/dexon-consensus/core/db"
- "github.com/dexon-foundation/dexon-consensus/core/types"
- "github.com/dexon-foundation/dexon-consensus/core/utils"
-)
-
-// ErrParentNotAcked would be raised when some block doesn't
-// ack its parent block.
-var ErrParentNotAcked = errors.New("parent is not acked")
-
-// nodeStatus is a state holder for each node
-// during generating blocks.
-type nodeStatus struct {
- blocks []*types.Block
- genesisTime time.Time
- signer *utils.Signer
- tip *types.Block
- nextAckingIndex map[types.NodeID]uint64
-}
-
-// getAckedBlockHash would randomly pick one block between
-// last acked one to current head.
-func (ns *nodeStatus) getAckedBlockHash(
- ackedNID types.NodeID,
- ackedNode *nodeStatus,
- randGen *rand.Rand) (
- hash common.Hash, ok bool) {
- baseAckingIndex := ns.nextAckingIndex[ackedNID]
- totalBlockCount := uint64(len(ackedNode.blocks))
- if totalBlockCount <= baseAckingIndex {
- // There is no new block to ack.
- return
- }
- ackableRange := totalBlockCount - baseAckingIndex
- idx := uint64((randGen.Uint64() % ackableRange) + baseAckingIndex)
- ns.nextAckingIndex[ackedNID] = idx + 1
- hash = ackedNode.blocks[idx].Hash
- ok = true
- return
-}
-
-func (ns *nodeStatus) getNextBlockTime(
- timePicker func(time.Time) time.Time) time.Time {
- if ns.tip == nil {
- return timePicker(ns.genesisTime)
- }
- return timePicker(ns.tip.Timestamp)
-}
-
-// nodeSetStatus is a state holder for all nodes
-// during generating blocks.
-type nodeSetStatus struct {
- round uint64
- status map[types.NodeID]*nodeStatus
- proposerChain map[types.NodeID]uint32
- endTime time.Time
- nIDs []types.NodeID
- randGen *rand.Rand
- timePicker func(time.Time) time.Time
-}
-
-func newNodeSetStatus(
- numChains uint32,
- tips map[uint32]*types.Block,
- round uint64,
- genesisTime, endTime time.Time,
- timePicker func(time.Time) time.Time) *nodeSetStatus {
- var (
- status = make(map[types.NodeID]*nodeStatus)
- proposerChain = make(map[types.NodeID]uint32)
- nIDs = []types.NodeID{}
- )
- for i := uint32(0); i < numChains; i++ {
- prvKey, err := ecdsa.NewPrivateKey()
- if err != nil {
- panic(err)
- }
- nID := types.NewNodeID(prvKey.PublicKey())
- nIDs = append(nIDs, nID)
- status[nID] = &nodeStatus{
- blocks: []*types.Block{},
- genesisTime: genesisTime,
- signer: utils.NewSigner(prvKey),
- tip: tips[i],
- nextAckingIndex: make(map[types.NodeID]uint64),
- }
- proposerChain[nID] = i
- }
- return &nodeSetStatus{
- round: round,
- status: status,
- proposerChain: proposerChain,
- endTime: endTime,
- nIDs: nIDs,
- randGen: rand.New(rand.NewSource(time.Now().UnixNano())),
- timePicker: timePicker,
- }
-}
-
-// findIncompleteNodes is a helper to check which node doesn't generate
-// enough blocks.
-func (ns *nodeSetStatus) findIncompleteNodes() (nIDs []types.NodeID) {
- for nID, status := range ns.status {
- if status.tip == nil {
- nIDs = append(nIDs, nID)
- continue
- }
- if status.tip.Timestamp.After(ns.endTime) {
- continue
- }
- nIDs = append(nIDs, nID)
- }
- return
-}
-
-// prepareAcksForNewBlock collects acks for one block.
-func (ns *nodeSetStatus) prepareAcksForNewBlock(
- proposerID types.NodeID, ackingCount int) (
- acks common.Hashes, err error) {
- acks = common.Hashes{}
- if len(ns.status[proposerID].blocks) == 0 {
- // The 'Acks' filed of genesis blocks would always be empty.
- return
- }
- // Pick nodeIDs to be acked.
- ackingNIDs := map[types.NodeID]struct{}{}
- if ackingCount > 0 {
- ackingCount-- // We would always include ack to parent block.
- }
- for _, i := range ns.randGen.Perm(len(ns.nIDs))[:ackingCount] {
- ackingNIDs[ns.nIDs[i]] = struct{}{}
- }
- // Generate acks.
- for nID := range ackingNIDs {
- if nID == proposerID {
- continue
- }
- ack, ok := ns.status[proposerID].getAckedBlockHash(
- nID, ns.status[nID], ns.randGen)
- if !ok {
- if nID == proposerID {
- err = ErrParentNotAcked
- }
- continue
- }
- acks = append(acks, ack)
- }
- return
-}
-
-// proposeBlock propose new block and update node status.
-func (ns *nodeSetStatus) proposeBlock(
- proposerID types.NodeID, acks common.Hashes) (*types.Block, error) {
- status := ns.status[proposerID]
- parentHash := common.Hash{}
- blockHeight := uint64(0)
- if status.tip != nil {
- parentHash = status.tip.Hash
- blockHeight = status.tip.Position.Height + 1
- acks = append(acks, parentHash)
- }
- chainID := ns.proposerChain[proposerID]
- newBlock := &types.Block{
- ParentHash: parentHash,
- Position: types.Position{
- Round: ns.round,
- Height: blockHeight,
- ChainID: chainID,
- },
- Timestamp: status.getNextBlockTime(ns.timePicker),
- }
- newBlock.Acks = common.NewSortedHashes(acks)
- if err := status.signer.SignBlock(newBlock); err != nil {
- return nil, err
- }
- status.blocks = append(status.blocks, newBlock)
- status.tip = newBlock
- return newBlock, nil
-}
-
-// normalAckingCountGenerator would randomly pick acking count
-// by a normal distribution.
-func normalAckingCountGenerator(
- chainNum uint32, mean, deviation float64) func() int {
- return func() int {
- var expected float64
- for {
- expected = rand.NormFloat64()*deviation + mean
- if expected >= 0 && expected <= float64(chainNum) {
- break
- }
- }
- return int(math.Ceil(expected))
- }
-}
-
-// MaxAckingCountGenerator return generator which returns
-// fixed maximum acking count.
-func MaxAckingCountGenerator(count uint32) func() int {
- return func() int { return int(count) }
-}
-
-// generateNodePicker is a function generator, which would generate
-// a function to randomly pick one node ID from a slice of node ID.
-func generateNodePicker() func([]types.NodeID) types.NodeID {
- privateRand := rand.New(rand.NewSource(time.Now().UnixNano()))
- return func(nIDs []types.NodeID) types.NodeID {
- return nIDs[privateRand.Intn(len(nIDs))]
- }
-}
-
-// defaultTimePicker would pick a time based on reference time plus min.
-func generateTimePicker(min time.Duration) (f func(time.Time) time.Time) {
- privateRand := rand.New(rand.NewSource(time.Now().UnixNano()))
- return func(ref time.Time) time.Time {
- return ref.Add(min + time.Duration(
- privateRand.Int63n(int64(500*time.Millisecond))))
- }
-}
-
-// BlocksGeneratorConfig is the configuration for BlocksGenerator.
-type BlocksGeneratorConfig struct {
- NumChains uint32
- MinBlockTimeInterval time.Duration
-}
-
-// NewBlocksGeneratorConfig construct a BlocksGeneratorConfig instance.
-func NewBlocksGeneratorConfig(c *types.Config) *BlocksGeneratorConfig {
- return &BlocksGeneratorConfig{
- NumChains: c.NumChains,
- MinBlockTimeInterval: c.MinBlockInterval,
- }
-}
-
-// BlocksGenerator could generate blocks forming valid DAGs.
-type BlocksGenerator struct {
- config *BlocksGeneratorConfig
- nodePicker func([]types.NodeID) types.NodeID
- timePicker func(time.Time) time.Time
- ackingCountGenerator func() int
-}
-
-// NewBlocksGenerator constructs BlockGenerator.
-//
-// The caller is responsible to provide a function to generate count of
-// acked block for each new block. The prototype of ackingCountGenerator is
-// a function returning 'int'. For example, if you need to generate a group of
-// blocks and each of them has maximum 2 acks.
-// func () int { return 2 }
-// The default ackingCountGenerator would randomly pick a number based on
-// the nodeCount you provided with a normal distribution.
-func NewBlocksGenerator(
- config *BlocksGeneratorConfig,
- ackingCountGenerator func() int) *BlocksGenerator {
- if config.MinBlockTimeInterval == time.Duration(0) {
- panic(errors.New("min block interval cannot be 0"))
- }
- if ackingCountGenerator == nil {
- ackingCountGenerator = normalAckingCountGenerator(
- config.NumChains,
- float64(config.NumChains/5),
- float64(config.NumChains/7+1))
- }
- timePicker := generateTimePicker(config.MinBlockTimeInterval)
- return &BlocksGenerator{
- config: config,
- nodePicker: generateNodePicker(),
- timePicker: timePicker,
- ackingCountGenerator: ackingCountGenerator,
- }
-}
-
-// Generate is the entry point to generate blocks in one round.
-func (gen *BlocksGenerator) Generate(
- roundID uint64,
- roundBegin, roundEnd time.Time,
- dbInst db.Database) (err error) {
- // Find tips of previous round if available.
- tips := make(map[uint32]*types.Block)
- if roundID > 0 {
- tips, err = gen.findTips(roundID-1, dbInst)
- if err != nil {
- return
- }
- }
- status := newNodeSetStatus(gen.config.NumChains, tips, roundID,
- roundBegin, roundEnd, gen.timePicker)
- // We would record the smallest height of block that could be acked
- // from each node's point-of-view.
- toAck := make(map[types.NodeID]map[types.NodeID]uint64)
- for _, nID := range status.nIDs {
- toAck[nID] = make(map[types.NodeID]uint64)
- }
- for {
- // Find nodes that doesn't propose enough blocks and
- // pick one from them randomly.
- notYet := status.findIncompleteNodes()
- if len(notYet) == 0 {
- break
- }
- // Propose a new block.
- var (
- proposerID = gen.nodePicker(notYet)
- acks common.Hashes
- )
- if acks, err = status.prepareAcksForNewBlock(
- proposerID, gen.ackingCountGenerator()); err != nil {
- return
- }
- var newBlock *types.Block
- if newBlock, err = status.proposeBlock(proposerID, acks); err != nil {
- return
- }
- // Persist block to db.
- if err = dbInst.PutBlock(*newBlock); err != nil {
- return
- }
- }
- return
-}
-
-// findTips is an utility to find tips of each chain in that round in db.
-func (gen *BlocksGenerator) findTips(round uint64, dbInst db.Reader) (
- tips map[uint32]*types.Block, err error) {
- iter, err := dbInst.GetAllBlocks()
- if err != nil {
- return
- }
- revealer, err := NewRandomBlockRevealer(iter)
- if err != nil {
- return
- }
- tips = make(map[uint32]*types.Block)
- for {
- var b types.Block
- if b, err = revealer.NextBlock(); err != nil {
- if err == db.ErrIterationFinished {
- err = nil
- break
- }
- return
- }
- if b.Position.Round != round {
- continue
- }
- tip, exists := tips[b.Position.ChainID]
- if exists && tip.Position.Height > b.Position.Height {
- continue
- }
- tips[b.Position.ChainID] = &b
- }
- return
-}
diff --git a/core/test/blocks-generator_test.go b/core/test/blocks-generator_test.go
deleted file mode 100644
index bcbd749..0000000
--- a/core/test/blocks-generator_test.go
+++ /dev/null
@@ -1,323 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package test
-
-import (
- "sort"
- "testing"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core/db"
- "github.com/dexon-foundation/dexon-consensus/core/types"
- "github.com/stretchr/testify/suite"
-)
-
-type BlocksGeneratorTestSuite struct {
- suite.Suite
-}
-
-func (s *BlocksGeneratorTestSuite) TestGenerate() {
- // This test case is to make sure the generated blocks are legimate.
- var (
- config = &BlocksGeneratorConfig{
- NumChains: 19,
- MinBlockTimeInterval: 200 * time.Millisecond,
- }
- gen = NewBlocksGenerator(config, nil)
- req = s.Require()
- beginTime = time.Now().UTC()
- endTime = beginTime.Add(time.Minute)
- )
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
- req.NoError(gen.Generate(1, beginTime, endTime, dbInst))
- // Load all blocks in that database for further checking.
- iter, err := dbInst.GetAllBlocks()
- req.NoError(err)
- blocksByChain := make(map[uint32][]*types.Block)
- blocksByHash := make(map[common.Hash]*types.Block)
- for {
- block, err := iter.NextBlock()
- if err == db.ErrIterationFinished {
- break
- }
- req.NoError(err)
- // TODO(mission): Make sure each block is correctly signed once
- // we have a way to access core.hashBlock.
- req.NotEqual(block.Hash, common.Hash{})
- if !block.IsEmpty() {
- req.NotEmpty(block.Signature)
- }
- req.Equal(block.Position.Round, uint64(1))
- blocksByChain[block.Position.ChainID] =
- append(blocksByChain[block.Position.ChainID], &block)
- sort.Sort(types.ByPosition(blocksByChain[block.Position.ChainID]))
- blocksByHash[block.Hash] = &block
- }
- // Make sure these two rules are hold for these blocks:
- // - No backward acking: the later block should only ack new blocks
- // compared to its parent block.
- // - Parent Ack: always ack its parent block.
- // - Timestamp: timestamp are increasing, and with valid interval to
- // previous block.
- // - The last block of each chain should pass endTime.
- // - No Acks in genesis bloc
- for _, blocks := range blocksByChain {
- lastAckingHeights := map[uint32]uint64{}
- req.NotEmpty(blocks)
- // Check genesis block.
- genesisBlock := blocks[0]
- req.Equal(genesisBlock.ParentHash, common.Hash{})
- req.Equal(genesisBlock.Position.Height, uint64(0))
- req.Empty(genesisBlock.Acks)
- // Check normal blocks.
- for index, block := range blocks[1:] {
- parentAcked := false
- for _, ack := range block.Acks {
- if ack == block.ParentHash {
- parentAcked = true
- }
- ackedBlock := blocksByHash[ack]
- req.NotNil(ackedBlock)
- prevAckingHeight, exists :=
- lastAckingHeights[ackedBlock.Position.ChainID]
- if exists {
- s.True(prevAckingHeight < ackedBlock.Position.Height)
- }
- lastAckingHeights[ackedBlock.Position.ChainID] =
- ackedBlock.Position.Height
- // Block Height should always incremental by 1.
- //
- // Because we iterate blocks slice from 1,
- // we need to add 1 to the index.
- req.Equal(block.Position.Height, uint64(index+1))
- }
- req.True(parentAcked)
- }
- // The block time of the last block should be after end time.
- req.True(blocks[len(blocks)-1].Timestamp.After(endTime))
- }
-}
-
-func (s *BlocksGeneratorTestSuite) TestGenerateWithMaxAckCount() {
- var (
- config = &BlocksGeneratorConfig{
- NumChains: 13,
- MinBlockTimeInterval: 250 * time.Millisecond,
- }
- req = s.Require()
- totalAckingCount = 0
- totalBlockCount = 0
- genesisTime = time.Now().UTC()
- )
- // Generate with 0 acks.
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
- gen := NewBlocksGenerator(config, MaxAckingCountGenerator(0))
- req.NoError(gen.Generate(
- 0,
- genesisTime,
- genesisTime.Add(50*time.Second),
- dbInst))
- // Load blocks to check their acking count.
- iter, err := dbInst.GetAllBlocks()
- req.NoError(err)
- for {
- block, err := iter.NextBlock()
- if err == db.ErrIterationFinished {
- break
- }
- req.NoError(err)
- if block.IsGenesis() {
- continue
- }
- req.Len(block.Acks, 1)
- }
- // Generate with acks as many as possible.
- dbInst, err = db.NewMemBackedDB()
- req.NoError(err)
- gen = NewBlocksGenerator(config, MaxAckingCountGenerator(config.NumChains))
- req.NoError(gen.Generate(
- 0,
- genesisTime,
- genesisTime.Add(50*time.Second),
- dbInst))
- // Load blocks to verify the average acking count.
- iter, err = dbInst.GetAllBlocks()
- req.NoError(err)
- for {
- block, err := iter.NextBlock()
- if err == db.ErrIterationFinished {
- break
- }
- req.NoError(err)
- if block.IsGenesis() {
- continue
- }
- totalAckingCount += len(block.Acks)
- totalBlockCount++
- }
- req.NotZero(totalBlockCount)
- req.True((totalAckingCount / totalBlockCount) >= int(config.NumChains/2))
-}
-
-// TestFindTips make sure findTips works as expected.
-func (s *BlocksGeneratorTestSuite) TestFindTips() {
- var (
- config = &BlocksGeneratorConfig{
- NumChains: 10,
- MinBlockTimeInterval: 250 * time.Millisecond,
- }
- req = s.Require()
- genesisTime = time.Now().UTC()
- endTime = genesisTime.Add(100 * time.Second)
- )
- gen := NewBlocksGenerator(config, nil)
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
- req.NoError(gen.Generate(
- 0,
- genesisTime,
- endTime,
- dbInst))
- tips, err := gen.findTips(0, dbInst)
- req.NoError(err)
- req.Len(tips, int(config.NumChains))
- for _, b := range tips {
- req.True(b.Timestamp.After(endTime))
- }
-}
-
-func (s *BlocksGeneratorTestSuite) TestConcateBlocksFromRounds() {
- // This test case run these steps:
- // - generate blocks by round but sharing one db.
- // - if those rounds are continuous, they should be concated.
- var (
- req = s.Require()
- genesisTime = time.Now().UTC()
- )
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
- // Generate round 0 blocks.
- gen := NewBlocksGenerator(&BlocksGeneratorConfig{
- NumChains: 4,
- MinBlockTimeInterval: 250 * time.Millisecond,
- }, MaxAckingCountGenerator(4))
- req.NoError(gen.Generate(
- 0,
- genesisTime,
- genesisTime.Add(10*time.Second),
- dbInst))
- tips0, err := gen.findTips(0, dbInst)
- req.NoError(err)
- req.Len(tips0, 4)
- // Generate round 1 blocks.
- gen = NewBlocksGenerator(&BlocksGeneratorConfig{
- NumChains: 10,
- MinBlockTimeInterval: 250 * time.Millisecond,
- }, MaxAckingCountGenerator(10))
- req.NoError(gen.Generate(
- 1,
- genesisTime.Add(10*time.Second),
- genesisTime.Add(20*time.Second),
- dbInst))
- tips1, err := gen.findTips(1, dbInst)
- req.NoError(err)
- req.Len(tips1, 10)
- // Generate round 2 blocks.
- gen = NewBlocksGenerator(&BlocksGeneratorConfig{
- NumChains: 7,
- MinBlockTimeInterval: 250 * time.Millisecond,
- }, MaxAckingCountGenerator(7))
- req.NoError(gen.Generate(
- 2,
- genesisTime.Add(20*time.Second),
- genesisTime.Add(30*time.Second),
- dbInst))
- tips2, err := gen.findTips(2, dbInst)
- req.NoError(err)
- req.Len(tips2, 7)
- // Check results, make sure tips0, tips1 are acked by correct blocks.
- iter, err := dbInst.GetAllBlocks()
- req.NoError(err)
- revealer, err := NewRandomBlockRevealer(iter)
- req.NoError(err)
- removeTip := func(tips map[uint32]*types.Block, b *types.Block) {
- toRemove := []uint32{}
- for chainID, tip := range tips {
- if b.ParentHash == tip.Hash {
- req.Equal(b.Position.Height, tip.Position.Height+1)
- req.Equal(b.Position.Round, tip.Position.Round+1)
- req.True(b.IsAcking(tip.Hash))
- toRemove = append(toRemove, chainID)
- }
- }
- for _, ID := range toRemove {
- delete(tips, ID)
- }
- }
- // Make sure all tips are acked by loading blocks from db
- // and check them one by one.
- for {
- b, err := revealer.NextBlock()
- if err != nil {
- if err == db.ErrIterationFinished {
- err = nil
- break
- }
- req.NoError(err)
- }
- switch b.Position.Round {
- case 1:
- removeTip(tips0, &b)
- case 2:
- removeTip(tips1, &b)
- }
- }
- req.Empty(tips0)
- req.Len(tips1, 3)
- req.Contains(tips1, uint32(7))
- req.Contains(tips1, uint32(8))
- req.Contains(tips1, uint32(9))
- // Check the acking frequency of last round, it might be wrong.
- totalBlockCount := 0
- totalAckCount := 0
- revealer.Reset()
- for {
- b, err := revealer.NextBlock()
- if err != nil {
- if err == db.ErrIterationFinished {
- err = nil
- break
- }
- req.NoError(err)
- }
- if b.Position.Round != 2 {
- continue
- }
- totalBlockCount++
- totalAckCount += len(b.Acks)
- }
- // At least all blocks can ack some non-parent block.
- req.True(totalAckCount/totalBlockCount >= 2)
-}
-
-func TestBlocksGenerator(t *testing.T) {
- suite.Run(t, new(BlocksGeneratorTestSuite))
-}
diff --git a/core/test/interface.go b/core/test/interface.go
index 000b835..0712ad5 100644
--- a/core/test/interface.go
+++ b/core/test/interface.go
@@ -34,22 +34,6 @@ type BlockRevealer interface {
Reset()
}
-// Stopper defines an interface for Scheduler to tell when to stop execution.
-type Stopper interface {
- // ShouldStop is provided with the ID of the handler just finishes an event.
- // It's thread-safe to access internal/shared state of the handler at this
- // moment.
- // The Stopper should check state of that handler and return 'true'
- // if the execution could be stopped.
- ShouldStop(nID types.NodeID) bool
-}
-
-// EventHandler defines an interface to handle a Scheduler event.
-type EventHandler interface {
- // Handle the event belongs to this handler, and return derivated events.
- Handle(*Event) []*Event
-}
-
// TransportPeerType defines the type of peer, either 'peer' or 'server'.
type TransportPeerType string
diff --git a/core/test/network.go b/core/test/network.go
index 000037f..f5d1c6e 100644
--- a/core/test/network.go
+++ b/core/test/network.go
@@ -166,7 +166,7 @@ type Network struct {
unreceivedRandomness map[common.Hash]chan<- common.Hash
cache *utils.NodeSetCache
notarySetCachesLock sync.Mutex
- notarySetCaches map[uint64]map[uint32]map[types.NodeID]struct{}
+ notarySetCaches map[uint64]map[types.NodeID]struct{}
dkgSetCachesLock sync.Mutex
dkgSetCaches map[uint64]map[types.NodeID]struct{}
}
@@ -188,9 +188,8 @@ func NewNetwork(pubKey crypto.PublicKey, config NetworkConfig) (
unreceivedBlocks: make(map[common.Hash]chan<- common.Hash),
unreceivedRandomness: make(map[common.Hash]chan<- common.Hash),
peers: make(map[types.NodeID]struct{}),
- notarySetCaches: make(
- map[uint64]map[uint32]map[types.NodeID]struct{}),
- dkgSetCaches: make(map[uint64]map[types.NodeID]struct{}),
+ notarySetCaches: make(map[uint64]map[types.NodeID]struct{}),
+ dkgSetCaches: make(map[uint64]map[types.NodeID]struct{}),
voteCache: make(
map[types.Position]map[types.VoteHeader]*types.Vote),
}
@@ -226,8 +225,7 @@ func (n *Network) PullRandomness(hashes common.Hashes) {
// BroadcastVote implements core.Network interface.
func (n *Network) BroadcastVote(vote *types.Vote) {
- if err := n.trans.Broadcast(
- n.getNotarySet(vote.Position.Round, vote.Position.ChainID),
+ if err := n.trans.Broadcast(n.getNotarySet(vote.Position.Round),
n.config.DirectLatency, vote); err != nil {
panic(err)
}
@@ -238,7 +236,7 @@ func (n *Network) BroadcastVote(vote *types.Vote) {
func (n *Network) BroadcastBlock(block *types.Block) {
// Avoid data race in fake transport.
block = n.cloneForFake(block).(*types.Block)
- notarySet := n.getNotarySet(block.Position.Round, block.Position.ChainID)
+ notarySet := n.getNotarySet(block.Position.Round)
if err := n.trans.Broadcast(
notarySet, n.config.DirectLatency, block); err != nil {
panic(err)
@@ -276,8 +274,7 @@ func (n *Network) BroadcastRandomnessResult(
return
}
// Send to notary set first.
- notarySet := n.getNotarySet(
- randResult.Position.Round, randResult.Position.ChainID)
+ notarySet := n.getNotarySet(randResult.Position.Round)
if err := n.trans.Broadcast(
notarySet, n.config.DirectLatency, randResult); err != nil {
panic(err)
@@ -568,7 +565,7 @@ func (n *Network) pullVotesAsync(pos types.Position) {
Identity: pos,
}
// Get corresponding notary set.
- notarySet := n.getNotarySet(pos.Round, pos.ChainID)
+ notarySet := n.getNotarySet(pos.Round)
// Randomly select one peer from notary set and send a pull request.
sentCount := 0
for nID := range notarySet {
@@ -727,8 +724,7 @@ func (n *Network) cloneForFake(v interface{}) interface{} {
}
// getNotarySet gets notary set for that (round, chain) from cache.
-func (n *Network) getNotarySet(
- round uint64, chain uint32) map[types.NodeID]struct{} {
+func (n *Network) getNotarySet(round uint64) map[types.NodeID]struct{} {
if n.cache == nil {
// Default behavior is to broadcast to all peers, which makes it easier
// to be used in simple test cases.
@@ -736,19 +732,14 @@ func (n *Network) getNotarySet(
}
n.notarySetCachesLock.Lock()
defer n.notarySetCachesLock.Unlock()
- roundSets, exists := n.notarySetCaches[round]
- if !exists {
- roundSets = make(map[uint32]map[types.NodeID]struct{})
- n.notarySetCaches[round] = roundSets
- }
- set, exists := roundSets[chain]
+ set, exists := n.notarySetCaches[round]
if !exists {
var err error
- set, err = n.cache.GetNotarySet(round, chain)
+ set, err = n.cache.GetNotarySet(round, 0)
if err != nil {
panic(err)
}
- roundSets[chain] = set
+ n.notarySetCaches[round] = set
}
return set
}
diff --git a/core/test/scheduler-event.go b/core/test/scheduler-event.go
deleted file mode 100644
index 180ed07..0000000
--- a/core/test/scheduler-event.go
+++ /dev/null
@@ -1,79 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package test
-
-import (
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/core/types"
-)
-
-// Event defines a scheduler event.
-type Event struct {
- // HistoryIndex is the index of this event in history.
- HistoryIndex int
- // NodeID is the ID of handler that this event deginated to.
- NodeID types.NodeID
- // Time is the expected execution time of this event.
- Time time.Time
- // ExecError record the error when handling this event.
- ExecError error
- // Payload is application specific data carried by this event.
- Payload interface{}
- // ParentHistoryIndex is the index of parent event in history.
- ParentHistoryIndex int
- // ExecInterval is the latency to execute this event
- ExecInterval time.Duration
-}
-
-// eventQueue implements heap.Interface.
-type eventQueue []*Event
-
-func (eq eventQueue) Len() int { return len(eq) }
-
-func (eq eventQueue) Less(i, j int) bool {
- return eq[i].Time.Before(eq[j].Time)
-}
-
-func (eq eventQueue) Swap(i, j int) {
- eq[i], eq[j] = eq[j], eq[i]
-}
-
-func (eq *eventQueue) Push(x interface{}) {
- *eq = append(*eq, x.(*Event))
-}
-
-func (eq *eventQueue) Pop() interface{} {
- pos := len(*eq) - 1
- item := (*eq)[pos]
- *eq = (*eq)[0:pos]
- return item
-}
-
-// NewEvent is the constructor for Event.
-func NewEvent(
- nID types.NodeID, when time.Time, payload interface{}) *Event {
-
- return &Event{
- HistoryIndex: -1,
- ParentHistoryIndex: -1,
- NodeID: nID,
- Time: when,
- Payload: payload,
- }
-}
diff --git a/core/test/scheduler.go b/core/test/scheduler.go
deleted file mode 100644
index f6c7eed..0000000
--- a/core/test/scheduler.go
+++ /dev/null
@@ -1,215 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package test
-
-import (
- "container/heap"
- "context"
- "fmt"
- "sync"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/core/types"
-)
-
-var (
- // ErrSchedulerAlreadyStarted means callers attempt to insert some
- // seed events after calling 'Run'.
- ErrSchedulerAlreadyStarted = fmt.Errorf("scheduler already started")
- // errNilEventWhenNotified is an internal error which means a worker routine
- // can't get an event when notified.
- errNilEventWhenNotified = fmt.Errorf("nil event when notified")
-)
-
-type schedulerHandlerRecord struct {
- handler EventHandler
- lock sync.Mutex
-}
-
-// Scheduler is an event scheduler.
-type Scheduler struct {
- events eventQueue
- eventsLock sync.Mutex
- history []*Event
- historyLock sync.RWMutex
- isStarted bool
- handlers map[types.NodeID]*schedulerHandlerRecord
- handlersLock sync.RWMutex
- eventNotification chan struct{}
- ctx context.Context
- cancelFunc context.CancelFunc
- stopper Stopper
-}
-
-// NewScheduler constructs an Scheduler instance.
-func NewScheduler(stopper Stopper) *Scheduler {
- ctx, cancel := context.WithCancel(context.Background())
- return &Scheduler{
- events: eventQueue{},
- history: []*Event{},
- handlers: make(map[types.NodeID]*schedulerHandlerRecord),
- eventNotification: make(chan struct{}, 100000),
- ctx: ctx,
- cancelFunc: cancel,
- stopper: stopper,
- }
-}
-
-// Run would run the scheduler. If you need strict incrememtal execution order
-// of events based on their 'Time' field, assign 'numWorkers' as 1. If you need
-// faster execution, assign 'numWorkers' a larger number.
-func (sch *Scheduler) Run(numWorkers int) {
- var wg sync.WaitGroup
-
- sch.isStarted = true
- for i := 0; i < numWorkers; i++ {
- wg.Add(1)
- go sch.workerRoutine(&wg)
- }
- // Blocks until all routines are finished.
- wg.Wait()
-}
-
-// Seed is used to provide the scheduler some seed events.
-func (sch *Scheduler) Seed(e *Event) error {
- sch.eventsLock.Lock()
- defer sch.eventsLock.Unlock()
-
- if sch.isStarted {
- return ErrSchedulerAlreadyStarted
- }
- sch.addEvent(e)
- return nil
-}
-
-// RegisterEventHandler register an event handler by providing ID of
-// corresponding node.
-func (sch *Scheduler) RegisterEventHandler(
- nID types.NodeID,
- handler EventHandler) {
-
- sch.handlersLock.Lock()
- defer sch.handlersLock.Unlock()
-
- sch.handlers[nID] = &schedulerHandlerRecord{handler: handler}
-}
-
-// nextTick would pick the oldest event from eventQueue.
-func (sch *Scheduler) nextTick() (e *Event) {
- sch.eventsLock.Lock()
- defer sch.eventsLock.Unlock()
-
- if len(sch.events) == 0 {
- return nil
- }
- return heap.Pop(&sch.events).(*Event)
-}
-
-// addEvent is an helper function to add events into eventQueue sorted by
-// their 'Time' field.
-func (sch *Scheduler) addEvent(e *Event) {
- // Perform sorted insertion.
- heap.Push(&sch.events, e)
- sch.eventNotification <- struct{}{}
-}
-
-// CloneExecutionHistory returns a cloned event execution history.
-func (sch *Scheduler) CloneExecutionHistory() (cloned []*Event) {
- sch.historyLock.RLock()
- defer sch.historyLock.RUnlock()
-
- cloned = make([]*Event, len(sch.history))
- copy(cloned, sch.history)
- return
-}
-
-// workerRoutine is the mainloop when handling events.
-func (sch *Scheduler) workerRoutine(wg *sync.WaitGroup) {
- defer wg.Done()
-
- handleEvent := func(e *Event) {
- // Find correspond handler record.
- hRec := func(nID types.NodeID) *schedulerHandlerRecord {
- sch.handlersLock.RLock()
- defer sch.handlersLock.RUnlock()
-
- return sch.handlers[nID]
- }(e.NodeID)
-
- newEvents := func() []*Event {
- // This lock makes sure there would be no concurrent access
- // against each handler.
- hRec.lock.Lock()
- defer hRec.lock.Unlock()
-
- // Handle incoming event, and record its execution time.
- beforeExecution := time.Now().UTC()
- newEvents := hRec.handler.Handle(e)
- e.ExecInterval = time.Now().UTC().Sub(beforeExecution)
- // It's safe to check status of that node under 'hRec.lock'.
- if sch.stopper.ShouldStop(e.NodeID) {
- sch.cancelFunc()
- }
- return newEvents
- }()
- // Record executed events as history.
- func() {
- sch.historyLock.Lock()
- defer sch.historyLock.Unlock()
-
- e.HistoryIndex = len(sch.history)
- sch.history = append(sch.history, e)
- }()
- // Include the execution interval of parent event to the expected time
- // to execute child events.
- for _, newEvent := range newEvents {
- newEvent.ParentHistoryIndex = e.HistoryIndex
- newEvent.Time = newEvent.Time.Add(e.ExecInterval)
- }
- // Add derivated events back to event queue.
- func() {
- sch.eventsLock.Lock()
- defer sch.eventsLock.Unlock()
-
- for _, newEvent := range newEvents {
- sch.addEvent(newEvent)
- }
- }()
- }
-
-Done:
- for {
- // We favor scheduler-shutdown signal than other events.
- select {
- case <-sch.ctx.Done():
- break Done
- default:
- }
- // Block until new event arrival or scheduler shutdown.
- select {
- case <-sch.eventNotification:
- e := sch.nextTick()
- if e == nil {
- panic(errNilEventWhenNotified)
- }
- handleEvent(e)
- case <-sch.ctx.Done():
- break Done
- }
- }
-}
diff --git a/core/test/scheduler_test.go b/core/test/scheduler_test.go
deleted file mode 100644
index 4c39f95..0000000
--- a/core/test/scheduler_test.go
+++ /dev/null
@@ -1,176 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package test
-
-import (
- "sync"
- "testing"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core/types"
- "github.com/stretchr/testify/suite"
-)
-
-type SchedulerTestSuite struct {
- suite.Suite
-}
-
-type simpleStopper struct {
- lock sync.Mutex
- touched map[types.NodeID]int
- touchedCount int
-}
-
-func newSimpleStopper(
- nodes []types.NodeID, touchedCount int) *simpleStopper {
-
- touched := make(map[types.NodeID]int)
- for _, nID := range nodes {
- touched[nID] = 0
- }
- return &simpleStopper{
- touched: touched,
- touchedCount: touchedCount,
- }
-}
-
-func (stopper *simpleStopper) ShouldStop(nID types.NodeID) bool {
- stopper.lock.Lock()
- defer stopper.lock.Unlock()
-
- stopper.touched[nID] = stopper.touched[nID] + 1
- for _, count := range stopper.touched {
- if count < stopper.touchedCount {
- return false
- }
- }
- return true
-}
-
-type simpleHandler struct {
- count int
- nID types.NodeID
-}
-
-func (handler *simpleHandler) Handle(e *Event) (events []*Event) {
- if e.NodeID == handler.nID {
- handler.count++
- }
- return
-}
-
-type fixedLatencyHandler struct {
- nID types.NodeID
-}
-
-func (handler *fixedLatencyHandler) Handle(e *Event) (events []*Event) {
- // Simulate execution time.
- time.Sleep(500 * time.Millisecond)
- return []*Event{&Event{
- NodeID: handler.nID,
- Time: e.Time.Add(800 * time.Millisecond),
- }}
-}
-
-func (s *SchedulerTestSuite) TestEventSequence() {
- // This test case makes sure the event sequence is correctly increment
- // by their timestamps in 'Time' field.
- var (
- sch = NewScheduler(nil)
- req = s.Require()
- )
-
- req.NotNil(sch)
- now := time.Now()
- req.Nil(sch.Seed(&Event{Time: now.Add(100 * time.Second), Payload: 1}))
- req.Nil(sch.Seed(&Event{Time: now.Add(99 * time.Second), Payload: 2}))
- req.Nil(sch.Seed(&Event{Time: now.Add(98 * time.Second), Payload: 3}))
- req.Nil(sch.Seed(&Event{Time: now.Add(97 * time.Second), Payload: 4}))
- req.Nil(sch.Seed(&Event{Time: now.Add(96 * time.Second), Payload: 5}))
-
- req.Equal(sch.nextTick().Payload.(int), 5)
- req.Equal(sch.nextTick().Payload.(int), 4)
- req.Equal(sch.nextTick().Payload.(int), 3)
- req.Equal(sch.nextTick().Payload.(int), 2)
- req.Equal(sch.nextTick().Payload.(int), 1)
- req.Nil(sch.nextTick())
-}
-
-func (s *SchedulerTestSuite) TestBasicRound() {
- // This test case makes sure these facts:
- // - event is dispatched by NodeID attached to each handler.
- // - stopper can stop the execution when condition is met.
- var (
- req = s.Require()
- nodes = GenerateRandomNodeIDs(3)
- stopper = newSimpleStopper(nodes, 2)
- sch = NewScheduler(stopper)
- handlers = make(map[types.NodeID]*simpleHandler)
- )
-
- for _, nID := range nodes {
- handler := &simpleHandler{nID: nID}
- handlers[nID] = handler
- sch.RegisterEventHandler(nID, handler)
- req.Nil(sch.Seed(&Event{NodeID: nID}))
- req.Nil(sch.Seed(&Event{NodeID: nID}))
- }
- sch.Run(10)
- // Verify result.
- for _, h := range handlers {
- req.Equal(h.count, 2)
- }
-}
-
-func (s *SchedulerTestSuite) TestChildEvent() {
- // This test case makes sure these fields of child events are
- // assigned correctly.
- var (
- req = s.Require()
- nID = types.NodeID{Hash: common.NewRandomHash()}
- stopper = newSimpleStopper(types.NodeIDs{nID}, 3)
- handler = &fixedLatencyHandler{nID: nID}
- sch = NewScheduler(stopper)
- )
-
- sch.RegisterEventHandler(nID, handler)
- req.Nil(sch.Seed(&Event{
- NodeID: nID,
- Time: time.Now().UTC(),
- }))
- sch.Run(1)
- // Verify result.
- history := sch.CloneExecutionHistory()
- req.Len(history, 3)
- curEvent := history[0]
- for _, e := range history[1:] {
- // Make sure the time difference between events are more than
- // 1.3 second.
- req.True(e.Time.Sub(curEvent.Time) >= 1300*time.Millisecond)
- // Make sure ParentTime field is set and is equal to parent event's
- // time.
- req.NotEqual(-1, e.ParentHistoryIndex)
- req.Equal(e.ParentHistoryIndex, curEvent.HistoryIndex)
- curEvent = e
- }
-}
-
-func TestScheduler(t *testing.T) {
- suite.Run(t, new(SchedulerTestSuite))
-}
diff --git a/core/test/stopper.go b/core/test/stopper.go
deleted file mode 100644
index 2ba31d3..0000000
--- a/core/test/stopper.go
+++ /dev/null
@@ -1,140 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package test
-
-import (
- "sync"
-
- "github.com/dexon-foundation/dexon-consensus/core/db"
- "github.com/dexon-foundation/dexon-consensus/core/types"
-)
-
-// StopByConfirmedBlocks would make sure each nodes confirms
-// at least X blocks proposed by itself.
-type StopByConfirmedBlocks struct {
- apps map[types.NodeID]*App
- dbs map[types.NodeID]db.Database
- lastCheckDelivered map[types.NodeID]int
- confirmedBlocks map[types.NodeID]int
- blockCount int
- lock sync.Mutex
-}
-
-// NewStopByConfirmedBlocks construct an StopByConfirmedBlocks instance.
-func NewStopByConfirmedBlocks(
- blockCount int,
- apps map[types.NodeID]*App,
- dbs map[types.NodeID]db.Database) *StopByConfirmedBlocks {
- confirmedBlocks := make(map[types.NodeID]int)
- for nID := range apps {
- confirmedBlocks[nID] = 0
- }
- return &StopByConfirmedBlocks{
- apps: apps,
- dbs: dbs,
- lastCheckDelivered: make(map[types.NodeID]int),
- confirmedBlocks: confirmedBlocks,
- blockCount: blockCount,
- }
-}
-
-// ShouldStop implements Stopper interface.
-func (s *StopByConfirmedBlocks) ShouldStop(nID types.NodeID) bool {
- s.lock.Lock()
- defer s.lock.Unlock()
- // Accumulate confirmed blocks proposed by this node in this round.
- lastChecked := s.lastCheckDelivered[nID]
- currentConfirmedBlocks := s.confirmedBlocks[nID]
- dbInst := s.dbs[nID]
- s.apps[nID].WithLock(func(app *App) {
- for _, h := range app.DeliverSequence[lastChecked:] {
- b, err := dbInst.GetBlock(h)
- if err != nil {
- panic(err)
- }
- if b.ProposerID == nID {
- currentConfirmedBlocks++
- }
- }
- s.lastCheckDelivered[nID] = len(app.DeliverSequence)
- })
- s.confirmedBlocks[nID] = currentConfirmedBlocks
- // Check if all nodes confirmed at least 'blockCount' blocks.
- for _, v := range s.confirmedBlocks {
- if v < s.blockCount {
- return false
- }
- }
- return true
-}
-
-// StopByRound would make sure at least one block at round R is delivered
-// at each node.
-type StopByRound struct {
- untilRound uint64
- currentRounds map[types.NodeID]uint64
- lastCheckDelivered map[types.NodeID]int
- apps map[types.NodeID]*App
- dbs map[types.NodeID]db.Database
- lock sync.Mutex
-}
-
-// NewStopByRound constructs an StopByRound instance.
-func NewStopByRound(
- round uint64,
- apps map[types.NodeID]*App,
- dbs map[types.NodeID]db.Database) *StopByRound {
- return &StopByRound{
- untilRound: round,
- currentRounds: make(map[types.NodeID]uint64),
- lastCheckDelivered: make(map[types.NodeID]int),
- apps: apps,
- dbs: dbs,
- }
-}
-
-// ShouldStop implements Stopper interface.
-func (s *StopByRound) ShouldStop(nID types.NodeID) bool {
- s.lock.Lock()
- defer s.lock.Unlock()
- // Cache latest round of this node.
- if curRound := s.currentRounds[nID]; curRound < s.untilRound {
- lastChecked := s.lastCheckDelivered[nID]
- dbInst := s.dbs[nID]
- s.apps[nID].WithLock(func(app *App) {
- for _, h := range app.DeliverSequence[lastChecked:] {
- b, err := dbInst.GetBlock(h)
- if err != nil {
- panic(err)
- }
- if b.Position.Round > curRound {
- curRound = b.Position.Round
- }
- }
- s.lastCheckDelivered[nID] = len(app.DeliverSequence)
- s.currentRounds[nID] = curRound
- })
- }
- // Check if latest round on each node is later than untilRound.
- for _, round := range s.currentRounds {
- if round < s.untilRound {
- return false
- }
- }
- return true
-}
diff --git a/core/test/stopper_test.go b/core/test/stopper_test.go
deleted file mode 100644
index 758a0e4..0000000
--- a/core/test/stopper_test.go
+++ /dev/null
@@ -1,182 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package test
-
-import (
- "testing"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core"
- "github.com/dexon-foundation/dexon-consensus/core/db"
- "github.com/dexon-foundation/dexon-consensus/core/types"
- "github.com/stretchr/testify/suite"
-)
-
-type StopperTestSuite struct {
- suite.Suite
-}
-
-func (s *StopperTestSuite) deliver(
- blocks []*types.Block, app *App, dbInst db.Database) {
- hashes := common.Hashes{}
- for _, b := range blocks {
- hashes = append(hashes, b.Hash)
- s.Require().NoError(dbInst.PutBlock(*b))
- }
- for _, h := range hashes {
- app.BlockConfirmed(types.Block{Hash: h})
- }
- app.TotalOrderingDelivered(hashes, core.TotalOrderingModeNormal)
- for _, h := range hashes {
- app.BlockDelivered(h, types.Position{}, types.FinalizationResult{
- Timestamp: time.Time{},
- })
- }
-}
-
-func (s *StopperTestSuite) deliverToAllNodes(
- blocks []*types.Block,
- apps map[types.NodeID]*App,
- dbInsts map[types.NodeID]db.Database) {
- for nID := range apps {
- s.deliver(blocks, apps[nID], dbInsts[nID])
- }
-}
-
-func (s *StopperTestSuite) TestStopByConfirmedBlocks() {
- // This test case makes sure this stopper would stop when
- // all nodes confirmed at least 'x' count of blocks produced
- // by themselves.
- var (
- req = s.Require()
- apps = make(map[types.NodeID]*App)
- dbInsts = make(map[types.NodeID]db.Database)
- nodes = GenerateRandomNodeIDs(2)
- )
- for _, nID := range nodes {
- apps[nID] = NewApp(0, nil)
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
- dbInsts[nID] = dbInst
- }
- stopper := NewStopByConfirmedBlocks(2, apps, dbInsts)
- b00 := &types.Block{
- ProposerID: nodes[0],
- Hash: common.NewRandomHash(),
- }
- s.deliverToAllNodes([]*types.Block{b00}, apps, dbInsts)
- b10 := &types.Block{
- ProposerID: nodes[1],
- Hash: common.NewRandomHash(),
- }
- b11 := &types.Block{
- ProposerID: nodes[1],
- ParentHash: b10.Hash,
- Hash: common.NewRandomHash(),
- }
- s.deliverToAllNodes([]*types.Block{b10, b11}, apps, dbInsts)
- req.False(stopper.ShouldStop(nodes[1]))
- b12 := &types.Block{
- ProposerID: nodes[1],
- ParentHash: b11.Hash,
- Hash: common.NewRandomHash(),
- }
- s.deliverToAllNodes([]*types.Block{b12}, apps, dbInsts)
- req.False(stopper.ShouldStop(nodes[1]))
- b01 := &types.Block{
- ProposerID: nodes[0],
- ParentHash: b00.Hash,
- Hash: common.NewRandomHash(),
- }
- s.deliverToAllNodes([]*types.Block{b01}, apps, dbInsts)
- req.True(stopper.ShouldStop(nodes[0]))
-}
-
-func (s *StopperTestSuite) TestStopByRound() {
- // This test case make sure at least one block from round R
- // is delivered by each node.
- var (
- req = s.Require()
- apps = make(map[types.NodeID]*App)
- dbInsts = make(map[types.NodeID]db.Database)
- nodes = GenerateRandomNodeIDs(2)
- )
- for _, nID := range nodes {
- apps[nID] = NewApp(0, nil)
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
- dbInsts[nID] = dbInst
- }
- stopper := NewStopByRound(10, apps, dbInsts)
- b00 := &types.Block{
- ProposerID: nodes[0],
- Position: types.Position{
- Round: 0,
- ChainID: 0,
- Height: 0,
- },
- Hash: common.NewRandomHash(),
- }
- s.deliverToAllNodes([]*types.Block{b00}, apps, dbInsts)
- b10 := &types.Block{
- ProposerID: nodes[1],
- Position: types.Position{
- Round: 0,
- ChainID: 1,
- Height: 0,
- },
- Hash: common.NewRandomHash(),
- }
- b11 := &types.Block{
- ProposerID: nodes[1],
- ParentHash: b10.Hash,
- Position: types.Position{
- Round: 0,
- ChainID: 1,
- Height: 1,
- },
- Hash: common.NewRandomHash(),
- }
- s.deliverToAllNodes([]*types.Block{b10, b11}, apps, dbInsts)
- req.False(stopper.ShouldStop(nodes[0]))
- req.False(stopper.ShouldStop(nodes[1]))
- // Deliver one block at round 10 to node0
- b12 := &types.Block{
- ProposerID: nodes[1],
- ParentHash: b11.Hash,
- Position: types.Position{
- Round: 10,
- ChainID: 1,
- Height: 2,
- },
- Hash: common.NewRandomHash(),
- }
- // None should stop when only one node reach that round.
- s.deliver([]*types.Block{b12}, apps[nodes[0]], dbInsts[nodes[0]])
- req.False(stopper.ShouldStop(nodes[0]))
- req.False(stopper.ShouldStop(nodes[1]))
- // Everyone should stop now.
- s.deliver([]*types.Block{b12}, apps[nodes[1]], dbInsts[nodes[1]])
- req.True(stopper.ShouldStop(nodes[1]))
- req.True(stopper.ShouldStop(nodes[0]))
-}
-
-func TestStopper(t *testing.T) {
- suite.Run(t, new(StopperTestSuite))
-}
diff --git a/core/total-ordering-syncer.go b/core/total-ordering-syncer.go
deleted file mode 100644
index 1360611..0000000
--- a/core/total-ordering-syncer.go
+++ /dev/null
@@ -1,177 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package core
-
-import (
- "sort"
- "sync"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core/types"
-)
-
-type totalOrderingSyncer struct {
- lock sync.RWMutex
-
- numChains uint32
- syncHeight map[uint32]uint64
- syncDeliverySetIdx int
- pendingBlocks []*types.Block
- inPendingBlocks map[common.Hash]struct{}
-
- bootstrapChain map[uint32]struct{}
-
- // Data to restore delivery set.
- pendingDeliveryBlocks []*types.Block
- deliverySet map[int][]*types.Block
- mapToDeliverySet map[common.Hash]int
-}
-
-func newTotalOrderingSyncer(numChains uint32) *totalOrderingSyncer {
- return &totalOrderingSyncer{
- numChains: numChains,
- syncHeight: make(map[uint32]uint64),
- syncDeliverySetIdx: -1,
- inPendingBlocks: make(map[common.Hash]struct{}),
- bootstrapChain: make(map[uint32]struct{}),
- deliverySet: make(map[int][]*types.Block),
- mapToDeliverySet: make(map[common.Hash]int),
- }
-}
-
-func (tos *totalOrderingSyncer) synced() bool {
- tos.lock.RLock()
- defer tos.lock.RUnlock()
- return tos.syncDeliverySetIdx != -1
-}
-
-func (tos *totalOrderingSyncer) processBlock(
- block *types.Block) (delivered []*types.Block) {
- if tos.synced() {
- if tos.syncHeight[block.Position.ChainID] >= block.Position.Height {
- return
- }
- delivered = append(delivered, block)
- return
- }
- tos.lock.Lock()
- defer tos.lock.Unlock()
- tos.inPendingBlocks[block.Hash] = struct{}{}
- tos.pendingBlocks = append(tos.pendingBlocks, block)
- if block.Position.Height == 0 {
- tos.bootstrapChain[block.Position.ChainID] = struct{}{}
- }
- if uint32(len(tos.bootstrapChain)) == tos.numChains {
- // Bootstrap mode.
- delivered = tos.pendingBlocks
- tos.syncDeliverySetIdx = 0
- for i := uint32(0); i < tos.numChains; i++ {
- tos.syncHeight[i] = uint64(0)
- }
- } else {
- maxDeliverySetIdx := -1
- // TODO(jimmy-dexon): below for loop can be optimized.
- PendingBlockLoop:
- for i, block := range tos.pendingBlocks {
- idx, exist := tos.mapToDeliverySet[block.Hash]
- if !exist {
- continue
- }
- deliverySet := tos.deliverySet[idx]
- // Check if all the blocks in deliverySet are in the pendingBlocks.
- for _, dBlock := range deliverySet {
- if _, exist := tos.inPendingBlocks[dBlock.Hash]; !exist {
- continue PendingBlockLoop
- }
- }
- if idx > maxDeliverySetIdx {
- maxDeliverySetIdx = idx
- }
- // Check if all of the chains have delivered.
- for _, dBlock := range deliverySet {
- if h, exist := tos.syncHeight[dBlock.Position.ChainID]; exist {
- if dBlock.Position.Height < h {
- continue
- }
- }
- tos.syncHeight[dBlock.Position.ChainID] = dBlock.Position.Height
- }
- if uint32(len(tos.syncHeight)) != tos.numChains {
- continue
- }
- // Core is fully synced, it can start delivering blocks from idx.
- tos.syncDeliverySetIdx = maxDeliverySetIdx
- delivered = make([]*types.Block, 0, i)
- break
- }
- if tos.syncDeliverySetIdx == -1 {
- return
- }
- // Generating delivering blocks.
- for i := maxDeliverySetIdx; i < len(tos.deliverySet); i++ {
- deliverySet := tos.deliverySet[i]
- sort.Sort(types.ByHash(deliverySet))
- for _, block := range deliverySet {
- if block.Position.Height > tos.syncHeight[block.Position.ChainID] {
- tos.syncHeight[block.Position.ChainID] = block.Position.Height
- }
- delivered = append(delivered, block)
- }
- }
- // Flush remaining blocks.
- for _, block := range tos.pendingBlocks {
- if _, exist := tos.mapToDeliverySet[block.Hash]; exist {
- continue
- }
- if block.Position.Height > tos.syncHeight[block.Position.ChainID] {
- tos.syncHeight[block.Position.ChainID] = block.Position.Height
- }
- delivered = append(delivered, block)
- }
- }
- // Clean internal data model to save memory.
- tos.pendingBlocks = nil
- tos.inPendingBlocks = nil
- tos.bootstrapChain = nil
- tos.pendingDeliveryBlocks = nil
- tos.deliverySet = nil
- tos.mapToDeliverySet = nil
- return
-}
-
-// The finalized block should be passed by the order of consensus height.
-func (tos *totalOrderingSyncer) processFinalizedBlock(block *types.Block) {
- if tos.synced() {
- return
- }
- tos.lock.Lock()
- defer tos.lock.Unlock()
- if len(tos.pendingDeliveryBlocks) > 0 {
- if block.Hash.Less(
- tos.pendingDeliveryBlocks[len(tos.pendingDeliveryBlocks)-1].Hash) {
- // pendingDeliveryBlocks forms a deliverySet.
- idx := len(tos.deliverySet)
- tos.deliverySet[idx] = tos.pendingDeliveryBlocks
- for _, block := range tos.pendingDeliveryBlocks {
- tos.mapToDeliverySet[block.Hash] = idx
- }
- tos.pendingDeliveryBlocks = []*types.Block{}
- }
- }
- tos.pendingDeliveryBlocks = append(tos.pendingDeliveryBlocks, block)
-}
diff --git a/core/total-ordering-syncer_test.go b/core/total-ordering-syncer_test.go
deleted file mode 100644
index 4a4e8e0..0000000
--- a/core/total-ordering-syncer_test.go
+++ /dev/null
@@ -1,312 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package core
-
-import (
- "testing"
- "time"
-
- "github.com/stretchr/testify/suite"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core/db"
- "github.com/dexon-foundation/dexon-consensus/core/test"
- "github.com/dexon-foundation/dexon-consensus/core/types"
-)
-
-type TotalOrderingSyncerTestSuite struct {
- suite.Suite
-}
-
-func (s *TotalOrderingSyncerTestSuite) genDeliverySet(numChains uint32) (
- deliverySet [][]*types.Block, revealer *test.RandomDAGBlockRevealer) {
-
- genesisTime := time.Now().UTC()
- genesisConfig := &types.Config{
- K: 0,
- PhiRatio: 0.5,
- NumChains: numChains,
- RoundInterval: 1000 * time.Second,
- }
-
- to := newTotalOrdering(genesisTime, 0, genesisConfig)
-
- gen := test.NewBlocksGenerator(&test.BlocksGeneratorConfig{
- NumChains: numChains,
- MinBlockTimeInterval: 250 * time.Millisecond,
- }, nil)
-
- dbInst, err := db.NewMemBackedDB()
- s.Require().NoError(err)
- s.Require().NoError(gen.Generate(
- 0,
- genesisTime,
- genesisTime.Add(20*time.Second),
- dbInst))
- iter, err := dbInst.GetAllBlocks()
- s.Require().NoError(err)
-
- revealer, err = test.NewRandomDAGBlockRevealer(iter)
- s.Require().NoError(err)
-
- revealer.Reset()
- for {
- // Reveal next block.
- b, err := revealer.NextBlock()
- if err != nil {
- if err == db.ErrIterationFinished {
- err = nil
- break
- }
- }
- s.Require().NoError(err)
- // Perform total ordering.
- s.Require().NoError(to.addBlock(&b))
- for {
- blocks, _, err := to.extractBlocks()
- s.Require().NoError(err)
- if len(blocks) == 0 {
- break
- }
- if len(blocks) > 0 {
- deliverySet = append(deliverySet, blocks)
- }
- }
- }
- return
-}
-
-func (s *TotalOrderingSyncerTestSuite) TestRandomSync() {
- numChains := uint32(13)
- skipSet := 2
- skipDAG := int(numChains) * skipSet
- repeat := 100
- if testing.Short() {
- repeat = 10
- }
-
- for ; repeat >= 0; repeat-- {
- toc := newTotalOrderingSyncer(numChains)
- deliverySet, revealer := s.genDeliverySet(numChains)
- blockToDeliverySet := make(map[common.Hash]int)
- deliverySetMap := make(map[int][]common.Hash)
- offset := 0
- for i, delivery := range deliverySet {
- if i > 0 {
- // The hash of last block of previous delivery set is less than the hash
- // of first block of current delivery set. The syncer cannot seperate
- // these two delevery set so they need to be combined. This will not
- // affect the final result because the output of syncer is also sorted
- // and it will be the same as the output of total ordering.
- if deliverySet[i-1][len(deliverySet[i-1])-1].Hash.Less(
- delivery[0].Hash) {
- offset++
- }
- }
- for _, block := range delivery {
- blockToDeliverySet[block.Hash] = i - offset
- deliverySetMap[i-offset] = append(deliverySetMap[i-offset], block.Hash)
- }
- }
-
- revealer.Reset()
- for i := 0; i < skipDAG; i++ {
- _, err := revealer.NextBlock()
- s.Require().NoError(err)
- }
-
- for _, delivery := range deliverySet {
- for _, block := range delivery {
- toc.processFinalizedBlock(block)
- }
- }
-
- minDeliverySetIdx := -1
- deliverySetMap2 := make(map[int][]common.Hash)
-
- for {
- b, err := revealer.NextBlock()
- if err != nil {
- if err != db.ErrIterationFinished {
- s.Require().NoError(err)
- }
- err = nil
- break
- }
- deliver := toc.processBlock(&b)
- for _, block := range deliver {
- idx, exist := blockToDeliverySet[block.Hash]
- if !exist {
- continue
- }
- if minDeliverySetIdx == -1 {
- minDeliverySetIdx = idx
- }
- s.Require().True(idx >= minDeliverySetIdx)
- deliverySetMap2[idx] = append(deliverySetMap2[idx], block.Hash)
- }
- }
-
- s.Require().NotEqual(-1, minDeliverySetIdx)
- for i := minDeliverySetIdx; ; i++ {
- if _, exist := deliverySetMap[i]; !exist {
- break
- }
- for _, v := range deliverySetMap[i] {
- s.Contains(deliverySetMap2[i], v)
- }
- s.Require().Len(deliverySetMap2[i], len(deliverySetMap[i]))
- }
- }
-}
-
-// TestMissingMiddleDeliverySet tests the following case
-// The number denotes the index of delivery set.
-// X means that the block is not synced in lattice.
-// O means that the block is synced but the index is not important.
-// The assumption is that once the block is synced, all newer blocks
-// on the same chain will be synced as well.
-// ********************
-// O O O 5 6
-// 3 3 3 X 5
-// ------------
-// 0 1 2 3 4(ChainID)
-// ********************
-// In this case, the block of delivery set 4 is not synced in lattice;
-// therefore, the minimum index of delivery set should be 5 instead of 3.
-// (Note: delivery set 6 is to make syncer identify delivery set 5)
-
-func (s *TotalOrderingSyncerTestSuite) TestMissingMiddleDeliverySet() {
- numChains := uint32(5)
- b00 := &types.Block{
- Hash: common.Hash{0x10},
- Position: types.Position{
- ChainID: uint32(0),
- Height: uint64(3),
- },
- }
- b10 := &types.Block{
- Hash: common.Hash{0x20},
- Position: types.Position{
- ChainID: uint32(1),
- Height: uint64(3),
- },
- }
- b20 := &types.Block{
- Hash: common.Hash{0x30},
- Position: types.Position{
- ChainID: uint32(2),
- Height: uint64(3),
- },
- }
- b30 := &types.Block{
- Hash: common.Hash{0x21},
- Position: types.Position{
- ChainID: uint32(3),
- Height: uint64(3),
- },
- }
- b31 := &types.Block{
- Hash: common.Hash{0x12},
- Position: types.Position{
- ChainID: uint32(3),
- Height: uint64(4),
- },
- }
- b40 := &types.Block{
- Hash: common.Hash{0x22},
- Position: types.Position{
- ChainID: uint32(4),
- Height: uint64(3),
- },
- }
- b41 := &types.Block{
- Hash: common.Hash{0x12},
- Position: types.Position{
- ChainID: uint32(4),
- Height: uint64(4),
- },
- }
- blocks := []*types.Block{b00, b10, b20, b30, b31, b40, b41}
-
- // Test process sequence 1.
- toc := newTotalOrderingSyncer(numChains)
-
- for _, block := range blocks {
- toc.processFinalizedBlock(block)
- }
-
- s.Require().Len(toc.processBlock(b00), 0)
- s.Require().Len(toc.processBlock(b10), 0)
- s.Require().Len(toc.processBlock(b20), 0)
- s.Require().Len(toc.processBlock(b31), 0)
- deliver := toc.processBlock(b40)
- s.Require().Len(deliver, 2)
- s.Equal(deliver[0], b31)
- s.Equal(deliver[1], b40)
-
- // Test process sequence 2.
- toc2 := newTotalOrderingSyncer(numChains)
-
- for _, block := range blocks {
- toc2.processFinalizedBlock(block)
- }
-
- s.Require().Len(toc2.processBlock(b31), 0)
- s.Require().Len(toc2.processBlock(b40), 0)
- s.Require().Len(toc2.processBlock(b20), 0)
- s.Require().Len(toc2.processBlock(b10), 0)
- deliver = toc2.processBlock(b00)
- s.Require().Len(deliver, 2)
- s.Equal(deliver[0], b31)
- s.Equal(deliver[1], b40)
-
-}
-
-func (s *TotalOrderingSyncerTestSuite) TestBootstrap() {
- numChains := uint32(13)
- toc := newTotalOrderingSyncer(numChains)
- deliverySet, revealer := s.genDeliverySet(numChains)
- deliveredNum := 0
- for _, delivery := range deliverySet {
- deliveredNum += len(delivery)
- }
-
- actualDeliveredNum := 0
- revealer.Reset()
- for {
- b, err := revealer.NextBlock()
- if err != nil {
- if err != db.ErrIterationFinished {
- s.Require().NoError(err)
- }
- err = nil
- break
- }
- deliver := toc.processBlock(&b)
- actualDeliveredNum += len(deliver)
- }
-
- // The last few blocks revealer might not be in the output of total order.
- // So the deliveredNum might be less than actualDeliveredNum.
- s.True(actualDeliveredNum >= deliveredNum)
-}
-
-func TestTotalOrderingSyncer(t *testing.T) {
- suite.Run(t, new(TotalOrderingSyncerTestSuite))
-}
diff --git a/core/total-ordering.go b/core/total-ordering.go
deleted file mode 100644
index 2e2158e..0000000
--- a/core/total-ordering.go
+++ /dev/null
@@ -1,1321 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package core
-
-import (
- "errors"
- "fmt"
- "math"
- "sort"
- "sync"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core/types"
-)
-
-const (
- infinity uint64 = math.MaxUint64
-)
-
-const (
- // TotalOrderingModeError returns mode error.
- TotalOrderingModeError uint32 = iota
- // TotalOrderingModeNormal returns mode normal.
- TotalOrderingModeNormal
- // TotalOrderingModeEarly returns mode early.
- TotalOrderingModeEarly
- // TotalOrderingModeFlush returns mode flush.
- TotalOrderingModeFlush
-)
-
-var (
- // ErrInvalidDAG is reported when block subbmitted to totalOrdering
- // didn't form a DAG.
- ErrInvalidDAG = errors.New("invalid dag")
- // ErrFutureRoundDelivered means some blocks from later rounds are
- // delivered, this means program error.
- ErrFutureRoundDelivered = errors.New("future round delivered")
- // ErrBlockFromPastRound means we receive some block from past round.
- ErrBlockFromPastRound = errors.New("block from past round")
- // ErrTotalOrderingHangs means total ordering hangs somewhere.
- ErrTotalOrderingHangs = errors.New("total ordering hangs")
- // ErrForwardAck means a block acking some blocks from newer round.
- ErrForwardAck = errors.New("forward ack")
- // ErrUnexpected means general (I'm lazy) errors.
- ErrUnexpected = errors.New("unexpected")
- // ErrTotalOrderingPhiRatio means invalid phi ratio
- ErrTotalOrderingPhiRatio = errors.New("invalid total ordering phi ratio")
-)
-
-// totalOrderingConfig is the configuration for total ordering.
-type totalOrderingConfig struct {
- roundBasedConfig
- // k represents the k in 'k-level total ordering'.
- // In short, only block height equals to (global minimum height + k)
- // would be taken into consideration.
- k uint64
-
- // phi is a const to control how strong the leading preceding block
- // should be.
- phi uint64
-
- numChains uint32
- isFlushRequired bool
-}
-
-func (config *totalOrderingConfig) fromConfig(round uint64, cfg *types.Config) {
- config.k = uint64(cfg.K)
- config.numChains = cfg.NumChains
- config.phi = uint64(float32(cfg.NumChains-1)*cfg.PhiRatio + 1)
- config.setupRoundBasedFields(round, cfg)
-}
-
-func newTotalOrderingConfig(
- prev *totalOrderingConfig, cur *types.Config) *totalOrderingConfig {
- c := &totalOrderingConfig{}
- c.fromConfig(prev.roundID+1, cur)
- c.setRoundBeginTime(prev.roundEndTime)
- prev.isFlushRequired = c.k != prev.k ||
- c.phi != prev.phi ||
- c.numChains != prev.numChains
- return c
-}
-
-// totalOrderingWinRecord caches the comparison of candidates calculated by
-// their height vector.
-type totalOrderingWinRecord struct {
- wins []int8
- count uint
-}
-
-func (rec *totalOrderingWinRecord) reset() {
- rec.count = 0
- for idx := range rec.wins {
- rec.wins[idx] = 0
- }
-}
-
-func newTotalOrderingWinRecord(numChains uint32) *totalOrderingWinRecord {
- return &totalOrderingWinRecord{
- wins: make([]int8, numChains),
- count: 0,
- }
-}
-
-// grade implements the 'grade' potential function in algorithm.
-func (rec *totalOrderingWinRecord) grade(
- numChains uint32, phi uint64, globalAnsLength uint64) int {
- if uint64(rec.count) >= phi {
- return 1
- } else if uint64(rec.count) < phi-uint64(numChains)+globalAnsLength {
- return 0
- } else {
- return -1
- }
-}
-
-// totalOrderingHeightRecord records:
-// - the minimum height of block which acks this block.
-// - the count of blocks acking this block.
-type totalOrderingHeightRecord struct{ minHeight, count uint64 }
-
-// totalOrderingCache caches objects for reuse and not being colloected by GC.
-// Each cached target has "get-" and "put-" functions for getting and reusing
-// of objects.
-type totalOrderingCache struct {
- ackedStatus [][]*totalOrderingHeightRecord
- heightVectors [][]uint64
- winRecords [][]*totalOrderingWinRecord
- winRecordPool sync.Pool
- ackedVectors []map[common.Hash]struct{}
- numChains uint32
-}
-
-// newTotalOrderingObjectCache constructs an totalOrderingCache instance.
-func newTotalOrderingObjectCache(numChains uint32) *totalOrderingCache {
- return &totalOrderingCache{
- winRecordPool: sync.Pool{
- New: func() interface{} {
- return newTotalOrderingWinRecord(numChains)
- },
- },
- numChains: numChains,
- }
-}
-
-// resize makes sure internal storage of totalOrdering instance can handle
-// maximum possible numChains in future configs.
-func (cache *totalOrderingCache) resize(numChains uint32) {
- // Basically, everything in cache needs to be cleaned.
- if cache.numChains >= numChains {
- return
- }
- cache.ackedStatus = nil
- cache.heightVectors = nil
- cache.winRecords = nil
- cache.ackedVectors = nil
- cache.numChains = numChains
- cache.winRecordPool = sync.Pool{
- New: func() interface{} {
- return newTotalOrderingWinRecord(numChains)
- },
- }
-}
-
-func (cache *totalOrderingCache) getAckedStatus() (
- acked []*totalOrderingHeightRecord) {
-
- if len(cache.ackedStatus) == 0 {
- acked = make([]*totalOrderingHeightRecord, cache.numChains)
- for idx := range acked {
- acked[idx] = &totalOrderingHeightRecord{count: 0}
- }
- } else {
- acked = cache.ackedStatus[len(cache.ackedStatus)-1]
- cache.ackedStatus = cache.ackedStatus[:len(cache.ackedStatus)-1]
- // Reset acked status.
- for idx := range acked {
- acked[idx].count = 0
- }
- }
- return
-}
-
-func (cache *totalOrderingCache) putAckedStatus(
- acked []*totalOrderingHeightRecord) {
- // If the recycled objects supports lower numChains than we required,
- // don't recycle it.
- if uint32(len(acked)) != cache.numChains {
- return
- }
- cache.ackedStatus = append(cache.ackedStatus, acked)
-}
-
-func (cache *totalOrderingCache) getWinRecord() (
- win *totalOrderingWinRecord) {
- win = cache.winRecordPool.Get().(*totalOrderingWinRecord)
- win.reset()
- return
-}
-
-func (cache *totalOrderingCache) putWinRecord(win *totalOrderingWinRecord) {
- if win == nil {
- return
- }
- // If the recycled objects supports lower numChains than we required,
- // don't recycle it.
- if uint32(len(win.wins)) != cache.numChains {
- return
- }
- cache.winRecordPool.Put(win)
-}
-
-func (cache *totalOrderingCache) getHeightVector() (hv []uint64) {
- if len(cache.heightVectors) == 0 {
- hv = make([]uint64, cache.numChains)
- } else {
- hv = cache.heightVectors[len(cache.heightVectors)-1]
- cache.heightVectors = cache.heightVectors[:len(cache.heightVectors)-1]
- }
- for idx := range hv {
- hv[idx] = infinity
- }
- return
-}
-
-func (cache *totalOrderingCache) putHeightVector(hv []uint64) {
- if uint32(len(hv)) != cache.numChains {
- return
- }
- cache.heightVectors = append(cache.heightVectors, hv)
-}
-
-func (cache *totalOrderingCache) getWinRecords() (w []*totalOrderingWinRecord) {
- if len(cache.winRecords) == 0 {
- w = make([]*totalOrderingWinRecord, cache.numChains)
- } else {
- w = cache.winRecords[len(cache.winRecords)-1]
- cache.winRecords = cache.winRecords[:len(cache.winRecords)-1]
- for idx := range w {
- w[idx] = nil
- }
- }
- return
-}
-
-func (cache *totalOrderingCache) putWinRecords(w []*totalOrderingWinRecord) {
- if uint32(len(w)) != cache.numChains {
- return
- }
- cache.winRecords = append(cache.winRecords, w)
-}
-
-func (cache *totalOrderingCache) getAckedVector() (
- acked map[common.Hash]struct{}) {
- if len(cache.ackedVectors) == 0 {
- acked = make(map[common.Hash]struct{})
- } else {
- acked, cache.ackedVectors =
- cache.ackedVectors[len(cache.ackedVectors)-1],
- cache.ackedVectors[:len(cache.ackedVectors)-1]
- for k := range acked {
- delete(acked, k)
- }
- }
- return
-}
-
-func (cache *totalOrderingCache) putAckedVector(
- acked map[common.Hash]struct{}) {
- if acked != nil {
- cache.ackedVectors = append(cache.ackedVectors, acked)
- }
-}
-
-// totalOrderingCandidateInfo stores proceding status for a candidate including
-// - acked status as height records, which keeps the number of blocks from other
-// chains acking this candidate.
-// - cached height vector, which valids height based on K-level used for
-// comparison in 'grade' function.
-// - cached result of grade function to other candidates.
-//
-// Height Record:
-// When block A acks block B, all blocks proposed from the same proposer as
-// block A with higher height also acks block B. Thus records below is needed
-// - the minimum height of acking block from that proposer
-// - count of acking blocks from that proposer
-// to repsent the acking status for block A.
-type totalOrderingCandidateInfo struct {
- ackedStatus []*totalOrderingHeightRecord
- cachedHeightVector []uint64
- winRecords []*totalOrderingWinRecord
- hash common.Hash
-}
-
-// newTotalOrderingCandidateInfo creates an totalOrderingCandidateInfo instance.
-func newTotalOrderingCandidateInfo(
- candidateHash common.Hash,
- objCache *totalOrderingCache) *totalOrderingCandidateInfo {
- return &totalOrderingCandidateInfo{
- ackedStatus: objCache.getAckedStatus(),
- winRecords: objCache.getWinRecords(),
- hash: candidateHash,
- }
-}
-
-// clean clears information related to another candidate, which should be called
-// when that candidate is selected in deliver set.
-func (v *totalOrderingCandidateInfo) clean(otherCandidateChainID uint32) {
- v.winRecords[otherCandidateChainID] = nil
-}
-
-// recycle recycles objects for later usage, this eases GC's work.
-func (v *totalOrderingCandidateInfo) recycle(objCache *totalOrderingCache) {
- if v.winRecords != nil {
- for _, win := range v.winRecords {
- objCache.putWinRecord(win)
- }
- objCache.putWinRecords(v.winRecords)
- }
- if v.cachedHeightVector != nil {
- objCache.putHeightVector(v.cachedHeightVector)
- }
- objCache.putAckedStatus(v.ackedStatus)
-}
-
-// addBlock would update totalOrderingCandidateInfo, it's caller's duty
-// to make sure the input block acutally acking the target block.
-func (v *totalOrderingCandidateInfo) addBlock(b *types.Block) error {
- rec := v.ackedStatus[b.Position.ChainID]
- if rec.count == 0 {
- rec.minHeight = b.Position.Height
- rec.count = 1
- } else {
- if b.Position.Height <= rec.minHeight {
- return ErrInvalidDAG
- }
- rec.count++
- }
- return nil
-}
-
-// getAckingNodeSetLength returns the size of acking node set. Only heights
-// larger than "global minimum height + k" are counted. For example, global
-// minimum acking height is 1 and k is 1, only block heights which is larger or
-// equal to 2 are added into acking node set.
-func (v *totalOrderingCandidateInfo) getAckingNodeSetLength(
- global *totalOrderingCandidateInfo,
- k uint64,
- numChains uint32) (count uint64) {
-
- var rec *totalOrderingHeightRecord
- for idx, gRec := range global.ackedStatus[:numChains] {
- if gRec.count == 0 {
- continue
- }
- rec = v.ackedStatus[idx]
- if rec.count == 0 {
- continue
- }
- if rec.minHeight+rec.count-1 >= gRec.minHeight+k {
- count++
- }
- }
- return
-}
-
-// updateAckingHeightVector would cached acking height vector.
-//
-// Only block height equals to (global minimum block height + k) would be
-// taken into consideration.
-func (v *totalOrderingCandidateInfo) updateAckingHeightVector(
- global *totalOrderingCandidateInfo,
- k uint64,
- dirtyChainIDs []int,
- objCache *totalOrderingCache) {
-
- var (
- idx int
- gRec, rec *totalOrderingHeightRecord
- )
- // The reason for not merging two loops is that the performance impact of map
- // iteration is large if the size is large. Iteration of dirty chains is
- // faster the map.
- // TODO(mission): merge the code in this if/else if the performance won't be
- // downgraded when adding a function for the shared part.
- if v.cachedHeightVector == nil {
- // Generate height vector from scratch.
- v.cachedHeightVector = objCache.getHeightVector()
- for idx, gRec = range global.ackedStatus {
- if gRec.count <= k {
- continue
- }
- rec = v.ackedStatus[idx]
- if rec.count == 0 {
- v.cachedHeightVector[idx] = infinity
- } else if rec.minHeight <= gRec.minHeight+k {
- // This check is sufficient to make sure the block height:
- //
- // gRec.minHeight + k
- //
- // would be included in this totalOrderingCandidateInfo.
- v.cachedHeightVector[idx] = gRec.minHeight + k
- } else {
- v.cachedHeightVector[idx] = infinity
- }
- }
- } else {
- // Return the cached one, only update dirty fields.
- for _, idx = range dirtyChainIDs {
- gRec = global.ackedStatus[idx]
- if gRec.count == 0 || gRec.count <= k {
- v.cachedHeightVector[idx] = infinity
- continue
- }
- rec = v.ackedStatus[idx]
- if rec.count == 0 {
- v.cachedHeightVector[idx] = infinity
- } else if rec.minHeight <= gRec.minHeight+k {
- v.cachedHeightVector[idx] = gRec.minHeight + k
- } else {
- v.cachedHeightVector[idx] = infinity
- }
- }
- }
- return
-}
-
-// updateWinRecord setups win records from two candidates.
-func (v *totalOrderingCandidateInfo) updateWinRecord(
- otherChainID uint32,
- other *totalOrderingCandidateInfo,
- dirtyChainIDs []int,
- objCache *totalOrderingCache,
- numChains uint32) {
- var (
- idx int
- height uint64
- )
- // The reason not to merge two loops is that the iteration of map is
- // expensive when chain count is large, iterating of dirty chains is cheaper.
- // TODO(mission): merge the code in this if/else if adding a function won't
- // affect the performance.
- win := v.winRecords[otherChainID]
- if win == nil {
- win = objCache.getWinRecord()
- v.winRecords[otherChainID] = win
- for idx, height = range v.cachedHeightVector[:numChains] {
- if height == infinity {
- continue
- }
- if other.cachedHeightVector[idx] == infinity {
- win.wins[idx] = 1
- win.count++
- }
- }
- } else {
- for _, idx = range dirtyChainIDs {
- if v.cachedHeightVector[idx] == infinity {
- if win.wins[idx] == 1 {
- win.wins[idx] = 0
- win.count--
- }
- continue
- }
- if other.cachedHeightVector[idx] == infinity {
- if win.wins[idx] == 0 {
- win.wins[idx] = 1
- win.count++
- }
- } else {
- if win.wins[idx] == 1 {
- win.wins[idx] = 0
- win.count--
- }
- }
- }
- }
-}
-
-// totalOrderingBreakpoint is a record of height discontinuity on a chain
-type totalOrderingBreakpoint struct {
- roundID uint64
- // height of last block.
- lastHeight uint64
-}
-
-// totalOrderingGroupVector keeps global status of current pending set.
-type totalOrderingGlobalVector struct {
- // blocks stores all blocks grouped by their proposers and sorted by height.
- // TODO(mission): slice used here reallocates frequently.
- blocks [][]*types.Block
-
- // breakpoints stores rounds for chains that blocks' height on them are
- // not consecutive, for example in chain i
- // Round Height
- // 0 0
- // 0 1
- // 1 2
- // 1 3
- // 1 4
- // 2 - <- a config change of chain number occured
- // 2 -
- // 3 -
- // 3 -
- // 4 0 <- a breakpoint for round 3 is cached here
- // 5 -
- // 5 -
- // 6 0 <- breakpoint again
- // breakpoints[i][0] == &totalOrderingBreakpoint{roundID: 4, lastHeight: 4}
- // breakpoints[i][1] == &totalOrderingBreakpoint{roundID: 6, lastHeight: 0}
- breakpoints [][]*totalOrderingBreakpoint
-
- // curRound stores the last round ID used for purging breakpoints.
- curRound uint64
-
- // tips records the last seen block for each chain.
- tips []*types.Block
-
- // Only ackedStatus in cachedCandidateInfo is used.
- cachedCandidateInfo *totalOrderingCandidateInfo
-}
-
-func newTotalOrderingGlobalVector(
- initRound uint64, numChains uint32) *totalOrderingGlobalVector {
- return &totalOrderingGlobalVector{
- blocks: make([][]*types.Block, numChains),
- tips: make([]*types.Block, numChains),
- breakpoints: make([][]*totalOrderingBreakpoint, numChains),
- curRound: initRound,
- }
-}
-
-func (global *totalOrderingGlobalVector) resize(numChains uint32) {
- if len(global.blocks) >= int(numChains) {
- return
- }
- // Resize blocks.
- newBlocks := make([][]*types.Block, numChains)
- copy(newBlocks, global.blocks)
- global.blocks = newBlocks
- // Resize breakpoints.
- newBreakPoints := make([][]*totalOrderingBreakpoint, numChains)
- copy(newBreakPoints, global.breakpoints)
- global.breakpoints = newBreakPoints
- // Resize tips.
- newTips := make([]*types.Block, numChains)
- copy(newTips, global.tips)
- global.tips = newTips
-}
-
-func (global *totalOrderingGlobalVector) switchRound(roundID uint64) {
- if global.curRound+1 != roundID {
- panic(ErrUnexpected)
- }
- global.curRound = roundID
- for chainID, bs := range global.breakpoints {
- if len(bs) == 0 {
- continue
- }
- if bs[0].roundID == roundID {
- global.breakpoints[chainID] = bs[1:]
- }
- }
-}
-
-func (global *totalOrderingGlobalVector) prepareHeightRecord(
- candidate *types.Block,
- info *totalOrderingCandidateInfo,
- acked map[common.Hash]struct{}) {
-
- var (
- chainID = candidate.Position.ChainID
- breakpoints = global.breakpoints[chainID]
- breakpoint *totalOrderingBreakpoint
- rec *totalOrderingHeightRecord
- )
- // Setup height record for own chain.
- rec = &totalOrderingHeightRecord{
- minHeight: candidate.Position.Height,
- }
- if len(breakpoints) == 0 {
- // If no breakpoint, count is the amount of blocks.
- rec.count = uint64(len(global.blocks[chainID]))
- } else {
- // If there are breakpoints, only the first counts.
- rec.count = breakpoints[0].lastHeight - candidate.Position.Height + 1
- }
- info.ackedStatus[chainID] = rec
- if acked == nil {
- return
- }
- for idx, blocks := range global.blocks {
- if idx == int(chainID) {
- continue
- }
- breakpoint = nil
- if len(global.breakpoints[idx]) > 0 {
- breakpoint = global.breakpoints[idx][0]
- }
- for i, b := range blocks {
- if breakpoint != nil && b.Position.Round >= breakpoint.roundID {
- break
- }
- if _, acked := acked[b.Hash]; !acked {
- continue
- }
- // If this block acks the candidate, all newer blocks from the same chain
- // also 'indirectly' acks the candidate.
- rec = info.ackedStatus[idx]
- rec.minHeight = b.Position.Height
- if breakpoint == nil {
- rec.count = uint64(len(blocks) - i)
- } else {
- rec.count = breakpoint.lastHeight - b.Position.Height + 1
- }
- break
- }
- }
-}
-
-func (global *totalOrderingGlobalVector) addBlock(
- b *types.Block) (isOldest bool, pending bool, err error) {
- // isOldest implies the block is the oldest in global vector
- chainID := b.Position.ChainID
- tip := global.tips[chainID]
- isOldest = len(global.blocks[chainID]) == 0
- if tip != nil {
- // Perform light weight sanity check based on tip.
- if tip.Position.Round > b.Position.Round {
- err = ErrInvalidDAG
- return
- }
- if DiffUint64(tip.Position.Round, b.Position.Round) > 1 {
- if b.Position.Height != 0 {
- err = ErrInvalidDAG
- return
- }
- // Add breakpoint.
- if b.Position.Round > global.curRound {
- global.breakpoints[chainID] = append(
- global.breakpoints[chainID],
- &totalOrderingBreakpoint{
- roundID: b.Position.Round,
- lastHeight: tip.Position.Height,
- })
- }
- } else {
- if b.Position.Height != tip.Position.Height+1 {
- err = ErrInvalidDAG
- return
- }
- }
- } else {
- if b.Position.Round < global.curRound {
- err = ErrBlockFromPastRound
- return
- }
- if b.Position.Round > global.curRound {
- // Add breakpoint.
- bp := &totalOrderingBreakpoint{
- roundID: b.Position.Round,
- lastHeight: 0,
- }
- global.breakpoints[chainID] = append(global.breakpoints[chainID], bp)
- }
- }
- bps := global.breakpoints[chainID]
- pending = len(bps) > 0 && bps[0].roundID <= b.Position.Round
- global.blocks[chainID] = append(global.blocks[chainID], b)
- global.tips[chainID] = b
- return
-}
-
-// updateCandidateInfo udpates cached candidate info.
-func (global *totalOrderingGlobalVector) updateCandidateInfo(
- dirtyChainIDs []int, objCache *totalOrderingCache) {
- var (
- idx int
- blocks []*types.Block
- block *types.Block
- info *totalOrderingCandidateInfo
- rec *totalOrderingHeightRecord
- breakpoint *totalOrderingBreakpoint
- )
- if global.cachedCandidateInfo == nil {
- info = newTotalOrderingCandidateInfo(common.Hash{}, objCache)
- for idx, blocks = range global.blocks {
- if len(blocks) == 0 {
- continue
- }
- rec = info.ackedStatus[idx]
- if len(global.breakpoints[idx]) > 0 {
- breakpoint = global.breakpoints[idx][0]
- block = blocks[0]
- if block.Position.Round >= breakpoint.roundID {
- continue
- }
- rec.minHeight = block.Position.Height
- rec.count = breakpoint.lastHeight - block.Position.Height + 1
- } else {
- rec.minHeight = blocks[0].Position.Height
- rec.count = uint64(len(blocks))
- }
- }
- global.cachedCandidateInfo = info
- } else {
- info = global.cachedCandidateInfo
- for _, idx = range dirtyChainIDs {
- blocks = global.blocks[idx]
- if len(blocks) == 0 {
- info.ackedStatus[idx].count = 0
- continue
- }
- rec = info.ackedStatus[idx]
- if len(global.breakpoints[idx]) > 0 {
- breakpoint = global.breakpoints[idx][0]
- block = blocks[0]
- if block.Position.Round >= breakpoint.roundID {
- continue
- }
- rec.minHeight = block.Position.Height
- rec.count = breakpoint.lastHeight - block.Position.Height + 1
- } else {
- rec.minHeight = blocks[0].Position.Height
- rec.count = uint64(len(blocks))
- }
- }
- }
- return
-}
-
-// totalOrdering represent a process unit to handle total ordering for blocks.
-type totalOrdering struct {
- // pendings stores blocks awaiting to be ordered.
- pendings map[common.Hash]*types.Block
-
- // The round of config used when performing total ordering.
- curRound uint64
-
- // duringFlush is a flag to switch the flush mode and normal mode.
- duringFlush bool
-
- // flushReadyChains checks if the last block of that chain arrived. Once
- // last blocks from all chains in current config are arrived, we can
- // perform flush.
- flushReadyChains map[uint32]struct{}
-
- // flushed is a map of flushed blocks.
- flushed map[uint32]struct{}
-
- // globalVector group all pending blocks by proposers and
- // sort them by block height. This structure is helpful when:
- //
- // - build global height vector
- // - picking candidates next round
- globalVector *totalOrderingGlobalVector
-
- // candidates caches result of potential function during generating preceding
- // set.
- candidates []*totalOrderingCandidateInfo
-
- // acked stores the 'block A acked by block B' by acked[A.Hash][B.Hash]
- acked map[common.Hash]map[common.Hash]struct{}
-
- // dirtyChainIDs stores chainIDs that is "dirty", i.e. needed updating all
- // cached statuses (win record, acking status).
- dirtyChainIDs []int
-
- // objCache caches allocated objects, like map.
- objCache *totalOrderingCache
-
- // candidateChainMapping keeps a mapping from candidate's hash to
- // their chain IDs.
- candidateChainMapping map[uint32]common.Hash
-
- // candidateChainIDs records chain ID of all candidates.
- candidateChainIDs []uint32
-
- // configs keeps configuration for each round in continuous way.
- configs []*totalOrderingConfig
-}
-
-// newTotalOrdering constructs an totalOrdering instance.
-func newTotalOrdering(
- dMoment time.Time, round uint64, cfg *types.Config) *totalOrdering {
- config := &totalOrderingConfig{}
- config.fromConfig(round, cfg)
- config.setRoundBeginTime(dMoment)
- candidates := make([]*totalOrderingCandidateInfo, config.numChains)
- to := &totalOrdering{
- pendings: make(map[common.Hash]*types.Block),
- dirtyChainIDs: make([]int, 0, config.numChains),
- acked: make(map[common.Hash]map[common.Hash]struct{}),
- objCache: newTotalOrderingObjectCache(config.numChains),
- candidateChainMapping: make(map[uint32]common.Hash),
- candidates: candidates,
- candidateChainIDs: make([]uint32, 0, config.numChains),
- curRound: config.roundID,
- globalVector: newTotalOrderingGlobalVector(
- config.roundID, config.numChains),
- }
- to.configs = []*totalOrderingConfig{config}
- return to
-}
-
-// appendConfig add new configs for upcoming rounds. If you add a config for
-// round R, next time you can only add the config for round R+1.
-func (to *totalOrdering) appendConfig(
- round uint64, config *types.Config) error {
-
- if round != uint64(len(to.configs))+to.configs[0].roundID {
- return ErrRoundNotIncreasing
- }
- if config.PhiRatio < 0.5 || config.PhiRatio > 1.0 {
- return ErrTotalOrderingPhiRatio
- }
- to.configs = append(
- to.configs,
- newTotalOrderingConfig(to.configs[len(to.configs)-1], config))
- // Resize internal structures.
- to.globalVector.resize(config.NumChains)
- to.objCache.resize(config.NumChains)
- if int(config.NumChains) > len(to.candidates) {
- newCandidates := make([]*totalOrderingCandidateInfo, config.NumChains)
- copy(newCandidates, to.candidates)
- to.candidates = newCandidates
- }
- return nil
-}
-
-func (to *totalOrdering) switchRound() {
- to.curRound++
- to.globalVector.switchRound(to.curRound)
-}
-
-// buildBlockRelation update all its indirect acks recursively.
-func (to *totalOrdering) buildBlockRelation(b *types.Block) {
- var (
- curBlock, nextBlock *types.Block
- ack common.Hash
- acked map[common.Hash]struct{}
- exists, alreadyPopulated bool
- toCheck = []*types.Block{b}
- )
- for len(toCheck) != 0 {
- curBlock, toCheck = toCheck[len(toCheck)-1], toCheck[:len(toCheck)-1]
- if curBlock.Position.Round > b.Position.Round {
- // It's illegal for a block to ack some blocks in future round.
- panic(ErrForwardAck)
- }
- for _, ack = range curBlock.Acks {
- if acked, exists = to.acked[ack]; !exists {
- acked = to.objCache.getAckedVector()
- to.acked[ack] = acked
- }
- // Check if the block is handled.
- if _, alreadyPopulated = acked[b.Hash]; alreadyPopulated {
- continue
- }
- acked[b.Hash] = struct{}{}
- // See if we need to do this recursively.
- if nextBlock, exists = to.pendings[ack]; exists {
- toCheck = append(toCheck, nextBlock)
- }
- }
- }
-}
-
-// clean a block from working set. This behaviour would prevent
-// our memory usage growing infinity.
-func (to *totalOrdering) clean(b *types.Block) {
- var (
- h = b.Hash
- chainID = b.Position.ChainID
- )
- to.objCache.putAckedVector(to.acked[h])
- delete(to.acked, h)
- delete(to.pendings, h)
- to.candidates[chainID].recycle(to.objCache)
- to.candidates[chainID] = nil
- delete(to.candidateChainMapping, chainID)
- // Remove this candidate from candidate IDs.
- to.candidateChainIDs =
- removeFromSortedUint32Slice(to.candidateChainIDs, chainID)
- // Clear records of this candidate from other candidates.
- for _, idx := range to.candidateChainIDs {
- to.candidates[idx].clean(chainID)
- }
-}
-
-// updateVectors is a helper function to update all cached vectors.
-func (to *totalOrdering) updateVectors(
- b *types.Block) (isOldest bool, err error) {
- var (
- candidateHash common.Hash
- chainID uint32
- acked bool
- pending bool
- )
- // Update global height vector
- if isOldest, pending, err = to.globalVector.addBlock(b); err != nil {
- return
- }
- if to.duringFlush {
- // It makes no sense to calculate potential functions of total ordering
- // when flushing would be happened.
- return
- }
- if pending {
- // The chain of this block contains breakpoints, which means their
- // height are not continuous. This implementation of DEXON total
- // ordering algorithm assumes the height of blocks in working set should
- // be continuous.
- //
- // To workaround this issue, when block arrived after breakpoints,
- // their information would not be contributed to current working set.
- // This mechanism works because we switch rounds by flushing and
- // reset the whole working set.
- // This works because forward acking blocks are rejected.
- return
- }
- // Update candidates' acking status.
- for chainID, candidateHash = range to.candidateChainMapping {
- if _, acked = to.acked[candidateHash][b.Hash]; !acked {
- continue
- }
- if err = to.candidates[chainID].addBlock(b); err != nil {
- return
- }
- }
- return
-}
-
-// prepareCandidate builds totalOrderingCandidateInfo for a new candidate.
-func (to *totalOrdering) prepareCandidate(b *types.Block) {
- var (
- info = newTotalOrderingCandidateInfo(b.Hash, to.objCache)
- chainID = b.Position.ChainID
- )
- to.candidates[chainID] = info
- to.candidateChainMapping[chainID] = b.Hash
- // Add index to slot to allocated list, make sure the modified list is sorted.
- to.candidateChainIDs = append(to.candidateChainIDs, chainID)
- sort.Slice(to.candidateChainIDs, func(i, j int) bool {
- return to.candidateChainIDs[i] < to.candidateChainIDs[j]
- })
- to.globalVector.prepareHeightRecord(b, info, to.acked[b.Hash])
- return
-}
-
-// isCandidate checks if a block only contains acks to delivered blocks.
-func (to *totalOrdering) isCandidate(b *types.Block) bool {
- for _, ack := range b.Acks {
- if _, exists := to.pendings[ack]; exists {
- return false
- }
- }
- return true
-}
-
-// output finishes the delivery of preceding set.
-func (to *totalOrdering) output(
- precedings map[common.Hash]struct{},
- numChains uint32) (ret []*types.Block) {
-
- for p := range precedings {
- // Remove the first element from corresponding blockVector.
- b := to.pendings[p]
- chainID := b.Position.ChainID
- // TODO(mission): frequent reallocation here.
- to.globalVector.blocks[chainID] = to.globalVector.blocks[chainID][1:]
- ret = append(ret, b)
- // Remove block relations.
- to.clean(b)
- to.dirtyChainIDs = append(to.dirtyChainIDs, int(chainID))
- }
- sort.Sort(types.ByHash(ret))
- // Find new candidates from global vector's tips.
- // The complexity here is O(N^2logN).
- // TODO(mission): only tips which acking some blocks in the devliered set
- // should be checked. This improvement related to the latency introduced by K.
- for chainID, blocks := range to.globalVector.blocks[:numChains] {
- if len(blocks) == 0 {
- continue
- }
- if _, picked := to.candidateChainMapping[uint32(chainID)]; picked {
- continue
- }
- if !to.isCandidate(blocks[0]) {
- continue
- }
- // Build totalOrderingCandidateInfo for new candidate.
- to.prepareCandidate(blocks[0])
- }
- return
-}
-
-// generateDeliverSet generates preceding set and checks if the preceding set
-// is deliverable by potential function.
-func (to *totalOrdering) generateDeliverSet() (
- delivered map[common.Hash]struct{}, mode uint32) {
-
- var (
- chainID, otherChainID uint32
- info, otherInfo *totalOrderingCandidateInfo
- precedings = make(map[uint32]struct{})
- cfg = to.getCurrentConfig()
- )
- mode = TotalOrderingModeNormal
- to.globalVector.updateCandidateInfo(to.dirtyChainIDs, to.objCache)
- globalInfo := to.globalVector.cachedCandidateInfo
- for _, chainID = range to.candidateChainIDs {
- to.candidates[chainID].updateAckingHeightVector(
- globalInfo, cfg.k, to.dirtyChainIDs, to.objCache)
- }
- // Update winning records for each candidate.
- // TODO(mission): It's not reasonable to request one routine for each
- // candidate, the context switch rate would be high.
- var wg sync.WaitGroup
- wg.Add(len(to.candidateChainIDs))
- for _, chainID := range to.candidateChainIDs {
- info = to.candidates[chainID]
- go func(can uint32, canInfo *totalOrderingCandidateInfo) {
- defer wg.Done()
- for _, otherChainID := range to.candidateChainIDs {
- if can == otherChainID {
- continue
- }
- canInfo.updateWinRecord(
- otherChainID,
- to.candidates[otherChainID],
- to.dirtyChainIDs,
- to.objCache,
- cfg.numChains)
- }
- }(chainID, info)
- }
- wg.Wait()
- // Reset dirty chains.
- to.dirtyChainIDs = to.dirtyChainIDs[:0]
- globalAnsLength := globalInfo.getAckingNodeSetLength(
- globalInfo, cfg.k, cfg.numChains)
-CheckNextCandidateLoop:
- for _, chainID = range to.candidateChainIDs {
- info = to.candidates[chainID]
- for _, otherChainID = range to.candidateChainIDs {
- if chainID == otherChainID {
- continue
- }
- otherInfo = to.candidates[otherChainID]
- // TODO(mission): grade should be bounded by current numChains.
- if otherInfo.winRecords[chainID].grade(
- cfg.numChains, cfg.phi, globalAnsLength) != 0 {
- continue CheckNextCandidateLoop
- }
- }
- precedings[chainID] = struct{}{}
- }
- if len(precedings) == 0 {
- return
- }
- // internal is a helper function to verify internal stability.
- internal := func() bool {
- var (
- isPreceding, beaten bool
- p uint32
- )
- for _, chainID = range to.candidateChainIDs {
- if _, isPreceding = precedings[chainID]; isPreceding {
- continue
- }
- beaten = false
- for p = range precedings {
- // TODO(mission): grade should be bound by current numChains.
- if beaten = to.candidates[p].winRecords[chainID].grade(
- cfg.numChains, cfg.phi, globalAnsLength) == 1; beaten {
- break
- }
- }
- if !beaten {
- return false
- }
- }
- return true
- }
- // checkAHV is a helper function to verify external stability.
- // It would make sure some preceding block is strong enough
- // to lead the whole preceding set.
- checkAHV := func() bool {
- var (
- height, count uint64
- p uint32
- )
- for p = range precedings {
- count = 0
- info = to.candidates[p]
- for _, height = range info.cachedHeightVector {
- if height != infinity {
- count++
- if count > cfg.phi {
- return true
- }
- }
- }
- }
- return false
- }
- // checkANS is a helper function to verify external stability.
- // It would make sure all preceding blocks are strong enough
- // to be delivered.
- checkANS := func() bool {
- var chainAnsLength uint64
- for p := range precedings {
- chainAnsLength = to.candidates[p].getAckingNodeSetLength(
- globalInfo, cfg.k, cfg.numChains)
- if uint64(chainAnsLength) < uint64(cfg.numChains)-cfg.phi {
- return false
- }
- }
- return true
- }
- // If all chains propose enough blocks, we should force
- // to deliver since the whole picture of the DAG is revealed.
- if globalAnsLength != uint64(cfg.numChains) {
- // Check internal stability first.
- if !internal() {
- return
- }
-
- // The whole picture is not ready, we need to check if
- // exteranl stability is met, and we can deliver earlier.
- if checkAHV() && checkANS() {
- mode = TotalOrderingModeEarly
- } else {
- return
- }
- }
- delivered = make(map[common.Hash]struct{})
- for p := range precedings {
- delivered[to.candidates[p].hash] = struct{}{}
- }
- return
-}
-
-// flushBlocks flushes blocks.
-func (to *totalOrdering) flushBlocks() (
- flushed []*types.Block, mode uint32, err error) {
- mode = TotalOrderingModeFlush
- cfg := to.getCurrentConfig()
-
- // Flush blocks until last blocks from all chains appeared.
- if len(to.flushReadyChains) < int(cfg.numChains) {
- return
- }
- if len(to.flushReadyChains) > int(cfg.numChains) {
- // This case should never be occured.
- err = ErrFutureRoundDelivered
- return
- }
- // Dump all blocks in this round.
- for len(to.flushed) != int(cfg.numChains) {
- // Dump all candidates without checking potential function.
- flushedHashes := make(map[common.Hash]struct{})
- for _, chainID := range to.candidateChainIDs {
- candidateBlock := to.pendings[to.candidates[chainID].hash]
- if candidateBlock.Position.Round > to.curRound {
- continue
- }
- flushedHashes[candidateBlock.Hash] = struct{}{}
- }
- if len(flushedHashes) == 0 {
- err = ErrTotalOrderingHangs
- return
- }
- flushedBlocks := to.output(flushedHashes, cfg.numChains)
- for _, b := range flushedBlocks {
- if cfg.isLastBlock(b) {
- to.flushed[b.Position.ChainID] = struct{}{}
- }
- }
- flushed = append(flushed, flushedBlocks...)
- }
- // Switch back to non-flushing mode.
- to.duringFlush = false
- to.flushed = make(map[uint32]struct{})
- to.flushReadyChains = make(map[uint32]struct{})
- // Clean all cached intermediate stats.
- for idx := range to.candidates {
- if to.candidates[idx] == nil {
- continue
- }
- to.candidates[idx].recycle(to.objCache)
- to.candidates[idx] = nil
- }
- to.dirtyChainIDs = nil
- to.candidateChainMapping = make(map[uint32]common.Hash)
- to.candidateChainIDs = nil
- to.globalVector.cachedCandidateInfo = nil
- to.switchRound()
- // Force picking new candidates.
- numChains := to.getCurrentConfig().numChains
- to.output(map[common.Hash]struct{}{}, numChains)
- return
-}
-
-// deliverBlocks delivers blocks by DEXON total ordering algorithm.
-func (to *totalOrdering) deliverBlocks() (
- delivered []*types.Block, mode uint32, err error) {
-
- hashes, mode := to.generateDeliverSet()
- cfg := to.getCurrentConfig()
- // Output precedings.
- delivered = to.output(hashes, cfg.numChains)
- // Check if any block in delivered set is the last block in this round, if
- // there is, perform flush or round-switch.
- for _, b := range delivered {
- if b.Position.Round > to.curRound {
- err = ErrFutureRoundDelivered
- return
- }
- if !cfg.isLastBlock(b) {
- continue
- }
- // Code reaches here if a last block is processed. This triggers
- // "duringFlush" mode if config changes.
- if cfg.isFlushRequired {
- // Switch to flush mode.
- to.duringFlush = true
- to.flushReadyChains = make(map[uint32]struct{})
- to.flushed = make(map[uint32]struct{})
- } else {
- // Switch round directly.
- to.switchRound()
- }
- break
- }
- if to.duringFlush {
- // Collect last blocks until all last blocks appears and function
- // flushBlocks will be called.
- for _, b := range delivered {
- if cfg.isLastBlock(b) {
- to.flushed[b.Position.ChainID] = struct{}{}
- }
- }
- // Some last blocks for the round to be flushed might not be delivered
- // yet.
- for _, tip := range to.globalVector.tips[:cfg.numChains] {
- if tip.Position.Round > to.curRound || cfg.isLastBlock(tip) {
- to.flushReadyChains[tip.Position.ChainID] = struct{}{}
- }
- }
- }
- return
-}
-
-func (to *totalOrdering) getCurrentConfig() *totalOrderingConfig {
- cfgIdx := to.curRound - to.configs[0].roundID
- if cfgIdx >= uint64(len(to.configs)) {
- panic(fmt.Errorf("total ordering config is not ready: %v, %v, %v",
- to.curRound, to.configs[0].roundID, len(to.configs)))
- }
- return to.configs[cfgIdx]
-}
-
-// addBlock adds a block to the working set of total ordering module.
-func (to *totalOrdering) addBlock(b *types.Block) error {
- // NOTE: Block b is assumed to be in topologically sorted, i.e., all its
- // acking blocks are during or after total ordering stage.
- cfg := to.getCurrentConfig()
- to.pendings[b.Hash] = b
- to.buildBlockRelation(b)
- isOldest, err := to.updateVectors(b)
- if err != nil {
- return err
- }
- // Mark the proposer of incoming block as dirty.
- if b.Position.ChainID < cfg.numChains {
- to.dirtyChainIDs = append(to.dirtyChainIDs, int(b.Position.ChainID))
- _, exists := to.candidateChainMapping[b.Position.ChainID]
- if isOldest && !exists && to.isCandidate(b) {
- // isOldest means b is the oldest block in global vector, and isCandidate
- // is still needed here due to round change. For example:
- // o o o <- genesis block for round change, isCandidate returns true
- // | | but isOldest is false
- // o o
- // | |
- // o o o <- isOldest is true but isCandidate returns false
- // | | /
- // o o
- to.prepareCandidate(b)
- }
- }
- if to.duringFlush && cfg.isLastBlock(b) {
- to.flushReadyChains[b.Position.ChainID] = struct{}{}
- }
- return nil
-}
-
-// extractBlocks check if there is any deliverable set.
-func (to *totalOrdering) extractBlocks() ([]*types.Block, uint32, error) {
- if to.duringFlush {
- return to.flushBlocks()
- }
- return to.deliverBlocks()
-}
diff --git a/core/total-ordering_test.go b/core/total-ordering_test.go
deleted file mode 100644
index 24ad646..0000000
--- a/core/total-ordering_test.go
+++ /dev/null
@@ -1,1453 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package core
-
-import (
- "sort"
- "strings"
- "testing"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core/db"
- "github.com/dexon-foundation/dexon-consensus/core/test"
- "github.com/dexon-foundation/dexon-consensus/core/types"
- "github.com/stretchr/testify/suite"
-)
-
-type TotalOrderingTestSuite struct {
- suite.Suite
-}
-
-func (s *TotalOrderingTestSuite) genGenesisBlock(
- vIDs types.NodeIDs,
- chainID uint32,
- acks common.Hashes) *types.Block {
-
- return &types.Block{
- ProposerID: vIDs[chainID],
- ParentHash: common.Hash{},
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 0,
- ChainID: chainID,
- },
- Acks: common.NewSortedHashes(acks),
- }
-}
-
-func (s *TotalOrderingTestSuite) performOneRun(
- to *totalOrdering, revealer test.BlockRevealer) (revealed, ordered string) {
- revealer.Reset()
- curRound := uint64(0)
- revealedDAG := make(map[common.Hash]struct{})
- for {
- // Reveal next block.
- b, err := revealer.NextBlock()
- if err != nil {
- if err == db.ErrIterationFinished {
- err = nil
- break
- }
- }
- s.Require().NoError(err)
- revealed += b.Hash.String() + ","
- // Perform total ordering.
- s.Require().NoError(to.addBlock(&b))
- for {
- blocks, mode, err := to.extractBlocks()
- s.Require().NoError(err)
- if len(blocks) == 0 {
- break
- }
- for _, b := range blocks {
- ordered += b.Hash.String() + ","
- // Make sure the round ID is increasing, and no interleave.
- s.Require().True(b.Position.Round >= curRound)
- curRound = b.Position.Round
- // Make sure all acking blocks are already delivered.
- for _, ack := range b.Acks {
- s.Require().Contains(revealedDAG, ack)
- }
- if mode == TotalOrderingModeFlush {
- // For blocks delivered by flushing, the acking relations
- // would exist in one deliver set, however, only later block
- // would ack previous block, not backward.
- revealedDAG[b.Hash] = struct{}{}
- }
- }
- // For blocks not delivered by flushing, the acking relations only
- // exist between deliver sets.
- if mode != TotalOrderingModeFlush {
- for _, b := range blocks {
- revealedDAG[b.Hash] = struct{}{}
- }
- }
- }
- }
- return
-}
-
-func (s *TotalOrderingTestSuite) checkRandomResult(
- revealingSequence, orderingSequence map[string]struct{}) {
- // Make sure we test at least two different
- // revealing sequence.
- s.True(len(revealingSequence) > 1)
- // Make sure all ordering are equal or prefixed
- // to another one.
- for orderFrom := range orderingSequence {
- s.True(len(orderFrom) > 0)
- for orderTo := range orderingSequence {
- if orderFrom == orderTo {
- continue
- }
- ok := strings.HasPrefix(orderFrom, orderTo) ||
- strings.HasPrefix(orderTo, orderFrom)
- s.True(ok)
- }
- }
-}
-
-func (s *TotalOrderingTestSuite) checkNotDeliver(to *totalOrdering, b *types.Block) {
- err := to.addBlock(b)
- s.NoError(err)
- blocks, mode, err := to.extractBlocks()
- s.Empty(blocks)
- s.Equal(mode, TotalOrderingModeNormal)
- s.Nil(err)
-}
-
-func (s *TotalOrderingTestSuite) checkHashSequence(blocks []*types.Block, hashes common.Hashes) {
- sort.Sort(hashes)
- for i, h := range hashes {
- s.Equal(blocks[i].Hash, h)
- }
-}
-
-func (s *TotalOrderingTestSuite) checkNotInWorkingSet(
- to *totalOrdering, b *types.Block) {
-
- s.NotContains(to.pendings, b.Hash)
- s.NotContains(to.acked, b.Hash)
-}
-
-func (s *TotalOrderingTestSuite) TestBlockRelation() {
- // This test case would verify if 'acking' and 'acked'
- // accumulated correctly.
- //
- // The DAG used below is:
- // A <- B <- C
- nodes := test.GenerateRandomNodeIDs(5)
- vID := nodes[0]
- blockA := s.genGenesisBlock(nodes, 0, common.Hashes{})
- blockB := &types.Block{
- ProposerID: vID,
- ParentHash: blockA.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 1,
- ChainID: 0,
- },
- Acks: common.NewSortedHashes(common.Hashes{blockA.Hash}),
- }
- blockC := &types.Block{
- ProposerID: vID,
- ParentHash: blockB.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 2,
- ChainID: 0,
- },
- Acks: common.NewSortedHashes(common.Hashes{blockB.Hash}),
- }
-
- genesisConfig := &types.Config{
- RoundInterval: 1000 * time.Second,
- K: 1,
- PhiRatio: 0.6,
- NumChains: uint32(len(nodes)),
- }
- genesisTime := time.Now().UTC()
- to := newTotalOrdering(genesisTime, 0, genesisConfig)
- s.checkNotDeliver(to, blockA)
- s.checkNotDeliver(to, blockB)
- s.checkNotDeliver(to, blockC)
-
- // Check 'acked'.
- ackedA := to.acked[blockA.Hash]
- s.Require().NotNil(ackedA)
- s.Len(ackedA, 2)
- s.Contains(ackedA, blockB.Hash)
- s.Contains(ackedA, blockC.Hash)
-
- ackedB := to.acked[blockB.Hash]
- s.Require().NotNil(ackedB)
- s.Len(ackedB, 1)
- s.Contains(ackedB, blockC.Hash)
-
- s.Nil(to.acked[blockC.Hash])
-}
-
-func (s *TotalOrderingTestSuite) TestCreateAckingHeightVectorFromHeightVector() {
- var (
- cache = newTotalOrderingObjectCache(5)
- dirties = []int{0, 1, 2, 3, 4}
- )
- // Prepare global acking status.
- global := &totalOrderingCandidateInfo{
- ackedStatus: []*totalOrderingHeightRecord{
- &totalOrderingHeightRecord{minHeight: 0, count: 5},
- &totalOrderingHeightRecord{minHeight: 0, count: 5},
- &totalOrderingHeightRecord{minHeight: 0, count: 5},
- &totalOrderingHeightRecord{minHeight: 0, count: 5},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- }}
-
- // For 'not existed' record in local but exist in global,
- // should be infinity.
- candidate := &totalOrderingCandidateInfo{
- ackedStatus: []*totalOrderingHeightRecord{
- &totalOrderingHeightRecord{minHeight: 0, count: 2},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- }}
- candidate.updateAckingHeightVector(global, 0, dirties, cache)
- s.Equal(candidate.cachedHeightVector[0], uint64(0))
- s.Equal(candidate.cachedHeightVector[1], infinity)
- s.Equal(candidate.cachedHeightVector[2], infinity)
- s.Equal(candidate.cachedHeightVector[3], infinity)
-
- // For local min exceeds global's min+k-1, should be infinity
- candidate = &totalOrderingCandidateInfo{
- ackedStatus: []*totalOrderingHeightRecord{
- &totalOrderingHeightRecord{minHeight: 3, count: 1},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- }}
- candidate.updateAckingHeightVector(global, 2, dirties, cache)
- s.Equal(candidate.cachedHeightVector[0], infinity)
- candidate.updateAckingHeightVector(global, 3, dirties, cache)
- s.Equal(candidate.cachedHeightVector[0], uint64(3))
-
- candidate = &totalOrderingCandidateInfo{
- ackedStatus: []*totalOrderingHeightRecord{
- &totalOrderingHeightRecord{minHeight: 0, count: 3},
- &totalOrderingHeightRecord{minHeight: 0, count: 3},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- }}
- candidate.updateAckingHeightVector(global, 5, dirties, cache)
-}
-
-func (s *TotalOrderingTestSuite) TestCreateAckingNodeSetFromHeightVector() {
- global := &totalOrderingCandidateInfo{
- ackedStatus: []*totalOrderingHeightRecord{
- &totalOrderingHeightRecord{minHeight: 0, count: 5},
- &totalOrderingHeightRecord{minHeight: 0, count: 5},
- &totalOrderingHeightRecord{minHeight: 0, count: 5},
- &totalOrderingHeightRecord{minHeight: 0, count: 5},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- }}
-
- local := &totalOrderingCandidateInfo{
- ackedStatus: []*totalOrderingHeightRecord{
- &totalOrderingHeightRecord{minHeight: 1, count: 2},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- &totalOrderingHeightRecord{minHeight: 0, count: 0},
- }}
- s.Equal(local.getAckingNodeSetLength(global, 1, 5), uint64(1))
- s.Equal(local.getAckingNodeSetLength(global, 2, 5), uint64(1))
- s.Equal(local.getAckingNodeSetLength(global, 3, 5), uint64(0))
-}
-
-func (s *TotalOrderingTestSuite) TestGrade() {
- // This test case just fake some internal structure used
- // when performing total ordering.
- var (
- nodes = test.GenerateRandomNodeIDs(5)
- cache = newTotalOrderingObjectCache(5)
- dirtyNodes = []int{0, 1, 2, 3, 4}
- )
- ansLength := uint64(len(map[types.NodeID]struct{}{
- nodes[0]: struct{}{},
- nodes[1]: struct{}{},
- nodes[2]: struct{}{},
- nodes[3]: struct{}{},
- }))
- candidate1 := newTotalOrderingCandidateInfo(common.Hash{}, cache)
- candidate1.cachedHeightVector = []uint64{
- 1, infinity, infinity, infinity, infinity}
- candidate2 := newTotalOrderingCandidateInfo(common.Hash{}, cache)
- candidate2.cachedHeightVector = []uint64{
- 1, 1, 1, 1, infinity}
- candidate3 := newTotalOrderingCandidateInfo(common.Hash{}, cache)
- candidate3.cachedHeightVector = []uint64{
- 1, 1, infinity, infinity, infinity}
-
- candidate2.updateWinRecord(
- 0, candidate1, dirtyNodes, cache, 5)
- s.Equal(candidate2.winRecords[0].grade(5, 3, ansLength), 1)
- candidate1.updateWinRecord(
- 1, candidate2, dirtyNodes, cache, 5)
- s.Equal(candidate1.winRecords[1].grade(5, 3, ansLength), 0)
- candidate2.updateWinRecord(
- 2, candidate3, dirtyNodes, cache, 5)
- s.Equal(candidate2.winRecords[2].grade(5, 3, ansLength), -1)
- candidate3.updateWinRecord(
- 1, candidate2, dirtyNodes, cache, 5)
- s.Equal(candidate3.winRecords[1].grade(5, 3, ansLength), 0)
-}
-
-func (s *TotalOrderingTestSuite) TestCycleDetection() {
- // Make sure we don't get hang by cycle from
- // block's acks.
- nodes := test.GenerateRandomNodeIDs(5)
-
- // create blocks with cycles in acking relation.
- cycledHash := common.NewRandomHash()
- b00 := s.genGenesisBlock(nodes, 0, common.Hashes{cycledHash})
- b01 := &types.Block{
- ProposerID: nodes[0],
- ParentHash: b00.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 1,
- ChainID: 0,
- },
- Acks: common.NewSortedHashes(common.Hashes{b00.Hash}),
- }
- b02 := &types.Block{
- ProposerID: nodes[0],
- ParentHash: b01.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 2,
- ChainID: 0,
- },
- Acks: common.NewSortedHashes(common.Hashes{b01.Hash}),
- }
- b03 := &types.Block{
- ProposerID: nodes[0],
- ParentHash: b02.Hash,
- Hash: cycledHash,
- Position: types.Position{
- Height: 3,
- ChainID: 0,
- },
- Acks: common.NewSortedHashes(common.Hashes{b02.Hash}),
- }
-
- // Create a block acks self.
- b10 := s.genGenesisBlock(nodes, 1, common.Hashes{})
- b10.Acks = append(b10.Acks, b10.Hash)
-
- // Make sure we won't hang when cycle exists.
- genesisConfig := &types.Config{
- RoundInterval: 1000 * time.Second,
- K: 1,
- PhiRatio: 0.6,
- NumChains: uint32(len(nodes)),
- }
- genesisTime := time.Now().UTC()
- to := newTotalOrdering(genesisTime, 0, genesisConfig)
- s.checkNotDeliver(to, b00)
- s.checkNotDeliver(to, b01)
- s.checkNotDeliver(to, b02)
-
- // Should not hang in this line.
- s.checkNotDeliver(to, b03)
- // Should not hang in this line
- s.checkNotDeliver(to, b10)
-}
-
-func (s *TotalOrderingTestSuite) TestEarlyDeliver() {
- // The test scenario:
- //
- // o o o o o
- // : : : : : <- (K - 1) layers
- // o o o o o
- // \ v / |
- // o o
- // A B
- // Even when B is not received, A should
- // be able to be delivered.
- nodes := test.GenerateRandomNodeIDs(5)
- genesisConfig := &types.Config{
- RoundInterval: 1000 * time.Second,
- K: 2,
- PhiRatio: 0.6,
- NumChains: uint32(len(nodes)),
- }
- genesisTime := time.Now().UTC()
- to := newTotalOrdering(genesisTime, 0, genesisConfig)
- genNextBlock := func(b *types.Block) *types.Block {
- return &types.Block{
- ProposerID: b.ProposerID,
- ParentHash: b.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: b.Position.Height + 1,
- ChainID: b.Position.ChainID,
- },
- Acks: common.NewSortedHashes(common.Hashes{b.Hash}),
- }
- }
-
- b00 := s.genGenesisBlock(nodes, 0, common.Hashes{})
- b01 := genNextBlock(b00)
- b02 := genNextBlock(b01)
-
- b10 := s.genGenesisBlock(nodes, 1, common.Hashes{b00.Hash})
- b11 := genNextBlock(b10)
- b12 := genNextBlock(b11)
-
- b20 := s.genGenesisBlock(nodes, 2, common.Hashes{b00.Hash})
- b21 := genNextBlock(b20)
- b22 := genNextBlock(b21)
-
- b30 := s.genGenesisBlock(nodes, 3, common.Hashes{b00.Hash})
- b31 := genNextBlock(b30)
- b32 := genNextBlock(b31)
-
- // It's a valid block sequence to deliver
- // to total ordering algorithm: DAG.
- s.checkNotDeliver(to, b00)
- s.checkNotDeliver(to, b01)
- s.checkNotDeliver(to, b02)
-
- candidate := to.candidates[0]
- s.Require().NotNil(candidate)
- s.Equal(candidate.ackedStatus[0].minHeight,
- b00.Position.Height)
- s.Equal(candidate.ackedStatus[0].count, uint64(3))
-
- s.checkNotDeliver(to, b10)
- s.checkNotDeliver(to, b11)
- s.checkNotDeliver(to, b12)
- s.checkNotDeliver(to, b20)
- s.checkNotDeliver(to, b21)
- s.checkNotDeliver(to, b22)
- s.checkNotDeliver(to, b30)
- s.checkNotDeliver(to, b31)
-
- // Check the internal state before delivering.
- s.Len(to.candidateChainMapping, 1) // b00 is the only candidate.
-
- candidate = to.candidates[0]
- s.Require().NotNil(candidate)
- s.Equal(candidate.ackedStatus[0].minHeight, b00.Position.Height)
- s.Equal(candidate.ackedStatus[0].count, uint64(3))
- s.Equal(candidate.ackedStatus[1].minHeight, b10.Position.Height)
- s.Equal(candidate.ackedStatus[1].count, uint64(3))
- s.Equal(candidate.ackedStatus[2].minHeight, b20.Position.Height)
- s.Equal(candidate.ackedStatus[2].count, uint64(3))
- s.Equal(candidate.ackedStatus[3].minHeight, b30.Position.Height)
- s.Equal(candidate.ackedStatus[3].count, uint64(2))
-
- s.Require().NoError(to.addBlock(b32))
- blocks, mode, err := to.extractBlocks()
- s.Require().Len(blocks, 1)
- s.Equal(mode, TotalOrderingModeEarly)
- s.Nil(err)
- s.checkHashSequence(blocks, common.Hashes{b00.Hash})
-
- // Check the internal state after delivered.
- s.Len(to.candidateChainMapping, 4) // b01, b10, b20, b30 are candidates.
-
- // Check b01.
- candidate = to.candidates[0]
- s.Require().NotNil(candidate)
- s.Equal(candidate.ackedStatus[0].minHeight, b01.Position.Height)
- s.Equal(candidate.ackedStatus[0].count, uint64(2))
-
- // Check b10.
- candidate = to.candidates[1]
- s.Require().NotNil(candidate)
- s.Equal(candidate.ackedStatus[1].minHeight, b10.Position.Height)
- s.Equal(candidate.ackedStatus[1].count, uint64(3))
-
- // Check b20.
- candidate = to.candidates[2]
- s.Require().NotNil(candidate)
- s.Equal(candidate.ackedStatus[2].minHeight, b20.Position.Height)
- s.Equal(candidate.ackedStatus[2].count, uint64(3))
-
- // Check b30.
- candidate = to.candidates[3]
- s.Require().NotNil(candidate)
- s.Equal(candidate.ackedStatus[3].minHeight, b30.Position.Height)
- s.Equal(candidate.ackedStatus[3].count, uint64(3))
-
- // Make sure b00 doesn't exist in current working set:
- s.checkNotInWorkingSet(to, b00)
-}
-
-func (s *TotalOrderingTestSuite) TestBasicCaseForK2() {
- // It's a handcrafted test case.
- nodes := test.GenerateRandomNodeIDs(5)
- genesisConfig := &types.Config{
- RoundInterval: 1000 * time.Second,
- K: 2,
- PhiRatio: 0.6,
- NumChains: uint32(len(nodes)),
- }
- genesisTime := time.Now().UTC()
- to := newTotalOrdering(genesisTime, 0, genesisConfig)
- // Setup blocks.
- b00 := s.genGenesisBlock(nodes, 0, common.Hashes{})
- b10 := s.genGenesisBlock(nodes, 1, common.Hashes{})
- b20 := s.genGenesisBlock(nodes, 2, common.Hashes{b10.Hash})
- b30 := s.genGenesisBlock(nodes, 3, common.Hashes{b20.Hash})
- b40 := s.genGenesisBlock(nodes, 4, common.Hashes{})
- b11 := &types.Block{
- ProposerID: nodes[1],
- ParentHash: b10.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 1,
- ChainID: 1,
- },
- Acks: common.NewSortedHashes(common.Hashes{b10.Hash, b00.Hash}),
- }
- b01 := &types.Block{
- ProposerID: nodes[0],
- ParentHash: b00.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 1,
- ChainID: 0,
- },
- Acks: common.NewSortedHashes(common.Hashes{b00.Hash, b11.Hash}),
- }
- b21 := &types.Block{
- ProposerID: nodes[2],
- ParentHash: b20.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 1,
- ChainID: 2,
- },
- Acks: common.NewSortedHashes(common.Hashes{b20.Hash, b01.Hash}),
- }
- b31 := &types.Block{
- ProposerID: nodes[3],
- ParentHash: b30.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 1,
- ChainID: 3,
- },
- Acks: common.NewSortedHashes(common.Hashes{b30.Hash, b21.Hash}),
- }
- b02 := &types.Block{
- ProposerID: nodes[0],
- ParentHash: b01.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 2,
- ChainID: 0,
- },
- Acks: common.NewSortedHashes(common.Hashes{b01.Hash, b21.Hash}),
- }
- b12 := &types.Block{
- ProposerID: nodes[1],
- ParentHash: b11.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 2,
- ChainID: 1,
- },
- Acks: common.NewSortedHashes(common.Hashes{b11.Hash, b21.Hash}),
- }
- b32 := &types.Block{
- ProposerID: nodes[3],
- ParentHash: b31.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 2,
- ChainID: 3,
- },
- Acks: common.NewSortedHashes(common.Hashes{b31.Hash}),
- }
- b22 := &types.Block{
- ProposerID: nodes[2],
- ParentHash: b21.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 2,
- ChainID: 2,
- },
- Acks: common.NewSortedHashes(common.Hashes{b21.Hash, b32.Hash}),
- }
- b23 := &types.Block{
- ProposerID: nodes[2],
- ParentHash: b22.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 3,
- ChainID: 2,
- },
- Acks: common.NewSortedHashes(common.Hashes{b22.Hash}),
- }
- b03 := &types.Block{
- ProposerID: nodes[0],
- ParentHash: b02.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 3,
- ChainID: 0,
- },
- Acks: common.NewSortedHashes(common.Hashes{b02.Hash, b22.Hash}),
- }
- b13 := &types.Block{
- ProposerID: nodes[1],
- ParentHash: b12.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 3,
- ChainID: 1,
- },
- Acks: common.NewSortedHashes(common.Hashes{b12.Hash, b22.Hash}),
- }
- b14 := &types.Block{
- ProposerID: nodes[1],
- ParentHash: b13.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 4,
- ChainID: 1,
- },
- Acks: common.NewSortedHashes(common.Hashes{b13.Hash}),
- }
- b41 := &types.Block{
- ProposerID: nodes[4],
- ParentHash: b40.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 1,
- ChainID: 4,
- },
- Acks: common.NewSortedHashes(common.Hashes{b40.Hash}),
- }
- b42 := &types.Block{
- ProposerID: nodes[4],
- ParentHash: b41.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 2,
- ChainID: 4,
- },
- Acks: common.NewSortedHashes(common.Hashes{b41.Hash}),
- }
-
- s.checkNotDeliver(to, b00)
- s.checkNotDeliver(to, b10)
- s.checkNotDeliver(to, b11)
- s.checkNotDeliver(to, b01)
- s.checkNotDeliver(to, b20)
- s.checkNotDeliver(to, b30)
- s.checkNotDeliver(to, b21)
- s.checkNotDeliver(to, b31)
- s.checkNotDeliver(to, b32)
- s.checkNotDeliver(to, b22)
- s.checkNotDeliver(to, b12)
-
- // Make sure 'acked' for current precedings is correct.
- acked := to.acked[b00.Hash]
- s.Require().NotNil(acked)
- s.Len(acked, 7)
- s.Contains(acked, b01.Hash)
- s.Contains(acked, b11.Hash)
- s.Contains(acked, b12.Hash)
- s.Contains(acked, b21.Hash)
- s.Contains(acked, b22.Hash)
- s.Contains(acked, b31.Hash)
- s.Contains(acked, b32.Hash)
-
- acked = to.acked[b10.Hash]
- s.Require().NotNil(acked)
- s.Len(acked, 9)
- s.Contains(acked, b01.Hash)
- s.Contains(acked, b11.Hash)
- s.Contains(acked, b12.Hash)
- s.Contains(acked, b20.Hash)
- s.Contains(acked, b21.Hash)
- s.Contains(acked, b22.Hash)
- s.Contains(acked, b30.Hash)
- s.Contains(acked, b31.Hash)
- s.Contains(acked, b32.Hash)
-
- // Make sure there are 2 candidates.
- s.Require().Len(to.candidateChainMapping, 2)
-
- // Check b00's height vector.
- candidate := to.candidates[0]
- s.Require().NotNil(candidate)
- s.Equal(candidate.ackedStatus[0].minHeight, b00.Position.Height)
- s.Equal(candidate.ackedStatus[0].count, uint64(2))
- s.Equal(candidate.ackedStatus[1].minHeight, b11.Position.Height)
- s.Equal(candidate.ackedStatus[1].count, uint64(2))
- s.Equal(candidate.ackedStatus[2].minHeight, b21.Position.Height)
- s.Equal(candidate.ackedStatus[2].count, uint64(2))
- s.Equal(candidate.ackedStatus[3].minHeight, b31.Position.Height)
- s.Equal(candidate.ackedStatus[3].count, uint64(2))
- s.Equal(candidate.ackedStatus[4].count, uint64(0))
-
- // Check b10's height vector.
- candidate = to.candidates[1]
- s.Require().NotNil(candidate)
- s.Equal(candidate.ackedStatus[0].minHeight, b01.Position.Height)
- s.Equal(candidate.ackedStatus[0].count, uint64(1))
- s.Equal(candidate.ackedStatus[1].minHeight, b10.Position.Height)
- s.Equal(candidate.ackedStatus[1].count, uint64(3))
- s.Equal(candidate.ackedStatus[2].minHeight, b20.Position.Height)
- s.Equal(candidate.ackedStatus[2].count, uint64(3))
- s.Equal(candidate.ackedStatus[3].minHeight, b30.Position.Height)
- s.Equal(candidate.ackedStatus[3].count, uint64(3))
- s.Equal(candidate.ackedStatus[4].count, uint64(0))
-
- // Check the first deliver.
- s.NoError(to.addBlock(b02))
- blocks, mode, err := to.extractBlocks()
- s.Equal(mode, TotalOrderingModeEarly)
- s.Nil(err)
- s.checkHashSequence(blocks, common.Hashes{b00.Hash, b10.Hash})
-
- // Make sure b00, b10 are removed from current working set.
- s.checkNotInWorkingSet(to, b00)
- s.checkNotInWorkingSet(to, b10)
-
- // Check if candidates of next round are picked correctly.
- s.Len(to.candidateChainMapping, 2)
-
- // Check b01's height vector.
- candidate = to.candidates[1]
- s.Require().NotNil(candidate)
- s.Equal(candidate.ackedStatus[0].minHeight, b01.Position.Height)
- s.Equal(candidate.ackedStatus[0].count, uint64(2))
- s.Equal(candidate.ackedStatus[1].minHeight, b11.Position.Height)
- s.Equal(candidate.ackedStatus[1].count, uint64(2))
- s.Equal(candidate.ackedStatus[2].minHeight, b21.Position.Height)
- s.Equal(candidate.ackedStatus[2].count, uint64(2))
- s.Equal(candidate.ackedStatus[3].minHeight, b11.Position.Height)
- s.Equal(candidate.ackedStatus[3].count, uint64(2))
- s.Equal(candidate.ackedStatus[4].count, uint64(0))
-
- // Check b20's height vector.
- candidate = to.candidates[2]
- s.Require().NotNil(candidate)
- s.Equal(candidate.ackedStatus[0].minHeight, b02.Position.Height)
- s.Equal(candidate.ackedStatus[0].count, uint64(1))
- s.Equal(candidate.ackedStatus[1].minHeight, b12.Position.Height)
- s.Equal(candidate.ackedStatus[1].count, uint64(1))
- s.Equal(candidate.ackedStatus[2].minHeight, b20.Position.Height)
- s.Equal(candidate.ackedStatus[2].count, uint64(3))
- s.Equal(candidate.ackedStatus[3].minHeight, b30.Position.Height)
- s.Equal(candidate.ackedStatus[3].count, uint64(3))
- s.Equal(candidate.ackedStatus[4].count, uint64(0))
-
- s.checkNotDeliver(to, b13)
-
- // Check the second deliver.
- s.NoError(to.addBlock(b03))
- blocks, mode, err = to.extractBlocks()
- s.Equal(mode, TotalOrderingModeEarly)
- s.Nil(err)
- s.checkHashSequence(blocks, common.Hashes{b11.Hash, b20.Hash})
-
- // Make sure b11, b20 are removed from current working set.
- s.checkNotInWorkingSet(to, b11)
- s.checkNotInWorkingSet(to, b20)
-
- // Add b40, b41, b42 to pending set.
- s.checkNotDeliver(to, b40)
- s.checkNotDeliver(to, b41)
- s.checkNotDeliver(to, b42)
- s.checkNotDeliver(to, b14)
-
- // Make sure b01, b30, b40 are candidate in next round.
- s.Len(to.candidateChainMapping, 3)
- candidate = to.candidates[0]
- s.Require().NotNil(candidate)
- s.Equal(candidate.ackedStatus[0].minHeight, b01.Position.Height)
- s.Equal(candidate.ackedStatus[0].count, uint64(3))
- s.Equal(candidate.ackedStatus[1].minHeight, b12.Position.Height)
- s.Equal(candidate.ackedStatus[1].count, uint64(3))
- s.Equal(candidate.ackedStatus[2].minHeight, b21.Position.Height)
- s.Equal(candidate.ackedStatus[2].count, uint64(2))
- s.Equal(candidate.ackedStatus[3].minHeight, b31.Position.Height)
- s.Equal(candidate.ackedStatus[3].count, uint64(2))
- s.Equal(candidate.ackedStatus[4].count, uint64(0))
-
- candidate = to.candidates[3]
- s.Require().NotNil(candidate)
- s.Equal(candidate.ackedStatus[0].minHeight, b03.Position.Height)
- s.Equal(candidate.ackedStatus[0].count, uint64(1))
- s.Equal(candidate.ackedStatus[1].minHeight, b13.Position.Height)
- s.Equal(candidate.ackedStatus[1].count, uint64(2))
- s.Equal(candidate.ackedStatus[2].minHeight, b22.Position.Height)
- s.Equal(candidate.ackedStatus[2].count, uint64(1))
- s.Equal(candidate.ackedStatus[3].minHeight, b30.Position.Height)
- s.Equal(candidate.ackedStatus[3].count, uint64(3))
- s.Equal(candidate.ackedStatus[4].count, uint64(0))
-
- candidate = to.candidates[4]
- s.Require().NotNil(candidate)
- s.Equal(candidate.ackedStatus[0].count, uint64(0))
- s.Equal(candidate.ackedStatus[1].count, uint64(0))
- s.Equal(candidate.ackedStatus[2].count, uint64(0))
- s.Equal(candidate.ackedStatus[3].count, uint64(0))
- s.Equal(candidate.ackedStatus[4].minHeight, b40.Position.Height)
- s.Equal(candidate.ackedStatus[4].count, uint64(3))
-
- // Make 'Acking Node Set' contains blocks from all chains,
- // this should trigger not-early deliver.
- s.NoError(to.addBlock(b23))
- blocks, mode, err = to.extractBlocks()
- s.Equal(mode, TotalOrderingModeNormal)
- s.Nil(err)
- s.checkHashSequence(blocks, common.Hashes{b01.Hash, b30.Hash})
-
- // Make sure b01, b30 not in working set
- s.checkNotInWorkingSet(to, b01)
- s.checkNotInWorkingSet(to, b30)
-
- // Make sure b21, b40 are candidates of next round.
- s.Equal(to.candidateChainMapping[b21.Position.ChainID], b21.Hash)
- s.Equal(to.candidateChainMapping[b40.Position.ChainID], b40.Hash)
-}
-
-func (s *TotalOrderingTestSuite) TestBasicCaseForK0() {
- // This is a relatively simple test for K=0.
- //
- // 0 1 2 3 4
- // -------------------
- // . . . . .
- // . . . . .
- // o o o <- o <- o Height: 1
- // | \ | \ | |
- // v v v v
- // o o o <- o Height: 0
- var (
- nodes = test.GenerateRandomNodeIDs(5)
- genesisConfig = &types.Config{
- RoundInterval: 1000 * time.Second,
- K: 0,
- PhiRatio: 0.6,
- NumChains: uint32(len(nodes)),
- }
- req = s.Require()
- genesisTime = time.Now().UTC()
- to = newTotalOrdering(genesisTime, 0, genesisConfig)
- )
- // Setup blocks.
- b00 := s.genGenesisBlock(nodes, 0, common.Hashes{})
- b10 := s.genGenesisBlock(nodes, 1, common.Hashes{})
- b20 := s.genGenesisBlock(nodes, 2, common.Hashes{})
- b30 := s.genGenesisBlock(nodes, 3, common.Hashes{b20.Hash})
- b01 := &types.Block{
- ProposerID: nodes[0],
- ParentHash: b00.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 1,
- ChainID: 0,
- },
- Acks: common.NewSortedHashes(common.Hashes{b00.Hash, b10.Hash}),
- }
- b11 := &types.Block{
- ProposerID: nodes[1],
- ParentHash: b10.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 1,
- ChainID: 1,
- },
- Acks: common.NewSortedHashes(common.Hashes{b10.Hash, b20.Hash}),
- }
- b21 := &types.Block{
- ProposerID: nodes[2],
- ParentHash: b20.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 1,
- ChainID: 2,
- },
- Acks: common.NewSortedHashes(common.Hashes{b20.Hash}),
- }
- b31 := &types.Block{
- ProposerID: nodes[3],
- ParentHash: b30.Hash,
- Hash: common.NewRandomHash(),
- Position: types.Position{
- Height: 1,
- ChainID: 3,
- },
- Acks: common.NewSortedHashes(common.Hashes{b21.Hash, b30.Hash}),
- }
- b40 := s.genGenesisBlock(nodes, 4, common.Hashes{b31.Hash})
-
- s.checkNotDeliver(to, b00)
- s.checkNotDeliver(to, b10)
- s.checkNotDeliver(to, b20)
- s.checkNotDeliver(to, b30)
- s.checkNotDeliver(to, b01)
- s.checkNotDeliver(to, b11)
- s.checkNotDeliver(to, b21)
- s.checkNotDeliver(to, b31)
-
- // Check candidate status before delivering.
- candidate := to.candidates[0]
- req.NotNil(candidate)
- req.Equal(candidate.ackedStatus[0].minHeight, b00.Position.Height)
- req.Equal(candidate.ackedStatus[0].count, uint64(2))
-
- candidate = to.candidates[1]
- req.NotNil(candidate)
- req.Equal(candidate.ackedStatus[0].minHeight, b01.Position.Height)
- req.Equal(candidate.ackedStatus[0].count, uint64(1))
- req.Equal(candidate.ackedStatus[1].minHeight, b10.Position.Height)
- req.Equal(candidate.ackedStatus[1].count, uint64(2))
-
- candidate = to.candidates[2]
- req.NotNil(candidate)
- req.Equal(candidate.ackedStatus[1].minHeight, b11.Position.Height)
- req.Equal(candidate.ackedStatus[1].count, uint64(1))
- req.Equal(candidate.ackedStatus[2].minHeight, b20.Position.Height)
- req.Equal(candidate.ackedStatus[2].count, uint64(2))
- req.Equal(candidate.ackedStatus[3].minHeight, b30.Position.Height)
- req.Equal(candidate.ackedStatus[3].count, uint64(2))
-
- // This new block should trigger non-early deliver.
- req.NoError(to.addBlock(b40))
- blocks, mode, err := to.extractBlocks()
- req.Equal(mode, TotalOrderingModeNormal)
- req.Nil(err)
- s.checkHashSequence(blocks, common.Hashes{b20.Hash})
-
- // Make sure b20 is no long existing in working set.
- s.checkNotInWorkingSet(to, b20)
-
- // Make sure b10, b30 are candidates for next round.
- req.Equal(to.candidateChainMapping[b00.Position.ChainID], b00.Hash)
- req.Equal(to.candidateChainMapping[b10.Position.ChainID], b10.Hash)
- req.Equal(to.candidateChainMapping[b30.Position.ChainID], b30.Hash)
-}
-
-func (s *TotalOrderingTestSuite) baseTestRandomlyGeneratedBlocks(
- totalOrderingConstructor func(chainNum uint32) *totalOrdering,
- chainNum uint32,
- ackingCountGenerator func() int,
- repeat int) {
- var (
- req = s.Require()
- revealingSequence = make(map[string]struct{})
- orderingSequence = make(map[string]struct{})
- genesisTime = time.Now().UTC()
- )
- gen := test.NewBlocksGenerator(&test.BlocksGeneratorConfig{
- NumChains: chainNum,
- MinBlockTimeInterval: 250 * time.Millisecond,
- }, ackingCountGenerator)
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
- req.NoError(gen.Generate(
- 0,
- genesisTime,
- genesisTime.Add(20*time.Second),
- dbInst))
- iter, err := dbInst.GetAllBlocks()
- req.NoError(err)
- // Setup a revealer that would reveal blocks forming
- // valid DAGs.
- revealer, err := test.NewRandomDAGBlockRevealer(iter)
- req.NoError(err)
- // TODO (mission): make this part run concurrently.
- for i := 0; i < repeat; i++ {
- revealed, ordered := s.performOneRun(
- totalOrderingConstructor(chainNum), revealer)
- revealingSequence[revealed] = struct{}{}
- orderingSequence[ordered] = struct{}{}
- }
- s.checkRandomResult(revealingSequence, orderingSequence)
-}
-
-func (s *TotalOrderingTestSuite) TestRandomlyGeneratedBlocks() {
- var (
- numChains = uint32(20)
- phi = float32(0.5)
- repeat = 15
- genesisTime = time.Now().UTC()
- )
- if testing.Short() {
- numChains = 10
- phi = 0.5
- repeat = 3
- }
-
- ackingCountGenerators := []func() int{
- nil, // Acking frequency with normal distribution.
- test.MaxAckingCountGenerator(0), // Low acking frequency.
- test.MaxAckingCountGenerator(numChains), // High acking frequency.
- }
-
- // Test based on different acking frequency.
- for _, gen := range ackingCountGenerators {
- // Test for K=0.
- constructor := func(numChains uint32) *totalOrdering {
- genesisConfig := &types.Config{
- RoundInterval: 1000 * time.Second,
- K: 0,
- PhiRatio: phi,
- NumChains: numChains,
- }
- to := newTotalOrdering(genesisTime, 0, genesisConfig)
- // Add config for next round.
- s.Require().NoError(to.appendConfig(1, &types.Config{
- K: 0,
- PhiRatio: 0.5,
- NumChains: numChains,
- }))
- return to
- }
- s.baseTestRandomlyGeneratedBlocks(constructor, numChains, gen, repeat)
- // Test for K=1.
- constructor = func(numChains uint32) *totalOrdering {
- genesisConfig := &types.Config{
- RoundInterval: 1000 * time.Second,
- K: 1,
- PhiRatio: phi,
- NumChains: numChains,
- }
- to := newTotalOrdering(genesisTime, 0, genesisConfig)
- // Add config for next round.
- s.Require().NoError(to.appendConfig(1, &types.Config{
- K: 1,
- PhiRatio: 0.5,
- NumChains: numChains,
- }))
- return to
- }
- s.baseTestRandomlyGeneratedBlocks(constructor, numChains, gen, repeat)
- // Test for K=2.
- constructor = func(numChains uint32) *totalOrdering {
- genesisConfig := &types.Config{
- RoundInterval: 1000 * time.Second,
- K: 2,
- PhiRatio: phi,
- NumChains: numChains,
- }
- to := newTotalOrdering(genesisTime, 0, genesisConfig)
- s.Require().NoError(to.appendConfig(1, &types.Config{
- K: 2,
- PhiRatio: 0.5,
- NumChains: numChains,
- }))
- return to
- }
- s.baseTestRandomlyGeneratedBlocks(constructor, numChains, gen, repeat)
- // Test for K=3.
- constructor = func(numChains uint32) *totalOrdering {
- genesisConfig := &types.Config{
- RoundInterval: 1000 * time.Second,
- K: 3,
- PhiRatio: phi,
- NumChains: numChains,
- }
- to := newTotalOrdering(genesisTime, 0, genesisConfig)
- s.Require().NoError(to.appendConfig(1, &types.Config{
- K: 3,
- PhiRatio: 0.5,
- NumChains: numChains,
- }))
- return to
- }
- s.baseTestRandomlyGeneratedBlocks(constructor, numChains, gen, repeat)
- }
-}
-
-func (s *TotalOrderingTestSuite) baseTestForRoundChange(
- repeat int, configs []*types.Config) {
- var (
- req = s.Require()
- genesisTime = time.Now().UTC()
- )
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
- // Generate DAG for rounds.
- // NOTE: the last config won't be tested, just avoid panic
- // when round switching.
- begin := genesisTime
- for roundID, config := range configs[:len(configs)-1] {
- gen := test.NewBlocksGenerator(
- test.NewBlocksGeneratorConfig(config), nil)
- end := begin.Add(config.RoundInterval)
- req.NoError(gen.Generate(uint64(roundID), begin, end, dbInst))
- begin = end
- }
- // Test, just dump the whole DAG to total ordering and make sure
- // repeating it won't change it delivered sequence.
- iter, err := dbInst.GetAllBlocks()
- req.NoError(err)
- revealer, err := test.NewRandomDAGBlockRevealer(iter)
- req.NoError(err)
- revealingSequence := make(map[string]struct{})
- orderingSequence := make(map[string]struct{})
- for i := 0; i < repeat; i++ {
- to := newTotalOrdering(genesisTime, 0, configs[0])
- for roundID, config := range configs[1:] {
- req.NoError(to.appendConfig(uint64(roundID+1), config))
- }
- revealed, ordered := s.performOneRun(to, revealer)
- revealingSequence[revealed] = struct{}{}
- orderingSequence[ordered] = struct{}{}
- }
- s.checkRandomResult(revealingSequence, orderingSequence)
-}
-
-func (s *TotalOrderingTestSuite) TestNumChainsChanged() {
- // This test fixes K, Phi, and changes 'numChains' for each round.
- fix := func(c *types.Config) *types.Config {
- c.K = 1
- c.PhiRatio = 0.5
- c.MinBlockInterval = 250 * time.Millisecond
- c.RoundInterval = 10 * time.Second
- return c
- }
- var (
- repeat = 7
- configs = []*types.Config{
- fix(&types.Config{NumChains: 7}),
- fix(&types.Config{NumChains: 10}),
- fix(&types.Config{NumChains: 4}),
- fix(&types.Config{NumChains: 13}),
- fix(&types.Config{NumChains: 4}),
- }
- )
- s.baseTestForRoundChange(repeat, configs)
-}
-
-func (s *TotalOrderingTestSuite) TestPhiChanged() {
- // This test fixes K, numChains, and changes Phi each round.
- fix := func(c *types.Config) *types.Config {
- c.K = 1
- c.NumChains = 10
- c.MinBlockInterval = 250 * time.Millisecond
- c.RoundInterval = 10 * time.Second
- return c
- }
- var (
- repeat = 7
- configs = []*types.Config{
- fix(&types.Config{PhiRatio: 0.5}),
- fix(&types.Config{PhiRatio: 0.7}),
- fix(&types.Config{PhiRatio: 1}),
- fix(&types.Config{PhiRatio: 0.5}),
- fix(&types.Config{PhiRatio: 0.7}),
- }
- )
- s.baseTestForRoundChange(repeat, configs)
-}
-
-func (s *TotalOrderingTestSuite) TestKChanged() {
- // This test fixes phi, numChains, and changes K each round.
- fix := func(c *types.Config) *types.Config {
- c.NumChains = 10
- c.PhiRatio = 0.7
- c.MinBlockInterval = 250 * time.Millisecond
- c.RoundInterval = 10 * time.Second
- return c
- }
- var (
- repeat = 7
- configs = []*types.Config{
- fix(&types.Config{K: 0}),
- fix(&types.Config{K: 4}),
- fix(&types.Config{K: 1}),
- fix(&types.Config{K: 2}),
- fix(&types.Config{K: 0}),
- }
- )
- s.baseTestForRoundChange(repeat, configs)
-}
-
-func (s *TotalOrderingTestSuite) TestRoundChanged() {
- // This test changes everything when round changed.
- fix := func(c *types.Config) *types.Config {
- c.MinBlockInterval = 250 * time.Millisecond
- c.RoundInterval = 10 * time.Second
- return c
- }
- var (
- repeat = 7
- configs = []*types.Config{
- fix(&types.Config{K: 0, NumChains: 4, PhiRatio: 0.5}),
- fix(&types.Config{K: 1, NumChains: 10, PhiRatio: 0.7}),
- fix(&types.Config{K: 2, NumChains: 7, PhiRatio: 0.8}),
- fix(&types.Config{K: 0, NumChains: 4, PhiRatio: 0.5}),
- fix(&types.Config{K: 3, NumChains: 10, PhiRatio: 0.8}),
- fix(&types.Config{K: 0, NumChains: 7, PhiRatio: 0.5}),
- fix(&types.Config{K: 2, NumChains: 13, PhiRatio: 0.7}),
- }
- )
- s.baseTestForRoundChange(repeat, configs)
-}
-
-// TestSync tests sync mode of total ordering, which is started not from genesis
-// but some blocks which is on the cut of delivery set.
-func (s *TotalOrderingTestSuite) TestSync() {
- var (
- req = s.Require()
- numChains = uint32(19)
- genesisTime = time.Now().UTC()
- )
- gen := test.NewBlocksGenerator(&test.BlocksGeneratorConfig{
- NumChains: numChains,
- MinBlockTimeInterval: 250 * time.Millisecond,
- }, nil)
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
- err = gen.Generate(0, genesisTime, genesisTime.Add(20*time.Second), dbInst)
- req.NoError(err)
- iter, err := dbInst.GetAllBlocks()
- req.NoError(err)
-
- revealer, err := test.NewRandomDAGBlockRevealer(iter)
- req.NoError(err)
-
- genesisConfig := &types.Config{
- RoundInterval: 1000 * time.Second,
- K: 0,
- PhiRatio: 0.67,
- NumChains: numChains,
- }
- to1 := newTotalOrdering(genesisTime, 0, genesisConfig)
- s.Require().NoError(to1.appendConfig(1, &types.Config{
- K: 0,
- PhiRatio: 0.5,
- NumChains: numChains,
- }))
- deliveredBlockSets1 := [][]*types.Block{}
- for {
- b, err := revealer.NextBlock()
- if err != nil {
- if err == db.ErrIterationFinished {
- err = nil
- break
- }
- }
- s.Require().NoError(err)
- s.Require().NoError(to1.addBlock(&b))
- bs, _, err := to1.extractBlocks()
- s.Require().Nil(err)
- if len(bs) > 0 {
- deliveredBlockSets1 = append(deliveredBlockSets1, bs)
- }
- }
- // Run new total ordering again.
- offset := len(deliveredBlockSets1) / 2
- to2 := newTotalOrdering(genesisTime, 0, genesisConfig)
- s.Require().NoError(to2.appendConfig(1, &types.Config{
- K: 0,
- PhiRatio: 0.5,
- NumChains: numChains,
- }))
- deliveredBlockSets2 := [][]*types.Block{}
- for i := offset; i < len(deliveredBlockSets1); i++ {
- for _, b := range deliveredBlockSets1[i] {
- req.NoError(to2.addBlock(b))
- bs, _, err := to2.extractBlocks()
- req.NoError(err)
- if len(bs) > 0 {
- deliveredBlockSets2 = append(deliveredBlockSets2, bs)
- }
- }
- }
- // Check deliver1 and deliver2.
- for i := 0; i < len(deliveredBlockSets2); i++ {
- req.Equal(len(deliveredBlockSets1[offset+i]), len(deliveredBlockSets2[i]))
- for j := 0; j < len(deliveredBlockSets2[i]); j++ {
- req.Equal(deliveredBlockSets1[offset+i][j], deliveredBlockSets2[i][j])
- }
- }
-}
-
-func (s *TotalOrderingTestSuite) TestSyncWithConfigChange() {
- var (
- req = s.Require()
- genesisTime = time.Now().UTC()
- roundInterval = 30 * time.Second
- )
-
- // Configs for round change, notice configs[0] is the same as genesisConfig.
- configs := []*types.Config{
- &types.Config{
- K: 0,
- PhiRatio: 0.67,
- NumChains: uint32(19),
- RoundInterval: roundInterval,
- },
- &types.Config{
- K: 2,
- PhiRatio: 0.5,
- NumChains: uint32(17),
- RoundInterval: roundInterval,
- },
- &types.Config{
- K: 0,
- PhiRatio: 0.8,
- NumChains: uint32(22),
- RoundInterval: roundInterval,
- },
- &types.Config{
- K: 3,
- PhiRatio: 0.5,
- NumChains: uint32(25),
- RoundInterval: roundInterval,
- },
- &types.Config{
- K: 1,
- PhiRatio: 0.7,
- NumChains: uint32(20),
- RoundInterval: roundInterval,
- },
- // Sometimes all generated blocks would be delivered, thus the total
- // ordering module would proceed to next round. We need to prepare
- // one additional configuration for that possibility.
- &types.Config{
- K: 1,
- PhiRatio: 0.7,
- NumChains: uint32(20),
- RoundInterval: roundInterval,
- },
- }
-
- blocks := []*types.Block{}
- dbInst, err := db.NewMemBackedDB()
- req.NoError(err)
-
- for i, cfg := range configs[:len(configs)-1] {
- gen := test.NewBlocksGenerator(&test.BlocksGeneratorConfig{
- NumChains: cfg.NumChains,
- MinBlockTimeInterval: 250 * time.Millisecond,
- }, nil)
- err = gen.Generate(
- uint64(i),
- genesisTime.Add(time.Duration(i)*cfg.RoundInterval),
- genesisTime.Add(time.Duration(i+1)*cfg.RoundInterval),
- dbInst,
- )
- req.NoError(err)
- }
-
- iter, err := dbInst.GetAllBlocks()
- req.NoError(err)
-
- revealer, err := test.NewRandomDAGBlockRevealer(iter)
- req.NoError(err)
-
- for {
- b, err := revealer.NextBlock()
- if err != nil {
- if err == db.ErrIterationFinished {
- err = nil
- break
- }
- }
- req.NoError(err)
- blocks = append(blocks, &b)
- }
-
- to1 := newTotalOrdering(genesisTime, 0, configs[0])
- for i, cfg := range configs[1:] {
- req.NoError(to1.appendConfig(uint64(i+1), cfg))
- }
-
- deliveredBlockSets1 := [][]*types.Block{}
- deliveredBlockModes := []uint32{}
- for _, b := range blocks {
- req.NoError(to1.addBlock(b))
- bs, mode, err := to1.extractBlocks()
- req.NoError(err)
- if len(bs) > 0 {
- deliveredBlockSets1 = append(deliveredBlockSets1, bs)
- deliveredBlockModes = append(deliveredBlockModes, mode)
- }
- }
-
- // Find the offset that can be used in the second run of total ordering. And
- // the mode of deliver set should not be "flush".
- for test := 0; test < 3; test++ {
- offset := len(deliveredBlockSets1) * (3 + test) / 7
- for deliveredBlockModes[offset] == TotalOrderingModeFlush {
- offset++
- }
- offsetRound := deliveredBlockSets1[offset][0].Position.Round
- // The range of offset's round should not be the first nor the last round,
- // or nothing is tested.
- req.True(uint64(0) < offsetRound && offsetRound < uint64(len(configs)-1))
-
- to2 := newTotalOrdering(genesisTime, 0, configs[0])
- for i, cfg := range configs[1:] {
- req.NoError(to2.appendConfig(uint64(i+1), cfg))
- }
- // Skip useless configs.
- for i := uint64(0); i < deliveredBlockSets1[offset][0].Position.Round; i++ {
- to2.switchRound()
- }
- // Run total ordering again from offset.
- deliveredBlockSets2 := [][]*types.Block{}
- for i := offset; i < len(deliveredBlockSets1); i++ {
- for _, b := range deliveredBlockSets1[i] {
- req.NoError(to2.addBlock(b))
- bs, _, err := to2.extractBlocks()
- req.NoError(err)
- if len(bs) > 0 {
- deliveredBlockSets2 = append(deliveredBlockSets2, bs)
- }
- }
- }
- // Check deliver1 and deliver2.
- for i := 0; i < len(deliveredBlockSets2); i++ {
- req.Equal(len(deliveredBlockSets1[offset+i]), len(deliveredBlockSets2[i]))
- for j := 0; j < len(deliveredBlockSets2[i]); j++ {
- req.Equal(deliveredBlockSets1[offset+i][j], deliveredBlockSets2[i][j])
- }
- }
- }
-}
-
-func (s *TotalOrderingTestSuite) TestModeDefinition() {
- // Make sure the copied deliver mode definition is identical between
- // core and test package.
- s.Require().Equal(TotalOrderingModeError, test.TotalOrderingModeError)
- s.Require().Equal(TotalOrderingModeNormal, test.TotalOrderingModeNormal)
- s.Require().Equal(TotalOrderingModeEarly, test.TotalOrderingModeEarly)
- s.Require().Equal(TotalOrderingModeFlush, test.TotalOrderingModeFlush)
-}
-
-func TestTotalOrdering(t *testing.T) {
- suite.Run(t, new(TotalOrderingTestSuite))
-}
diff --git a/core/types/block-randomness.go b/core/types/block-randomness.go
index 65cb635..74360c7 100644
--- a/core/types/block-randomness.go
+++ b/core/types/block-randomness.go
@@ -34,7 +34,7 @@ type AgreementResult struct {
func (r *AgreementResult) String() string {
return fmt.Sprintf("agreementResult{Hash:%s %s}",
- r.BlockHash.String()[:6], &r.Position)
+ r.BlockHash.String()[:6], r.Position)
}
// BlockRandomnessResult describes a block randomness result
@@ -46,7 +46,7 @@ type BlockRandomnessResult struct {
func (r *BlockRandomnessResult) String() string {
return fmt.Sprintf("blockRandomness{Block:%s Pos:%s Rand:%s}",
- r.BlockHash.String()[:6], &r.Position,
+ r.BlockHash.String()[:6], r.Position,
hex.EncodeToString(r.Randomness)[:6],
)
}
diff --git a/core/types/block.go b/core/types/block.go
index b2a8f57..a2b697c 100644
--- a/core/types/block.go
+++ b/core/types/block.go
@@ -198,7 +198,7 @@ func (b *Block) DecodeRLP(s *rlp.Stream) error {
}
func (b *Block) String() string {
- return fmt.Sprintf("Block{Hash:%v %s}", b.Hash.String()[:6], &b.Position)
+ return fmt.Sprintf("Block{Hash:%v %s}", b.Hash.String()[:6], b.Position)
}
// Clone returns a deep copy of a block.
@@ -263,62 +263,62 @@ func (b ByHash) Swap(i int, j int) {
b[i], b[j] = b[j], b[i]
}
-// ByPosition is the helper type for sorting slice of blocks by position.
-type ByPosition []*Block
+// BlocksByPosition is the helper type for sorting slice of blocks by position.
+type BlocksByPosition []*Block
// Len implements Len method in sort.Sort interface.
-func (bs ByPosition) Len() int {
+func (bs BlocksByPosition) Len() int {
return len(bs)
}
// Less implements Less method in sort.Sort interface.
-func (bs ByPosition) Less(i int, j int) bool {
- return bs[j].Position.Newer(&bs[i].Position)
+func (bs BlocksByPosition) Less(i int, j int) bool {
+ return bs[j].Position.Newer(bs[i].Position)
}
// Swap implements Swap method in sort.Sort interface.
-func (bs ByPosition) Swap(i int, j int) {
+func (bs BlocksByPosition) Swap(i int, j int) {
bs[i], bs[j] = bs[j], bs[i]
}
// Push implements Push method in heap interface.
-func (bs *ByPosition) Push(x interface{}) {
+func (bs *BlocksByPosition) Push(x interface{}) {
*bs = append(*bs, x.(*Block))
}
// Pop implements Pop method in heap interface.
-func (bs *ByPosition) Pop() (ret interface{}) {
+func (bs *BlocksByPosition) Pop() (ret interface{}) {
n := len(*bs)
*bs, ret = (*bs)[0:n-1], (*bs)[n-1]
return
}
-// ByFinalizationHeight is the helper type for sorting slice of blocks by
+// BlocksByFinalizationHeight is the helper type for sorting slice of blocks by
// finalization height.
-type ByFinalizationHeight []*Block
+type BlocksByFinalizationHeight []*Block
// Len implements Len method in sort.Sort interface.
-func (bs ByFinalizationHeight) Len() int {
+func (bs BlocksByFinalizationHeight) Len() int {
return len(bs)
}
// Less implements Less method in sort.Sort interface.
-func (bs ByFinalizationHeight) Less(i int, j int) bool {
+func (bs BlocksByFinalizationHeight) Less(i int, j int) bool {
return bs[i].Finalization.Height < bs[j].Finalization.Height
}
// Swap implements Swap method in sort.Sort interface.
-func (bs ByFinalizationHeight) Swap(i int, j int) {
+func (bs BlocksByFinalizationHeight) Swap(i int, j int) {
bs[i], bs[j] = bs[j], bs[i]
}
// Push implements Push method in heap interface.
-func (bs *ByFinalizationHeight) Push(x interface{}) {
+func (bs *BlocksByFinalizationHeight) Push(x interface{}) {
*bs = append(*bs, x.(*Block))
}
// Pop implements Pop method in heap interface.
-func (bs *ByFinalizationHeight) Pop() (ret interface{}) {
+func (bs *BlocksByFinalizationHeight) Pop() (ret interface{}) {
n := len(*bs)
*bs, ret = (*bs)[0:n-1], (*bs)[n-1]
return
diff --git a/core/types/block_test.go b/core/types/block_test.go
index 9478e86..d47096f 100644
--- a/core/types/block_test.go
+++ b/core/types/block_test.go
@@ -136,16 +136,19 @@ func (s *BlockTestSuite) TestSortByHash() {
s.Equal(blocks[3].Hash, b3.Hash)
}
-func (s *BlockTestSuite) TestSortByPosition() {
- b00 := &Block{Position: Position{Height: 0}}
- b01 := &Block{Position: Position{Height: 1}}
- b02 := &Block{Position: Position{Height: 2}}
- b10 := &Block{Position: Position{Round: 1, Height: 0}}
- b11 := &Block{Position: Position{Round: 1, Height: 1}}
- b12 := &Block{Position: Position{Round: 1, Height: 2}}
+func (s *BlockTestSuite) TestSortBlocksByPosition() {
+ b00 := &Block{Hash: common.NewRandomHash(), Position: Position{Height: 0}}
+ b01 := &Block{Hash: common.NewRandomHash(), Position: Position{Height: 1}}
+ b02 := &Block{Hash: common.NewRandomHash(), Position: Position{Height: 2}}
+ b10 := &Block{Hash: common.NewRandomHash(),
+ Position: Position{Round: 1, Height: 0}}
+ b11 := &Block{Hash: common.NewRandomHash(),
+ Position: Position{Round: 1, Height: 1}}
+ b12 := &Block{Hash: common.NewRandomHash(),
+ Position: Position{Round: 1, Height: 2}}
blocks := []*Block{b12, b11, b10, b02, b01, b00}
- sort.Sort(ByPosition(blocks))
+ sort.Sort(BlocksByPosition(blocks))
s.Equal(blocks[0].Hash, b00.Hash)
s.Equal(blocks[1].Hash, b01.Hash)
s.Equal(blocks[2].Hash, b02.Hash)
diff --git a/core/types/nodeset.go b/core/types/nodeset.go
index 21a1e52..fccfbb6 100644
--- a/core/types/nodeset.go
+++ b/core/types/nodeset.go
@@ -85,11 +85,8 @@ func NewNodeSetFromMap(nodes map[NodeID]struct{}) *NodeSet {
}
// NewNotarySetTarget is the target for getting Notary Set.
-func NewNotarySetTarget(crs common.Hash, chainID uint32) *SubSetTarget {
- binaryChainID := make([]byte, 4)
- binary.LittleEndian.PutUint32(binaryChainID, chainID)
-
- return newTarget(targetNotarySet, crs[:], binaryChainID)
+func NewNotarySetTarget(crs common.Hash) *SubSetTarget {
+ return newTarget(targetNotarySet, crs[:])
}
// NewDKGSetTarget is the target for getting DKG Set.
@@ -98,15 +95,10 @@ func NewDKGSetTarget(crs common.Hash) *SubSetTarget {
}
// NewNodeLeaderTarget is the target for getting leader of fast BA.
-func NewNodeLeaderTarget(crs common.Hash, pos Position) *SubSetTarget {
- binaryRoundID := make([]byte, 8)
- binary.LittleEndian.PutUint64(binaryRoundID, pos.Round)
- binaryChainID := make([]byte, 4)
- binary.LittleEndian.PutUint32(binaryChainID, pos.ChainID)
+func NewNodeLeaderTarget(crs common.Hash, height uint64) *SubSetTarget {
binaryHeight := make([]byte, 8)
- binary.LittleEndian.PutUint64(binaryHeight, pos.Height)
- return newTarget(targetNodeLeader, crs[:],
- binaryRoundID, binaryChainID, binaryHeight)
+ binary.LittleEndian.PutUint64(binaryHeight, height)
+ return newTarget(targetNodeLeader, crs[:], binaryHeight)
}
// Add a NodeID to the set.
diff --git a/core/types/nodeset_test.go b/core/types/nodeset_test.go
index 2780ed2..ef9ac24 100644
--- a/core/types/nodeset_test.go
+++ b/core/types/nodeset_test.go
@@ -35,7 +35,7 @@ func (s *NodeSetTestSuite) TestGetSubSet() {
for len(nodes.IDs) < total {
nodes.IDs[NodeID{common.NewRandomHash()}] = struct{}{}
}
- target := NewNotarySetTarget(crs, 0)
+ target := NewNotarySetTarget(crs)
ranks := make(map[NodeID]*nodeRank, len(nodes.IDs))
for nID := range nodes.IDs {
ranks[nID] = newNodeRank(nID, target)
diff --git a/core/types/position.go b/core/types/position.go
index 8822f6e..902a55f 100644
--- a/core/types/position.go
+++ b/core/types/position.go
@@ -28,14 +28,14 @@ type Position struct {
Height uint64 `json:"height"`
}
-func (pos *Position) String() string {
+func (pos Position) String() string {
return fmt.Sprintf("Position{Round:%d Chain:%d Height:%d}",
pos.Round, pos.ChainID, pos.Height)
}
// Equal checks if two positions are equal, it panics when their chainIDs
// are different.
-func (pos *Position) Equal(other *Position) bool {
+func (pos Position) Equal(other Position) bool {
if pos.ChainID != other.ChainID {
panic(fmt.Errorf("unexpected chainID %d, should be %d",
other.ChainID, pos.ChainID))
@@ -45,7 +45,7 @@ func (pos *Position) Equal(other *Position) bool {
// Newer checks if one block is newer than another one on the same chain.
// If two blocks on different chain compared by this function, it would panic.
-func (pos *Position) Newer(other *Position) bool {
+func (pos Position) Newer(other Position) bool {
if pos.ChainID != other.ChainID {
panic(fmt.Errorf("unexpected chainID %d, should be %d",
other.ChainID, pos.ChainID))
@@ -56,7 +56,7 @@ func (pos *Position) Newer(other *Position) bool {
// Older checks if one block is older than another one on the same chain.
// If two blocks on different chain compared by this function, it would panic.
-func (pos *Position) Older(other *Position) bool {
+func (pos Position) Older(other Position) bool {
if pos.ChainID != other.ChainID {
panic(fmt.Errorf("unexpected chainID %d, should be %d",
other.ChainID, pos.ChainID))
@@ -64,12 +64,3 @@ func (pos *Position) Older(other *Position) bool {
return pos.Round < other.Round ||
(pos.Round == other.Round && pos.Height < other.Height)
}
-
-// Clone a position instance.
-func (pos *Position) Clone() *Position {
- return &Position{
- ChainID: pos.ChainID,
- Round: pos.Round,
- Height: pos.Height,
- }
-}
diff --git a/core/types/position_test.go b/core/types/position_test.go
index 5816bab..213c15f 100644
--- a/core/types/position_test.go
+++ b/core/types/position_test.go
@@ -35,24 +35,24 @@ func (s *PositionTestSuite) TestNewer() {
Height: 1,
}
s.Panics(func() {
- pos.Newer(&Position{ChainID: 2})
+ pos.Newer(Position{ChainID: 2})
})
- s.False(pos.Newer(&Position{
+ s.False(pos.Newer(Position{
Round: 2,
ChainID: 1,
Height: 0,
}))
- s.False(pos.Newer(&Position{
+ s.False(pos.Newer(Position{
Round: 1,
ChainID: 1,
Height: 2,
}))
- s.True(pos.Newer(&Position{
+ s.True(pos.Newer(Position{
Round: 0,
ChainID: 1,
Height: 100,
}))
- s.True(pos.Newer(&Position{
+ s.True(pos.Newer(Position{
Round: 1,
ChainID: 1,
Height: 0,
@@ -66,24 +66,24 @@ func (s *PositionTestSuite) TestOlder() {
Height: 1,
}
s.Panics(func() {
- pos.Older(&Position{ChainID: 2})
+ pos.Older(Position{ChainID: 2})
})
- s.False(pos.Older(&Position{
+ s.False(pos.Older(Position{
Round: 0,
ChainID: 1,
Height: 0,
}))
- s.False(pos.Older(&Position{
+ s.False(pos.Older(Position{
Round: 1,
ChainID: 1,
Height: 0,
}))
- s.True(pos.Older(&Position{
+ s.True(pos.Older(Position{
Round: 2,
ChainID: 1,
Height: 0,
}))
- s.True(pos.Older(&Position{
+ s.True(pos.Older(Position{
Round: 1,
ChainID: 1,
Height: 100,
@@ -91,37 +91,37 @@ func (s *PositionTestSuite) TestOlder() {
}
func (s *PositionTestSuite) TestSearchInAsendingOrder() {
- positions := []*Position{
- &Position{Round: 0, Height: 1},
- &Position{Round: 0, Height: 2},
- &Position{Round: 0, Height: 3},
- &Position{Round: 2, Height: 0},
- &Position{Round: 2, Height: 1},
- &Position{Round: 2, Height: 2},
- &Position{Round: 4, Height: 0},
- &Position{Round: 4, Height: 1},
- &Position{Round: 4, Height: 2},
+ positions := []Position{
+ Position{Round: 0, Height: 1},
+ Position{Round: 0, Height: 2},
+ Position{Round: 0, Height: 3},
+ Position{Round: 2, Height: 0},
+ Position{Round: 2, Height: 1},
+ Position{Round: 2, Height: 2},
+ Position{Round: 4, Height: 0},
+ Position{Round: 4, Height: 1},
+ Position{Round: 4, Height: 2},
}
- search := func(pos *Position) int {
+ search := func(pos Position) int {
return sort.Search(len(positions), func(i int) bool {
return positions[i].Newer(pos) || positions[i].Equal(pos)
})
}
- s.Equal(0, search(&Position{Round: 0, Height: 0}))
- s.Equal(len(positions), search(&Position{Round: 4, Height: 4}))
- s.Equal(0, search(&Position{Round: 0, Height: 1}))
- s.Equal(len(positions)-1, search(&Position{Round: 4, Height: 2}))
- s.Equal(2, search(&Position{Round: 0, Height: 3}))
+ s.Equal(0, search(Position{Round: 0, Height: 0}))
+ s.Equal(len(positions), search(Position{Round: 4, Height: 4}))
+ s.Equal(0, search(Position{Round: 0, Height: 1}))
+ s.Equal(len(positions)-1, search(Position{Round: 4, Height: 2}))
+ s.Equal(2, search(Position{Round: 0, Height: 3}))
}
func (s *PositionTestSuite) TestEqual() {
pos := Position{}
s.Panics(func() {
- pos.Equal(&Position{ChainID: 1})
+ pos.Equal(Position{ChainID: 1})
})
- s.True(pos.Equal(&Position{}))
- s.False(pos.Equal(&Position{Round: 1}))
- s.False(pos.Equal(&Position{Height: 1}))
+ s.True(pos.Equal(Position{}))
+ s.False(pos.Equal(Position{Round: 1}))
+ s.False(pos.Equal(Position{Height: 1}))
}
func TestPosition(t *testing.T) {
diff --git a/core/types/vote.go b/core/types/vote.go
index 46ea1df..6481eb4 100644
--- a/core/types/vote.go
+++ b/core/types/vote.go
@@ -68,7 +68,7 @@ type Vote struct {
func (v *Vote) String() string {
return fmt.Sprintf("Vote{BP:%s %s Period:%d Type:%d Hash:%s}",
v.ProposerID.String()[:6],
- &v.Position, v.Period, v.Type, v.BlockHash.String()[:6])
+ v.Position, v.Period, v.Type, v.BlockHash.String()[:6])
}
// NewVote constructs a Vote instance with header fields.
diff --git a/core/utils/nodeset-cache.go b/core/utils/nodeset-cache.go
index 8a07c9d..8354128 100644
--- a/core/utils/nodeset-cache.go
+++ b/core/utils/nodeset-cache.go
@@ -19,6 +19,7 @@ package utils
import (
"errors"
+ "fmt"
"sync"
"github.com/dexon-foundation/dexon-consensus/common"
@@ -40,9 +41,9 @@ var (
type sets struct {
crs common.Hash
nodeSet *types.NodeSet
- notarySet []map[types.NodeID]struct{}
+ notarySet map[types.NodeID]struct{}
dkgSet map[types.NodeID]struct{}
- leaderNode []map[uint64]types.NodeID
+ leaderNode map[uint64]types.NodeID
}
// NodeSetCacheInterface interface specifies interface used by NodeSetCache.
@@ -112,30 +113,29 @@ func (cache *NodeSetCache) GetPublicKey(
}
// GetNodeSet returns IDs of nodes set of this round as map.
-func (cache *NodeSetCache) GetNodeSet(
- round uint64) (nIDs *types.NodeSet, err error) {
-
+func (cache *NodeSetCache) GetNodeSet(round uint64) (*types.NodeSet, error) {
IDs, exists := cache.get(round)
if !exists {
+ var err error
if IDs, err = cache.update(round); err != nil {
- return
+ return nil, err
}
}
- nIDs = IDs.nodeSet.Clone()
- return
+ return IDs.nodeSet.Clone(), nil
}
// GetNotarySet returns of notary set of this round.
+// TODO(mission): remove chainID parameter.
func (cache *NodeSetCache) GetNotarySet(
round uint64, chainID uint32) (map[types.NodeID]struct{}, error) {
+ if chainID != 0 {
+ panic(fmt.Errorf("non-zero chainID found: %d", chainID))
+ }
IDs, err := cache.getOrUpdate(round)
if err != nil {
return nil, err
}
- if chainID >= uint32(len(IDs.notarySet)) {
- return nil, ErrInvalidChainID
- }
- return cache.cloneMap(IDs.notarySet[chainID]), nil
+ return cache.cloneMap(IDs.notarySet), nil
}
// GetDKGSet returns of DKG set of this round.
@@ -155,24 +155,21 @@ func (cache *NodeSetCache) GetLeaderNode(pos types.Position) (
if err != nil {
return types.NodeID{}, err
}
- if pos.ChainID >= uint32(len(IDs.leaderNode)) {
- return types.NodeID{}, ErrInvalidChainID
- }
cache.lock.Lock()
defer cache.lock.Unlock()
- if _, exist := IDs.leaderNode[pos.ChainID][pos.Height]; !exist {
- notarySet := types.NewNodeSetFromMap(IDs.notarySet[pos.ChainID])
- leader :=
- notarySet.GetSubSet(1, types.NewNodeLeaderTarget(IDs.crs, pos))
+ if _, exist := IDs.leaderNode[pos.Height]; !exist {
+ notarySet := types.NewNodeSetFromMap(IDs.notarySet)
+ leader := notarySet.GetSubSet(1, types.NewNodeLeaderTarget(
+ IDs.crs, pos.Height))
if len(leader) != 1 {
panic(errors.New("length of leader is not one"))
}
for nID := range leader {
- IDs.leaderNode[pos.ChainID][pos.Height] = nID
+ IDs.leaderNode[pos.Height] = nID
break
}
}
- return IDs.leaderNode[pos.ChainID][pos.Height], nil
+ return IDs.leaderNode[pos.Height], nil
}
func (cache *NodeSetCache) cloneMap(
@@ -235,23 +232,17 @@ func (cache *NodeSetCache) update(
err = ErrConfigurationNotReady
return
}
+ nodesPerChain := cfg.RoundInterval / cfg.MinBlockInterval
nIDs = &sets{
crs: crs,
nodeSet: nodeSet,
- notarySet: make([]map[types.NodeID]struct{}, cfg.NumChains),
+ notarySet: make(map[types.NodeID]struct{}),
dkgSet: nodeSet.GetSubSet(
int(cfg.DKGSetSize), types.NewDKGSetTarget(crs)),
- leaderNode: make([]map[uint64]types.NodeID, cfg.NumChains),
- }
- for i := range nIDs.notarySet {
- nIDs.notarySet[i] = nodeSet.GetSubSet(
- int(cfg.NotarySetSize), types.NewNotarySetTarget(crs, uint32(i)))
+ leaderNode: make(map[uint64]types.NodeID, nodesPerChain),
}
- nodesPerChain := cfg.RoundInterval / cfg.MinBlockInterval
- for i := range nIDs.leaderNode {
- nIDs.leaderNode[i] = make(map[uint64]types.NodeID, nodesPerChain)
- }
-
+ nIDs.notarySet = nodeSet.GetSubSet(
+ int(cfg.NotarySetSize), types.NewNotarySetTarget(crs))
cache.rounds[round] = nIDs
// Purge older rounds.
for rID, nIDs := range cache.rounds {
diff --git a/core/utils/nodeset-cache_test.go b/core/utils/nodeset-cache_test.go
index c600f15..eb6008d 100644
--- a/core/utils/nodeset-cache_test.go
+++ b/core/utils/nodeset-cache_test.go
@@ -97,7 +97,7 @@ func (s *NodeSetCacheTestSuite) TestBasicUsage() {
chk(cache, 0, dkgSet)
leaderNode, err := cache.GetLeaderNode(types.Position{
Round: uint64(0),
- ChainID: uint32(3),
+ ChainID: uint32(0),
Height: uint64(10),
})
req.NoError(err)
diff --git a/integration_test/consensus_test.go b/integration_test/consensus_test.go
index 8af118b..454ed1e 100644
--- a/integration_test/consensus_test.go
+++ b/integration_test/consensus_test.go
@@ -239,10 +239,10 @@ Loop:
s.verifyNodes(nodes)
}
-func (s *ConsensusTestSuite) TestNumChainsChange() {
+func (s *ConsensusTestSuite) TestSetSizeChange() {
var (
req = s.Require()
- peerCount = 4
+ peerCount = 7
dMoment = time.Now().UTC()
untilRound = uint64(6)
)
@@ -261,24 +261,34 @@ func (s *ConsensusTestSuite) TestNumChainsChange() {
req.NoError(err)
req.NoError(seedGov.State().RequestChange(
test.StateChangeRoundInterval, 45*time.Second))
+ req.NoError(seedGov.State().RequestChange(
+ test.StateChangeNotarySetSize, uint32(4)))
+ req.NoError(seedGov.State().RequestChange(
+ test.StateChangeDKGSetSize, uint32(4)))
seedGov.CatchUpWithRound(0)
// Setup configuration for round 0 and round 1.
req.NoError(seedGov.State().RequestChange(
- test.StateChangeNumChains, uint32(5)))
- req.NoError(seedGov.State().RequestChange(
test.StateChangeRoundInterval, 55*time.Second))
+ req.NoError(seedGov.State().RequestChange(
+ test.StateChangeNotarySetSize, uint32(5)))
+ req.NoError(seedGov.State().RequestChange(
+ test.StateChangeDKGSetSize, uint32(6)))
seedGov.CatchUpWithRound(1)
// Setup configuration for round 2.
req.NoError(seedGov.State().RequestChange(
- test.StateChangeNumChains, uint32(6)))
- req.NoError(seedGov.State().RequestChange(
test.StateChangeRoundInterval, 55*time.Second))
+ req.NoError(seedGov.State().RequestChange(
+ test.StateChangeNotarySetSize, uint32(6)))
+ req.NoError(seedGov.State().RequestChange(
+ test.StateChangeDKGSetSize, uint32(5)))
seedGov.CatchUpWithRound(2)
// Setup configuration for round 3.
req.NoError(seedGov.State().RequestChange(
- test.StateChangeNumChains, uint32(5)))
- req.NoError(seedGov.State().RequestChange(
test.StateChangeRoundInterval, 75*time.Second))
+ req.NoError(seedGov.State().RequestChange(
+ test.StateChangeNotarySetSize, uint32(4)))
+ req.NoError(seedGov.State().RequestChange(
+ test.StateChangeDKGSetSize, uint32(4)))
seedGov.CatchUpWithRound(3)
// Setup nodes.
nodes := s.setupNodes(dMoment, prvKeys, seedGov)
@@ -289,14 +299,18 @@ func (s *ConsensusTestSuite) TestNumChainsChange() {
}
// Register configuration changes for round 4.
req.NoError(pickedNode.gov.RegisterConfigChange(
- 4, test.StateChangeNumChains, uint32(4)))
- req.NoError(pickedNode.gov.RegisterConfigChange(
4, test.StateChangeRoundInterval, 45*time.Second))
+ req.NoError(seedGov.State().RequestChange(
+ test.StateChangeNotarySetSize, uint32(5)))
+ req.NoError(seedGov.State().RequestChange(
+ test.StateChangeDKGSetSize, uint32(5)))
// Register configuration changes for round 5.
req.NoError(pickedNode.gov.RegisterConfigChange(
- 5, test.StateChangeNumChains, uint32(5)))
- req.NoError(pickedNode.gov.RegisterConfigChange(
5, test.StateChangeRoundInterval, 55*time.Second))
+ req.NoError(seedGov.State().RequestChange(
+ test.StateChangeNotarySetSize, uint32(4)))
+ req.NoError(seedGov.State().RequestChange(
+ test.StateChangeDKGSetSize, uint32(4)))
// Run test.
for _, n := range nodes {
go n.con.Run()
diff --git a/integration_test/node.go b/integration_test/node.go
deleted file mode 100644
index 732e54a..0000000
--- a/integration_test/node.go
+++ /dev/null
@@ -1,355 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package integration
-
-import (
- "fmt"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core"
- "github.com/dexon-foundation/dexon-consensus/core/crypto"
- "github.com/dexon-foundation/dexon-consensus/core/db"
- "github.com/dexon-foundation/dexon-consensus/core/test"
- "github.com/dexon-foundation/dexon-consensus/core/types"
- "github.com/dexon-foundation/dexon-consensus/core/utils"
-)
-
-type consensusEventType int
-
-const (
- evtProposeBlock consensusEventType = iota
- evtReceiveBlock
-)
-
-type consensusEventPayload struct {
- Type consensusEventType
- PiggyBack interface{}
-}
-
-// newProposeBlockEvent constructs an test.Event that would trigger
-// block proposing.
-func newProposeBlockEvent(nID types.NodeID,
- roundID uint64, chainID uint32, when time.Time) *test.Event {
- return test.NewEvent(nID, when, &consensusEventPayload{
- Type: evtProposeBlock,
- PiggyBack: &struct {
- round uint64
- chain uint32
- }{roundID, chainID},
- })
-}
-
-// newReceiveBlockEvent constructs an test.Event that would trigger
-// block received.
-func newReceiveBlockEvent(
- nID types.NodeID, when time.Time, block *types.Block) *test.Event {
-
- return test.NewEvent(nID, when, &consensusEventPayload{
- Type: evtReceiveBlock,
- PiggyBack: block,
- })
-}
-
-// Node is designed to work with test.Scheduler.
-type Node struct {
- ID types.NodeID
- ownChains []uint32
- roundEndTimes []time.Time
- roundToNotify uint64
- lattice *core.Lattice
- appModule *test.App
- stateModule *test.State
- govModule *test.Governance
- dbModule db.Database
- broadcastTargets map[types.NodeID]struct{}
- networkLatency test.LatencyModel
- proposingLatency test.LatencyModel
- prevFinalHeight uint64
- pendings []*types.Block
- prevHash common.Hash
- // This variable caches the maximum NumChains seen by this node when
- // it's notified for round switching.
- latticeMaxNumChains uint32
-}
-
-// newNode constructs an instance of Node.
-func newNode(
- gov *test.Governance,
- privateKey crypto.PrivateKey,
- dMoment time.Time,
- ownChains []uint32,
- networkLatency test.LatencyModel,
- proposingLatency test.LatencyModel) (*Node, error) {
- // Load all configs prepared in core.Governance into core.Lattice.
- copiedGov := gov.Clone()
- configs := loadAllConfigs(copiedGov)
- // Setup db.
- dbInst, err := db.NewMemBackedDB()
- if err != nil {
- return nil, err
- }
- // Setup test.App
- app := test.NewApp(0, copiedGov)
- // Setup lattice instance.
- lattice := core.NewLattice(
- dMoment,
- 0,
- configs[0],
- utils.NewSigner(privateKey),
- app,
- app,
- dbInst,
- &common.NullLogger{})
- n := &Node{
- ID: types.NewNodeID(privateKey.PublicKey()),
- ownChains: ownChains,
- roundEndTimes: genRoundEndTimes(configs, dMoment),
- roundToNotify: 2,
- networkLatency: networkLatency,
- proposingLatency: proposingLatency,
- appModule: app,
- stateModule: copiedGov.State(),
- dbModule: dbInst,
- govModule: copiedGov,
- lattice: lattice,
- latticeMaxNumChains: configs[0].NumChains,
- }
- for idx, config := range configs[1:] {
- if err := lattice.AppendConfig(uint64(idx+1), config); err != nil {
- return nil, err
- }
- if config.NumChains > n.latticeMaxNumChains {
- n.latticeMaxNumChains = config.NumChains
- }
- }
- return n, nil
-}
-
-// Handle implements test.EventHandler interface.
-func (n *Node) Handle(e *test.Event) (events []*test.Event) {
- payload := e.Payload.(*consensusEventPayload)
- switch payload.Type {
- case evtProposeBlock:
- events, e.ExecError = n.handleProposeBlock(e.Time, payload.PiggyBack)
- case evtReceiveBlock:
- events, e.ExecError = n.handleReceiveBlock(payload.PiggyBack)
- default:
- panic(fmt.Errorf("unknown consensus event type: %v", payload.Type))
- }
- return
-}
-
-func (n *Node) handleProposeBlock(when time.Time, payload interface{}) (
- events []*test.Event, err error) {
- pos := payload.(*struct {
- round uint64
- chain uint32
- })
- b, err := n.prepareBlock(pos.round, pos.chain, when)
- if err != nil {
- if err == utils.ErrInvalidChainID {
- // This chain is not included in this round, retry in next round.
- events = append(events, newProposeBlockEvent(
- n.ID, b.Position.Round+1, b.Position.ChainID,
- n.roundEndTimes[b.Position.Round]))
- }
- return
- }
- if events, err = n.processBlock(b); err != nil {
- // It's shouldn't be error when prepared.
- panic(err)
- }
- // Create 'block received' event for each other nodes.
- for nID := range n.broadcastTargets {
- events = append(events, newReceiveBlockEvent(
- nID, when.Add(n.networkLatency.Delay()), b.Clone()))
- }
- // Create next 'block proposing' event for this nodes.
- events = append(events, newProposeBlockEvent(n.ID,
- b.Position.Round,
- b.Position.ChainID,
- when.Add(n.proposingLatency.Delay())))
- return
-}
-
-func (n *Node) handleReceiveBlock(piggyback interface{}) (
- events []*test.Event, err error) {
- events, err = n.processBlock(piggyback.(*types.Block))
- if err != nil {
- panic(err)
- }
- return
-}
-
-func (n *Node) prepareBlock(
- round uint64, chainID uint32, when time.Time) (b *types.Block, err error) {
- b = &types.Block{
- Position: types.Position{
- Round: round,
- ChainID: chainID,
- }}
- if err = n.lattice.PrepareBlock(b, when); err != nil {
- if err == core.ErrRoundNotSwitch {
- b.Position.Round++
- err = n.lattice.PrepareBlock(b, when)
- }
- }
- return
-}
-
-func (n *Node) processBlock(b *types.Block) (events []*test.Event, err error) {
- // TODO(mission): this segment of code is identical to testLatticeMgr in
- // core/lattice_test.go, except the compaction-chain part.
- var (
- delivered []*types.Block
- )
- n.pendings = append([]*types.Block{b}, n.pendings...)
- for {
- var (
- newPendings []*types.Block
- tmpDelivered []*types.Block
- tmpErr error
- )
- updated := false
- for _, p := range n.pendings {
- if tmpErr = n.lattice.SanityCheck(p, false); tmpErr != nil {
- if tmpErr == core.ErrRetrySanityCheckLater {
- newPendings = append(newPendings, p)
- } else {
- // Those blocks are prepared by lattice module, they should
- // not be wrong.
- panic(tmpErr)
- }
- continue
- }
- if tmpDelivered, tmpErr =
- n.lattice.ProcessBlock(p); tmpErr != nil {
- // It's not allowed that sanity checked block failed to
- // be added to lattice.
- panic(tmpErr)
- }
- delivered = append(delivered, tmpDelivered...)
- updated = true
- }
- n.pendings = newPendings
- if !updated {
- break
- }
- }
- // Deliver blocks.
- for _, b = range delivered {
- b.Finalization.Height = n.prevFinalHeight + 1
- b.Finalization.ParentHash = n.prevHash
- n.appModule.BlockDelivered(b.Hash, b.Position, b.Finalization)
- n.prevFinalHeight++
- n.prevHash = b.Hash
- events = append(events, n.checkRoundSwitch(b)...)
- }
- if err = n.lattice.PurgeBlocks(delivered); err != nil {
- panic(err)
- }
- return
-}
-
-func (n *Node) checkRoundSwitch(b *types.Block) (evts []*test.Event) {
- if !b.Timestamp.After(n.roundEndTimes[b.Position.Round]) {
- return
- }
- if b.Position.Round+2 != n.roundToNotify {
- return
- }
- // Handle round switching logic.
- n.govModule.NotifyRound(n.roundToNotify)
- if n.roundToNotify == uint64(len(n.roundEndTimes)) {
- config := n.govModule.Configuration(n.roundToNotify)
- if config == nil {
- panic(fmt.Errorf(
- "config is not ready for round: %v", n.roundToNotify-1))
- }
- // Cache round ended time for each round.
- n.roundEndTimes = append(n.roundEndTimes,
- n.roundEndTimes[len(n.roundEndTimes)-1].Add(
- config.RoundInterval))
- // Add new config to lattice module.
- if err := n.lattice.AppendConfig(n.roundToNotify, config); err != nil {
- panic(err)
- }
- if config.NumChains > n.latticeMaxNumChains {
- // We can be sure that lattice module can support this number of
- // chains.
- for _, chainID := range n.ownChains {
- if chainID < n.latticeMaxNumChains {
- continue
- }
- if chainID >= config.NumChains {
- continue
- }
- // For newly added chains, add block proposing seed event.
- evts = append(evts, newProposeBlockEvent(n.ID, n.roundToNotify,
- chainID, n.roundEndTimes[n.roundToNotify-1]))
- }
- n.latticeMaxNumChains = config.NumChains
- }
- } else if n.roundToNotify > uint64(len(n.roundEndTimes)) {
- panic(fmt.Errorf(
- "config notification not incremental: %v, cached configs: %v",
- n.roundToNotify, len(n.roundEndTimes)))
- }
- n.roundToNotify++
- return
-}
-
-// Bootstrap this node with block proposing event.
-func (n *Node) Bootstrap(sch *test.Scheduler, now time.Time) (err error) {
- sch.RegisterEventHandler(n.ID, n)
- for _, chainID := range n.ownChains {
- if chainID >= n.latticeMaxNumChains {
- continue
- }
- err = sch.Seed(newProposeBlockEvent(n.ID, 0, chainID, now))
- if err != nil {
- return
- }
- }
- return
-}
-
-func (n *Node) setBroadcastTargets(targets map[types.NodeID]struct{}) {
- // Clone targets, except self.
- targetsCopy := make(map[types.NodeID]struct{})
- for nID := range targets {
- if nID == n.ID {
- continue
- }
- targetsCopy[nID] = struct{}{}
- }
- n.broadcastTargets = targetsCopy
-}
-
-func (n *Node) app() *test.App {
- return n.appModule
-}
-
-func (n *Node) db() db.Database {
- return n.dbModule
-}
-
-func (n *Node) gov() *test.Governance {
- return n.govModule
-}
diff --git a/integration_test/stats.go b/integration_test/stats.go
deleted file mode 100644
index 5d66e29..0000000
--- a/integration_test/stats.go
+++ /dev/null
@@ -1,192 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package integration
-
-import (
- "fmt"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/core/test"
- "github.com/dexon-foundation/dexon-consensus/core/types"
-)
-
-// Errors when calculating statistics for events.
-var (
- ErrUnknownEvent = fmt.Errorf("unknown event")
- ErrUnknownConsensusEventType = fmt.Errorf("unknown consensus event type")
-)
-
-// StatsSet represents accumulatee result of a group of related events
-// (ex. All events from one node).
-type StatsSet struct {
- ProposedBlockCount int
- ReceivedBlockCount int
- ConfirmedBlockCount int
- TotalOrderedBlockCount int
- DeliveredBlockCount int
- ProposingLatency time.Duration
- ReceivingLatency time.Duration
- PrepareExecLatency time.Duration
- ProcessExecLatency time.Duration
-}
-
-// newBlockProposeEvent accumulates a block proposing event.
-func (s *StatsSet) newBlockProposeEvent(
- e *test.Event, payload *consensusEventPayload, history []*test.Event) {
-
- // Find previous block proposing event.
- if e.ParentHistoryIndex != -1 {
- parentEvent := history[e.ParentHistoryIndex]
- s.ProposingLatency +=
- e.Time.Sub(parentEvent.Time) - parentEvent.ExecInterval
- }
- s.PrepareExecLatency += e.ExecInterval
- s.ProposedBlockCount++
-}
-
-// newBlockReceiveEvent accumulates a block received event.
-func (s *StatsSet) newBlockReceiveEvent(
- e *test.Event,
- payload *consensusEventPayload,
- history []*test.Event,
- app *test.App) {
-
- // Find previous block proposing event.
- parentEvent := history[e.ParentHistoryIndex]
- s.ReceivingLatency +=
- e.Time.Sub(parentEvent.Time) - parentEvent.ExecInterval
- s.ProcessExecLatency += e.ExecInterval
- s.ReceivedBlockCount++
-
- // Find statistics from test.App
- block := payload.PiggyBack.(*types.Block)
- app.WithLock(func(app *test.App) {
- // Is this block confirmed?
- if _, exists := app.Confirmed[block.Hash]; !exists {
- return
- }
- s.ConfirmedBlockCount++
- // Is this block total ordered?
- if _, exists := app.TotalOrderedByHash[block.Hash]; !exists {
- return
- }
- s.TotalOrderedBlockCount++
-
- // Is this block delivered?
- if _, exists := app.Delivered[block.Hash]; !exists {
- return
- }
- s.DeliveredBlockCount++
- })
-}
-
-// done would divide the latencies we cached with related event count. This way
-// to calculate average latency is more accurate.
-func (s *StatsSet) done(nodeCount int) {
- s.ProposingLatency /= time.Duration(s.ProposedBlockCount - nodeCount)
- s.ReceivingLatency /= time.Duration(s.ReceivedBlockCount)
- s.PrepareExecLatency /= time.Duration(s.ProposedBlockCount)
- s.ProcessExecLatency /= time.Duration(s.ReceivedBlockCount)
-}
-
-// Stats is statistics of a slice of test.Event generated by nodes.
-type Stats struct {
- ByNode map[types.NodeID]*StatsSet
- All *StatsSet
- BPS float64
- ExecutionTime time.Duration
-}
-
-// NewStats constructs an Stats instance by providing a slice of
-// test.Event.
-func NewStats(
- history []*test.Event, apps map[types.NodeID]*test.App) (
- stats *Stats, err error) {
-
- stats = &Stats{
- ByNode: make(map[types.NodeID]*StatsSet),
- All: &StatsSet{},
- }
- if err = stats.calculate(history, apps); err != nil {
- stats = nil
- }
- stats.summary(history)
- return
-}
-
-func (stats *Stats) calculate(
- history []*test.Event, apps map[types.NodeID]*test.App) error {
-
- defer func() {
- stats.All.done(len(stats.ByNode))
- for _, set := range stats.ByNode {
- set.done(1)
- }
- }()
-
- for _, e := range history {
- payload, ok := e.Payload.(*consensusEventPayload)
- if !ok {
- return ErrUnknownEvent
- }
- switch payload.Type {
- case evtProposeBlock:
- stats.All.newBlockProposeEvent(
- e, payload, history)
- stats.getStatsSetByNode(e.NodeID).newBlockProposeEvent(
- e, payload, history)
- case evtReceiveBlock:
- stats.All.newBlockReceiveEvent(
- e, payload, history, apps[e.NodeID])
- stats.getStatsSetByNode(e.NodeID).newBlockReceiveEvent(
- e, payload, history, apps[e.NodeID])
- default:
- return ErrUnknownConsensusEventType
- }
- }
- return nil
-}
-
-func (stats *Stats) getStatsSetByNode(
- vID types.NodeID) (s *StatsSet) {
-
- s = stats.ByNode[vID]
- if s == nil {
- s = &StatsSet{}
- stats.ByNode[vID] = s
- }
- return
-}
-
-func (stats *Stats) summary(history []*test.Event) {
- // Find average delivered block count among all blocks.
- totalConfirmedBlocks := 0
- for _, s := range stats.ByNode {
- totalConfirmedBlocks += s.DeliveredBlockCount
- }
- averageConfirmedBlocks := totalConfirmedBlocks / len(stats.ByNode)
-
- // Find execution time.
- // Note: it's a simplified way to calculate the execution time:
- // the latest event might not be at the end of history when
- // the number of worker routine is larger than 1.
- stats.ExecutionTime = history[len(history)-1].Time.Sub(history[0].Time)
- // Calculate BPS.
- latencyAsSecond := stats.ExecutionTime.Nanoseconds() / (1000 * 1000 * 1000)
- stats.BPS = float64(averageConfirmedBlocks) / float64(latencyAsSecond)
-}
diff --git a/integration_test/stats_test.go b/integration_test/stats_test.go
deleted file mode 100644
index 95bd5ac..0000000
--- a/integration_test/stats_test.go
+++ /dev/null
@@ -1,84 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package integration
-
-import (
- "testing"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core"
- "github.com/dexon-foundation/dexon-consensus/core/test"
- "github.com/stretchr/testify/suite"
-)
-
-type EventStatsTestSuite struct {
- suite.Suite
-}
-
-func (s *EventStatsTestSuite) TestCalculate() {
- // Setup a test with fixed latency in proposing and network,
- // and make sure the calculated statistics is expected.
- var (
- networkLatency = &test.FixedLatencyModel{Latency: 100}
- proposingLatency = &test.FixedLatencyModel{Latency: 300}
- req = s.Require()
- )
- prvKeys, pubKeys, err := test.NewKeys(7)
- req.NoError(err)
- gov, err := test.NewGovernance(
- test.NewState(
- pubKeys, 100*time.Millisecond, &common.NullLogger{}, true),
- core.ConfigRoundShift)
- req.NoError(err)
- nodes, err := PrepareNodes(
- gov, prvKeys, 7, networkLatency, proposingLatency)
- req.NoError(err)
- apps, dbs := CollectAppAndDBFromNodes(nodes)
- sch := test.NewScheduler(test.NewStopByConfirmedBlocks(50, apps, dbs))
- now := time.Now().UTC()
- for _, n := range nodes {
- req.NoError(n.Bootstrap(sch, now))
- }
- sch.Run(10)
- req.Nil(VerifyApps(apps))
- // Check total statistics result.
- stats, err := NewStats(sch.CloneExecutionHistory(), apps)
- req.Nil(err)
- req.True(stats.All.ProposedBlockCount > 350)
- req.True(stats.All.ReceivedBlockCount > 350)
- req.True(stats.All.ConfirmedBlockCount > 350)
- req.True(stats.All.TotalOrderedBlockCount >= 350)
- req.True(stats.All.DeliveredBlockCount >= 350)
- req.Equal(stats.All.ProposingLatency, 300*time.Millisecond)
- req.Equal(stats.All.ReceivingLatency, 100*time.Millisecond)
- // Check statistics for each node.
- for _, vStats := range stats.ByNode {
- req.True(vStats.ProposedBlockCount > 50)
- req.True(vStats.ReceivedBlockCount > 50)
- req.True(vStats.ConfirmedBlockCount > 50)
- req.True(vStats.TotalOrderedBlockCount >= 50)
- req.True(vStats.DeliveredBlockCount >= 50)
- req.Equal(vStats.ProposingLatency, 300*time.Millisecond)
- req.Equal(vStats.ReceivingLatency, 100*time.Millisecond)
- }
-}
-
-func TestEventStats(t *testing.T) {
- suite.Run(t, new(EventStatsTestSuite))
-}
diff --git a/integration_test/utils.go b/integration_test/utils.go
deleted file mode 100644
index 2fbfa1a..0000000
--- a/integration_test/utils.go
+++ /dev/null
@@ -1,140 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package integration
-
-import (
- "errors"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/core"
- "github.com/dexon-foundation/dexon-consensus/core/crypto"
- "github.com/dexon-foundation/dexon-consensus/core/db"
- "github.com/dexon-foundation/dexon-consensus/core/test"
- "github.com/dexon-foundation/dexon-consensus/core/types"
-)
-
-func genRoundEndTimes(
- configs []*types.Config, dMoment time.Time) (ends []time.Time) {
- now := dMoment
- for _, config := range configs {
- now = now.Add(config.RoundInterval)
- ends = append(ends, now)
- }
- return
-}
-
-// loadAllConfigs loads all prepared configuration from governance,
-// starts from round 0.
-func loadAllConfigs(gov core.Governance) (configs []*types.Config) {
- var round uint64
- for {
- config := gov.Configuration(round)
- if config == nil {
- break
- }
- configs = append(configs, config)
- round++
- }
- return
-}
-
-// decideOwnChains compute which chainIDs belongs to this node.
-func decideOwnChains(numChains uint32, numNodes, id int) (own []uint32) {
- var cur = uint32(id)
- if numNodes == 0 {
- panic(errors.New("attempt to arrange chains on 0 nodes"))
- }
- for {
- if cur >= numChains {
- break
- }
- own = append(own, cur)
- cur += uint32(numNodes)
- }
- return
-}
-
-// PrepareNodes setups nodes for testing.
-func PrepareNodes(
- gov *test.Governance,
- prvKeys []crypto.PrivateKey,
- maxNumChains uint32,
- networkLatency, proposingLatency test.LatencyModel) (
- nodes map[types.NodeID]*Node, err error) {
- if maxNumChains == 0 {
- err = errors.New("zero NumChains is unexpected")
- return
- }
- // Setup nodes, count of nodes is derived from the count of private keys
- // hold in Governance.
- nodes = make(map[types.NodeID]*Node)
- dMoment := time.Now().UTC()
- broadcastTargets := make(map[types.NodeID]struct{})
- for idx, prvKey := range prvKeys {
- nID := types.NewNodeID(prvKey.PublicKey())
- broadcastTargets[nID] = struct{}{}
- // Decides which chains are owned by this node.
- if nodes[nID], err = newNode(
- gov,
- prvKey,
- dMoment,
- decideOwnChains(maxNumChains, len(prvKeys), idx),
- networkLatency,
- proposingLatency); err != nil {
- return
- }
- }
- // Assign broadcast targets.
- for _, n := range nodes {
- n.setBroadcastTargets(broadcastTargets)
- n.gov().State().SwitchToRemoteMode()
- }
- return
-}
-
-// VerifyApps is a helper to check delivery between test.Apps
-func VerifyApps(apps map[types.NodeID]*test.App) (err error) {
- for vFrom, fromApp := range apps {
- if err = fromApp.Verify(); err != nil {
- return
- }
- for vTo, toApp := range apps {
- if vFrom == vTo {
- continue
- }
- if err = fromApp.Compare(toApp); err != nil {
- return
- }
- }
- }
- return
-}
-
-// CollectAppAndDBFromNodes collects test.App and db.Database
-// from nodes.
-func CollectAppAndDBFromNodes(nodes map[types.NodeID]*Node) (
- apps map[types.NodeID]*test.App,
- dbs map[types.NodeID]db.Database) {
- apps = make(map[types.NodeID]*test.App)
- dbs = make(map[types.NodeID]db.Database)
- for nID, node := range nodes {
- apps[nID] = node.app()
- dbs[nID] = node.db()
- }
- return
-}
diff --git a/integration_test/utils_test.go b/integration_test/utils_test.go
deleted file mode 100644
index 99b62c1..0000000
--- a/integration_test/utils_test.go
+++ /dev/null
@@ -1,61 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package integration
-
-import (
- "testing"
-
- "github.com/stretchr/testify/suite"
-)
-
-type UtilsTestSuite struct {
- suite.Suite
-}
-
-func (s *UtilsTestSuite) TestDecideOwnChains() {
- // Basic test for each node index.
- s.Empty(decideOwnChains(1, 1, 1))
- s.Equal(decideOwnChains(1, 1, 0), []uint32{0})
- s.Equal(decideOwnChains(30, 7, 4), []uint32{4, 11, 18, 25})
- // Make sure every chain is covered.
- isAllCovered := func(numChains uint32, numNodes int) bool {
- if numNodes == 0 {
- decideOwnChains(numChains, numNodes, 0)
- return false
- }
- covered := make(map[uint32]struct{})
- for i := 0; i < numNodes; i++ {
- for _, chainID := range decideOwnChains(numChains, numNodes, i) {
- s.Require().True(chainID < numChains)
- covered[chainID] = struct{}{}
- }
- }
- return uint32(len(covered)) == numChains
- }
- s.True(isAllCovered(100, 33))
- s.True(isAllCovered(100, 200))
- s.True(isAllCovered(100, 50))
- s.True(isAllCovered(100, 1))
- s.Panics(func() {
- isAllCovered(100, 0)
- })
-}
-
-func TestUtils(t *testing.T) {
- suite.Run(t, new(UtilsTestSuite))
-}
diff --git a/integration_test/with_scheduler_test.go b/integration_test/with_scheduler_test.go
deleted file mode 100644
index 66fc9b8..0000000
--- a/integration_test/with_scheduler_test.go
+++ /dev/null
@@ -1,150 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package integration
-
-import (
- "testing"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core"
- "github.com/dexon-foundation/dexon-consensus/core/test"
- "github.com/stretchr/testify/suite"
-)
-
-type WithSchedulerTestSuite struct {
- suite.Suite
-}
-
-func (s *WithSchedulerTestSuite) TestNonByzantine() {
- var (
- networkLatency = &test.NormalLatencyModel{
- Sigma: 20,
- Mean: 250,
- }
- proposingLatency = &test.NormalLatencyModel{
- Sigma: 30,
- Mean: 500,
- }
- numNodes = 25
- req = s.Require()
- )
- if testing.Short() {
- numNodes = 7
- }
- // Setup key pairs.
- prvKeys, pubKeys, err := test.NewKeys(numNodes)
- req.NoError(err)
- // Setup governance.
- gov, err := test.NewGovernance(
- test.NewState(
- pubKeys, 250*time.Millisecond, &common.NullLogger{}, true),
- core.ConfigRoundShift)
- req.NoError(err)
- // Setup nodes.
- nodes, err := PrepareNodes(
- gov, prvKeys, 25, networkLatency, proposingLatency)
- req.NoError(err)
- // Setup scheduler.
- apps, dbs := CollectAppAndDBFromNodes(nodes)
- now := time.Now().UTC()
- sch := test.NewScheduler(test.NewStopByConfirmedBlocks(50, apps, dbs))
- for _, n := range nodes {
- req.NoError(n.Bootstrap(sch, now))
- }
- sch.Run(4)
- // Check results by comparing test.App instances.
- req.NoError(VerifyApps(apps))
-}
-
-func (s *WithSchedulerTestSuite) TestConfigurationChange() {
- // This test case verify the correctness of core.Lattice when configuration
- // changes.
- // - Configuration changes are registered at 'pickedNode', and would carried
- // in blocks' payload and broadcast to other nodes.
- var (
- networkLatency = &test.NormalLatencyModel{
- Sigma: 20,
- Mean: 250,
- }
- proposingLatency = &test.NormalLatencyModel{
- Sigma: 30,
- Mean: 500,
- }
- numNodes = 4
- req = s.Require()
- maxNumChains = uint32(9)
- )
- // Setup key pairs.
- prvKeys, pubKeys, err := test.NewKeys(numNodes)
- req.NoError(err)
- // Setup governance.
- gov, err := test.NewGovernance(
- test.NewState(
- pubKeys, 250*time.Millisecond, &common.NullLogger{}, true),
- core.ConfigRoundShift)
- req.NoError(err)
- // Change default round interval, expect 1 round produce 30 blocks.
- gov.State().RequestChange(test.StateChangeRoundInterval, 15*time.Second)
- // Setup nodes.
- nodes, err := PrepareNodes(
- gov, prvKeys, maxNumChains, networkLatency, proposingLatency)
- req.NoError(err)
- // Set scheduler.
- apps, dbs := CollectAppAndDBFromNodes(nodes)
- now := time.Now().UTC()
- sch := test.NewScheduler(test.NewStopByRound(9, apps, dbs))
- for _, n := range nodes {
- req.NoError(n.Bootstrap(sch, now))
- }
- // Register some configuration changes at some node.
- var pickedNode *Node
- for _, pickedNode = range nodes {
- break
- }
- // Config changes for round 5, numChains from 4 to 7.
- req.NoError(pickedNode.gov().RegisterConfigChange(
- 5, test.StateChangeNumChains, uint32(7)))
- req.NoError(pickedNode.gov().RegisterConfigChange(
- 5, test.StateChangeK, 3))
- req.NoError(pickedNode.gov().RegisterConfigChange(
- 5, test.StateChangePhiRatio, float32(0.5)))
- // Config changes for round 6, numChains from 7 to 9.
- req.NoError(pickedNode.gov().RegisterConfigChange(
- 6, test.StateChangeNumChains, maxNumChains))
- req.NoError(pickedNode.gov().RegisterConfigChange(
- 6, test.StateChangeK, 0))
- // Config changes for round 7, numChains from 9 to 7.
- req.NoError(pickedNode.gov().RegisterConfigChange(
- 7, test.StateChangeNumChains, uint32(7)))
- req.NoError(pickedNode.gov().RegisterConfigChange(
- 7, test.StateChangeK, 1))
- // Config changes for round 8, numChains from 7 to 5.
- req.NoError(pickedNode.gov().RegisterConfigChange(
- 8, test.StateChangeNumChains, uint32(5)))
- req.NoError(pickedNode.gov().RegisterConfigChange(
- 8, test.StateChangeK, 1))
- // Perform test.
- sch.Run(4)
- // Check results by comparing test.App instances.
- req.NoError(VerifyApps(apps))
-}
-
-func TestWithScheduler(t *testing.T) {
- suite.Run(t, new(WithSchedulerTestSuite))
-}
diff --git a/simulation/app.go b/simulation/app.go
index 9f399ce..1a67a01 100644
--- a/simulation/app.go
+++ b/simulation/app.go
@@ -150,29 +150,6 @@ func (a *simApp) PrepareWitness(height uint64) (types.Witness, error) {
return a.latestWitness, nil
}
-// TotalOrderingDelivered is called when blocks are delivered by the total
-// ordering algorithm.
-func (a *simApp) TotalOrderingDelivered(
- blockHashes common.Hashes, mode uint32) {
- fmt.Println("OUTPUT", a.NodeID, mode, blockHashes)
- latencies := []time.Duration{}
- func() {
- a.lock.RLock()
- defer a.lock.RUnlock()
- for _, h := range blockHashes {
- latencies = append(latencies, time.Since(a.blockTimestamps[h][blockEventReceived]))
- }
- }()
- blockList := &BlockList{
- ID: a.DeliverID,
- BlockHash: blockHashes,
- ConfirmLatency: latencies,
- }
- // #nosec G104
- a.netModule.Report(blockList)
- a.DeliverID++
-}
-
// BlockDelivered is called when a block in compaction chain is delivered.
func (a *simApp) BlockDelivered(
blockHash common.Hash, pos types.Position, result types.FinalizationResult) {
diff --git a/simulation/block-list.go b/simulation/block-list.go
deleted file mode 100644
index 9e329a7..0000000
--- a/simulation/block-list.go
+++ /dev/null
@@ -1,64 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package simulation
-
-import (
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
-)
-
-// BlockList is the list of blocks from the result of Total Ordering Algorithm.
-type BlockList struct {
- ID int `json:"id"`
- BlockHash common.Hashes `json:"blockhash"`
- ConfirmLatency []time.Duration `json:"confirmlatency"`
- // The index is required by heap.Interface.
- index int
-}
-
-// PendingBlockList is a PrioirtyQueue maintaining the BlockList received
-// before the previous one (based on ID).
-type PendingBlockList []*BlockList
-
-// Len, Less and Swap are implementing heap.Interface
-func (p PendingBlockList) Len() int { return len(p) }
-func (p PendingBlockList) Less(i, j int) bool { return p[i].ID < p[j].ID }
-func (p PendingBlockList) Swap(i, j int) {
- p[i], p[j] = p[j], p[i]
- p[i].index = i
- p[j].index = j
-}
-
-// Push item in the Heap.
-func (p *PendingBlockList) Push(x interface{}) {
- n := len(*p)
- item := x.(*BlockList)
- item.index = n
- *p = append(*p, item)
-}
-
-// Pop the element from the Heap.
-func (p *PendingBlockList) Pop() interface{} {
- old := *p
- n := len(old)
- item := old[n-1]
- item.index = -1 // For safety.
- *p = old[0 : n-1]
- return item
-}
diff --git a/simulation/marshaller.go b/simulation/marshaller.go
index 6f8aee4..4d2b511 100644
--- a/simulation/marshaller.go
+++ b/simulation/marshaller.go
@@ -36,12 +36,6 @@ func (m *jsonMarshaller) Unmarshal(
break
}
msg = notif
- case "blocklist":
- var blocks BlockList
- if err = json.Unmarshal(payload, &blocks); err != nil {
- break
- }
- msg = &blocks
case "message":
var m message
if err = json.Unmarshal(payload, &m); err != nil {
@@ -63,8 +57,6 @@ func (m *jsonMarshaller) Marshal(msg interface{}) (
switch msg.(type) {
case serverNotification:
msgType = "server-notif"
- case *BlockList:
- msgType = "blocklist"
case *message:
msgType = "message"
default:
diff --git a/simulation/peer-server.go b/simulation/peer-server.go
index a98c59a..3a925a4 100644
--- a/simulation/peer-server.go
+++ b/simulation/peer-server.go
@@ -19,12 +19,10 @@ package simulation
import (
"context"
- "encoding/json"
"fmt"
"log"
"reflect"
"sort"
- "sync"
"time"
"github.com/dexon-foundation/dexon-consensus/common"
@@ -38,8 +36,6 @@ type PeerServer struct {
peers map[types.NodeID]struct{}
msgChannel chan *test.TransportEnvelope
trans test.TransportServer
- peerTotalOrder PeerTotalOrder
- peerTotalOrderMu sync.Mutex
verifiedLen uint64
cfg *config.Config
ctx context.Context
@@ -53,7 +49,6 @@ func NewPeerServer() *PeerServer {
ctx, cancel := context.WithCancel(context.Background())
return &PeerServer{
peers: make(map[types.NodeID]struct{}),
- peerTotalOrder: make(PeerTotalOrder),
ctx: ctx,
ctxCancel: cancel,
blockEvents: make(map[types.NodeID]map[common.Hash][]time.Time),
@@ -68,36 +63,6 @@ func (p *PeerServer) isNode(nID types.NodeID) bool {
return exist
}
-// handleBlockList is the handler for messages with BlockList as payload.
-func (p *PeerServer) handleBlockList(id types.NodeID, blocks *BlockList) {
- p.peerTotalOrderMu.Lock()
- defer p.peerTotalOrderMu.Unlock()
-
- readyForVerify := p.peerTotalOrder[id].PushBlocks(*blocks)
- if !readyForVerify {
- return
- }
- // Verify the total order result.
- go func(id types.NodeID) {
- p.peerTotalOrderMu.Lock()
- defer p.peerTotalOrderMu.Unlock()
-
- var correct bool
- var length int
- p.peerTotalOrder, correct, length = VerifyTotalOrder(id, p.peerTotalOrder)
- if !correct {
- log.Printf("The result of Total Ordering Algorithm has error.\n")
- }
- p.verifiedLen += uint64(length)
- if p.verifiedLen >= p.cfg.Node.MaxBlock {
- if err := p.trans.Broadcast(
- p.peers, &test.FixedLatencyModel{}, ntfShutdown); err != nil {
- panic(err)
- }
- }
- }(id)
-}
-
// handleMessage is the handler for messages with Message as payload.
func (p *PeerServer) handleMessage(id types.NodeID, m *message) {
switch m.Type {
@@ -107,16 +72,6 @@ func (p *PeerServer) handleMessage(id types.NodeID, m *message) {
if len(p.peers) == 0 {
p.ctxCancel()
}
- case blockTimestamp:
- msgs := []timestampMessage{}
- if err := json.Unmarshal(m.Payload, &msgs); err != nil {
- panic(err)
- }
- for _, msg := range msgs {
- if ok := p.peerTotalOrder[id].PushTimestamp(msg); !ok {
- panic(fmt.Errorf("unable to push timestamp: %v", m))
- }
- }
default:
panic(fmt.Errorf("unknown simulation message type: %v", m))
}
@@ -157,8 +112,6 @@ func (p *PeerServer) mainLoop() {
}
// Handle messages based on their type.
switch val := e.Msg.(type) {
- case *BlockList:
- p.handleBlockList(e.From, val)
case *message:
p.handleMessage(e.From, val)
case *test.BlockEventMessage:
@@ -235,14 +188,8 @@ func (p *PeerServer) Run() {
panic(err)
}
log.Println("Simulation is ready to go with", len(p.peers), "nodes")
- // Initialize total order result cache.
- for id := range p.peers {
- p.peerTotalOrder[id] = NewTotalOrderResult(id)
- }
// Block to handle incoming messages.
p.mainLoop()
- // The simulation is done, clean up.
- LogStatus(p.peerTotalOrder)
if err := p.trans.Close(); err != nil {
log.Printf("Error shutting down peerServer: %v\n", err)
}
diff --git a/simulation/verification.go b/simulation/verification.go
deleted file mode 100644
index a5aad8f..0000000
--- a/simulation/verification.go
+++ /dev/null
@@ -1,260 +0,0 @@
-// Copyright 2018 The dexon-consensus Authors
-// This file is part of the dexon-consensus library.
-//
-// The dexon-consensus 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 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 library. If not, see
-// <http://www.gnu.org/licenses/>.
-
-package simulation
-
-import (
- "container/heap"
- "log"
- "math"
- "time"
-
- "github.com/dexon-foundation/dexon-consensus/common"
- "github.com/dexon-foundation/dexon-consensus/core/test"
- "github.com/dexon-foundation/dexon-consensus/core/types"
-)
-
-type timeStamp struct {
- time time.Time
- length int
-}
-
-type totalOrderStatus struct {
- blockReceive []timeStamp
- confirmLatency []time.Duration
- blockSeen map[common.Hash]time.Time
- internalTimestampLatency []time.Duration
- externalTimestampLatency []time.Duration
-}
-
-// TotalOrderResult is the object maintaining peer's result of
-// Total Ordering Algorithm.
-type TotalOrderResult struct {
- nodeID types.NodeID
- hashList common.Hashes
- curID int
- pendingBlockList PendingBlockList
- status totalOrderStatus
-}
-
-// PeerTotalOrder stores the TotalOrderResult of each node.
-type PeerTotalOrder = map[types.NodeID]*TotalOrderResult
-
-// NewTotalOrderResult returns pointer to a a new TotalOrderResult instance.
-func NewTotalOrderResult(nID types.NodeID) *TotalOrderResult {
- totalOrder := &TotalOrderResult{
- nodeID: nID,
- status: totalOrderStatus{
- blockSeen: make(map[common.Hash]time.Time),
- },
- }
- heap.Init(&totalOrder.pendingBlockList)
- return totalOrder
-}
-
-func (totalOrder *TotalOrderResult) processStatus(blocks BlockList) {
- totalOrder.status.blockReceive = append(totalOrder.status.blockReceive,
- timeStamp{
- time: time.Now(),
- length: len(blocks.BlockHash),
- })
- totalOrder.status.confirmLatency = append(totalOrder.status.confirmLatency,
- blocks.ConfirmLatency...)
-}
-
-// PushBlocks push a BlockList into the TotalOrderResult and return true if
-// there are new blocks ready for verification.
-func (totalOrder *TotalOrderResult) PushBlocks(blocks BlockList) (ready bool) {
- totalOrder.processStatus(blocks)
- if blocks.ID != totalOrder.curID {
- heap.Push(&totalOrder.pendingBlockList, &blocks)
- return false
- }
-
- // Append all of the consecutive blockList in the pendingBlockList.
- for {
- totalOrder.hashList = append(totalOrder.hashList, blocks.BlockHash...)
- totalOrder.curID++
- if len(totalOrder.pendingBlockList) == 0 ||
- totalOrder.pendingBlockList[0].ID != totalOrder.curID {
- break
- }
- blocks = *heap.Pop(&totalOrder.pendingBlockList).(*BlockList)
- }
- return true
-}
-
-// PushTimestamp logs the information in the msg.
-func (totalOrder *TotalOrderResult) PushTimestamp(msg timestampMessage) bool {
- pushLatency := func(latency *[]time.Duration, t1, t2 time.Time) {
- *latency = append(*latency, t2.Sub(t1))
- }
- switch msg.Event {
- case blockSeen:
- totalOrder.status.blockSeen[msg.BlockHash] = msg.Timestamp
- case timestampConfirm:
- pushLatency(&totalOrder.status.internalTimestampLatency,
- totalOrder.status.blockSeen[msg.BlockHash], msg.Timestamp)
- case timestampAck:
- if seenTime, exist := totalOrder.status.blockSeen[msg.BlockHash]; exist {
- pushLatency(&totalOrder.status.externalTimestampLatency,
- seenTime, msg.Timestamp)
- }
- default:
- return false
- }
- return true
-}
-
-// CalculateBlocksPerSecond calculates the result using status.blockReceive
-func (totalOrder *TotalOrderResult) CalculateBlocksPerSecond() float64 {
- ts := totalOrder.status.blockReceive
- if len(ts) < 2 {
- return 0
- }
-
- diffTime := ts[len(ts)-1].time.Sub(ts[0].time).Seconds()
- if diffTime == 0 {
- return 0
- }
- totalBlocks := 0
- for _, blocks := range ts {
- // Blocks received at time zero are confirmed beforehand.
- if blocks.time == ts[0].time {
- continue
- }
- totalBlocks += blocks.length
- }
- return float64(totalBlocks) / diffTime
-}
-
-// CalculateAverageConfirmLatency calculates the result using
-// status.confirmLatency
-func (totalOrder *TotalOrderResult) CalculateAverageConfirmLatency() float64 {
- sum := 0.0
- for _, latency := range totalOrder.status.confirmLatency {
- sum += latency.Seconds()
- }
- return sum / float64(len(totalOrder.status.confirmLatency))
-}
-
-// CalculateAverageTimestampLatency calculates the result using
-// status.timestampLatency
-func (totalOrder *TotalOrderResult) CalculateAverageTimestampLatency() (
- internal float64, external float64) {
- for _, latency := range totalOrder.status.internalTimestampLatency {
- internal += latency.Seconds()
- }
- if internal > 0 {
- internal /= float64(len(totalOrder.status.internalTimestampLatency))
- }
- for _, latency := range totalOrder.status.externalTimestampLatency {
- external += latency.Seconds()
- }
- if external > 0 {
- external /= float64(len(totalOrder.status.externalTimestampLatency))
- }
- return
-}
-
-// VerifyTotalOrder verifies if the result of Total Ordering Algorithm
-// returned by all nodes are the same. However, the length of result
-// of each nodes may not be the same, so only the common part is verified.
-func VerifyTotalOrder(id types.NodeID,
- totalOrder PeerTotalOrder) (
- unverifiedMap PeerTotalOrder, correct bool, length int) {
-
- hasError := false
-
- // Get the common length from all nodes.
- length = math.MaxInt32
- for _, peerTotalOrder := range totalOrder {
- if len(peerTotalOrder.hashList) < length {
- length = len(peerTotalOrder.hashList)
- }
- }
-
- // Verify if the order of the blocks are the same by comparing
- // the hash value.
- for i := 0; i < length; i++ {
- hash := totalOrder[id].hashList[i]
- for vid, peerTotalOrder := range totalOrder {
- if peerTotalOrder.hashList[i] != hash {
- log.Printf("[%d] Unexpected hash %v from %v\n", i,
- peerTotalOrder.hashList[i], vid)
- hasError = true
- }
- }
- if hasError {
- log.Printf("[%d] Hash is %v from %v\n", i, hash, id)
- } else {
- log.Printf("Block %v confirmed\n", hash)
- }
- }
-
- // Remove verified block from list.
- if length > 0 {
- for vid := range totalOrder {
- totalOrder[vid].hashList =
- totalOrder[vid].hashList[length:]
- }
- }
- return totalOrder, !hasError, length
-}
-
-// LogStatus prints all the status to log.
-func LogStatus(peerTotalOrder PeerTotalOrder) {
- for nID, totalOrder := range peerTotalOrder {
- log.Printf("[Node %s]\n", nID)
- log.Printf(" BPS: %.6f\n", totalOrder.CalculateBlocksPerSecond())
- log.Printf(" Confirm Latency: %.2fms\n",
- totalOrder.CalculateAverageConfirmLatency()*1000)
- log.Printf(" Confirm Blocks: %v\n", len(totalOrder.status.confirmLatency))
- intLatency, extLatency := totalOrder.CalculateAverageTimestampLatency()
- log.Printf(" Internal Timestamp Latency: %.2fms\n", intLatency*1000)
- log.Printf(" External Timestamp Latency: %.2fms\n", extLatency*1000)
- }
- logOverallLatency(peerTotalOrder)
-}
-
-// logOverallLatency prints overall status related to latency.
-func logOverallLatency(peerTotalOrder PeerTotalOrder) {
- // Let's use brute-force way since the simulation should be done
- // at this moment.
- var (
- overallConfirmLatency []time.Duration
- overallInternalTimestampLatency []time.Duration
- overallExternalTimestampLatency []time.Duration
- )
- for _, totalOrder := range peerTotalOrder {
- overallConfirmLatency = append(
- overallConfirmLatency, totalOrder.status.confirmLatency...)
- overallInternalTimestampLatency = append(
- overallInternalTimestampLatency,
- totalOrder.status.internalTimestampLatency...)
- overallExternalTimestampLatency = append(
- overallExternalTimestampLatency,
- totalOrder.status.externalTimestampLatency...)
- }
- log.Print("[Overall]\n")
- avg, dev := test.CalcLatencyStatistics(overallConfirmLatency)
- log.Printf(" Confirm Latency: %v, dev: %v\n", avg, dev)
- avg, dev = test.CalcLatencyStatistics(overallInternalTimestampLatency)
- log.Printf(" Interal Timestamp Latency: %v, dev: %v\n", avg, dev)
- avg, dev = test.CalcLatencyStatistics(overallExternalTimestampLatency)
- log.Printf(" External Timestamp Latency: %v, dev: %v\n", avg, dev)
-}