aboutsummaryrefslogtreecommitdiffstats
path: root/simulation
diff options
context:
space:
mode:
Diffstat (limited to 'simulation')
-rw-r--r--simulation/app.go22
-rw-r--r--simulation/config/config.go8
-rw-r--r--simulation/governance.go50
-rw-r--r--simulation/kubernetes/config.toml.in8
-rwxr-xr-xsimulation/kubernetes/entrypoint.sh2
-rw-r--r--simulation/kubernetes/node.yaml.in (renamed from simulation/kubernetes/validator.yaml.in)4
-rwxr-xr-xsimulation/kubernetes/run_scheduler.sh8
-rwxr-xr-xsimulation/kubernetes/run_simulation.sh12
-rw-r--r--simulation/network.go38
-rw-r--r--simulation/node.go (renamed from simulation/validator.go)70
-rw-r--r--simulation/peer-server.go24
-rw-r--r--simulation/simulation.go8
-rw-r--r--simulation/verification.go22
13 files changed, 138 insertions, 138 deletions
diff --git a/simulation/app.go b/simulation/app.go
index 87672fd..9c5619a 100644
--- a/simulation/app.go
+++ b/simulation/app.go
@@ -29,27 +29,27 @@ import (
// simApp is an DEXON app for simulation.
type simApp struct {
- ValidatorID types.ValidatorID
- Outputs []*types.Block
- Early bool
- netModule *network
- DeliverID int
+ NodeID types.NodeID
+ Outputs []*types.Block
+ Early bool
+ netModule *network
+ DeliverID int
// blockSeen stores the time when block is delivered by Total Ordering.
blockSeen map[common.Hash]time.Time
// uncofirmBlocks stores the blocks whose timestamps are not ready.
- unconfirmedBlocks map[types.ValidatorID]common.Hashes
+ unconfirmedBlocks map[types.NodeID]common.Hashes
blockByHash map[common.Hash]*types.Block
blockByHashMutex sync.RWMutex
}
// newSimApp returns point to a new instance of simApp.
-func newSimApp(id types.ValidatorID, netModule *network) *simApp {
+func newSimApp(id types.NodeID, netModule *network) *simApp {
return &simApp{
- ValidatorID: id,
+ NodeID: id,
netModule: netModule,
DeliverID: 0,
blockSeen: make(map[common.Hash]time.Time),
- unconfirmedBlocks: make(map[types.ValidatorID]common.Hashes),
+ unconfirmedBlocks: make(map[types.NodeID]common.Hashes),
blockByHash: make(map[common.Hash]*types.Block),
}
}
@@ -116,13 +116,13 @@ func (a *simApp) TotalOrderingDeliver(blockHashes common.Hashes, early bool) {
}
a.Outputs = blocks
a.Early = early
- fmt.Println("OUTPUT", a.ValidatorID, a.Early, a.Outputs)
+ fmt.Println("OUTPUT", a.NodeID, a.Early, a.Outputs)
confirmLatency := []time.Duration{}
payload := []timestampMessage{}
for _, block := range blocks {
- if block.ProposerID == a.ValidatorID {
+ if block.ProposerID == a.NodeID {
confirmLatency = append(confirmLatency,
now.Sub(block.Timestamp))
}
diff --git a/simulation/config/config.go b/simulation/config/config.go
index ceb0abd..bddb19d 100644
--- a/simulation/config/config.go
+++ b/simulation/config/config.go
@@ -49,8 +49,8 @@ type Legacy struct {
ProposeIntervalSigma float64
}
-// Validator config for the simulation.
-type Validator struct {
+// Node config for the simulation.
+type Node struct {
Consensus Consensus
Legacy Legacy
Num int
@@ -75,7 +75,7 @@ type Scheduler struct {
// Config represents the configuration for simulation.
type Config struct {
Title string
- Validator Validator
+ Node Node
Networking Networking
Scheduler Scheduler
}
@@ -90,7 +90,7 @@ func GenerateDefault(path string) error {
config := Config{
Title: "DEXON Consensus Simulation Config",
- Validator: Validator{
+ Node: Node{
Consensus: Consensus{
PhiRatio: float32(2) / 3,
K: 1,
diff --git a/simulation/governance.go b/simulation/governance.go
index 159d536..0f2bbbf 100644
--- a/simulation/governance.go
+++ b/simulation/governance.go
@@ -29,28 +29,28 @@ import (
// simGovernance is a simulated governance contract implementing the
// core.Governance interface.
type simGovernance struct {
- id types.ValidatorID
- lock sync.RWMutex
- notarySet map[types.ValidatorID]struct{}
- expectedNumValidators int
- k int
- phiRatio float32
- chainNum uint32
- crs string
- dkgComplaint map[uint64][]*types.DKGComplaint
- dkgMasterPublicKey map[uint64][]*types.DKGMasterPublicKey
- lambda time.Duration
- network *network
+ id types.NodeID
+ lock sync.RWMutex
+ notarySet map[types.NodeID]struct{}
+ expectedNumNodes int
+ k int
+ phiRatio float32
+ chainNum uint32
+ crs string
+ dkgComplaint map[uint64][]*types.DKGComplaint
+ dkgMasterPublicKey map[uint64][]*types.DKGMasterPublicKey
+ lambda time.Duration
+ network *network
}
// newSimGovernance returns a new simGovernance instance.
func newSimGovernance(
- id types.ValidatorID,
- numValidators int, consensusConfig config.Consensus) *simGovernance {
+ id types.NodeID,
+ numNodes int, consensusConfig config.Consensus) *simGovernance {
return &simGovernance{
- id: id,
- notarySet: make(map[types.ValidatorID]struct{}),
- expectedNumValidators: numValidators,
+ id: id,
+ notarySet: make(map[types.NodeID]struct{}),
+ expectedNumNodes: numNodes,
k: consensusConfig.K,
phiRatio: consensusConfig.PhiRatio,
chainNum: consensusConfig.ChainNum,
@@ -66,12 +66,12 @@ func (g *simGovernance) setNetwork(network *network) {
}
// GetNotarySet returns the current notary set.
-func (g *simGovernance) GetNotarySet() map[types.ValidatorID]struct{} {
+func (g *simGovernance) GetNotarySet() map[types.NodeID]struct{} {
g.lock.RLock()
defer g.lock.RUnlock()
// Return the cloned notarySet.
- ret := make(map[types.ValidatorID]struct{})
+ ret := make(map[types.NodeID]struct{})
for k := range g.notarySet {
ret[k] = struct{}{}
}
@@ -90,18 +90,18 @@ func (g *simGovernance) GetConfiguration(blockHeight uint64) *types.Config {
}
}
-// addValidator add a new validator into the simulated governance contract.
-func (g *simGovernance) addValidator(vID types.ValidatorID) {
+// addNode add a new node into the simulated governance contract.
+func (g *simGovernance) addNode(nID types.NodeID) {
g.lock.Lock()
defer g.lock.Unlock()
- if _, exists := g.notarySet[vID]; exists {
+ if _, exists := g.notarySet[nID]; exists {
return
}
- if len(g.notarySet) == g.expectedNumValidators {
- panic(fmt.Errorf("attempt to add validator when ready"))
+ if len(g.notarySet) == g.expectedNumNodes {
+ panic(fmt.Errorf("attempt to add node when ready"))
}
- g.notarySet[vID] = struct{}{}
+ g.notarySet[nID] = struct{}{}
}
// AddDKGComplaint adds a DKGComplaint.
diff --git a/simulation/kubernetes/config.toml.in b/simulation/kubernetes/config.toml.in
index 2f12956..546fa8e 100644
--- a/simulation/kubernetes/config.toml.in
+++ b/simulation/kubernetes/config.toml.in
@@ -1,17 +1,17 @@
title = "DEXON Consensus Simulation Config"
-[validator]
-num = {{numValidators}}
+[node]
+num = {{numNode}}
lambda = 250
max_block = 1000
-[validator.consensus]
+[node.consensus]
phi_ratio = 6.66670024394989e-01
k = 1
chain_num = 7
genesis_crs = "In DEXON we trust."
-[validator.legacy]
+[node.legacy]
propose_interval_mean = 5e+02
propose_interval_sigma = 5e+01
diff --git a/simulation/kubernetes/entrypoint.sh b/simulation/kubernetes/entrypoint.sh
index 370b0c2..3340f2d 100755
--- a/simulation/kubernetes/entrypoint.sh
+++ b/simulation/kubernetes/entrypoint.sh
@@ -1,6 +1,6 @@
#!/bin/sh
-if [ "$ROLE" = "validator" ]; then
+if [ "$ROLE" = "node" ]; then
exec ./dexcon-simulation -config config.toml
elif [ "$ROLE" = "peer-server" ]; then
exec ./dexcon-simulation-peer-server -config config.toml
diff --git a/simulation/kubernetes/validator.yaml.in b/simulation/kubernetes/node.yaml.in
index 25b005f..bc457e6 100644
--- a/simulation/kubernetes/validator.yaml.in
+++ b/simulation/kubernetes/node.yaml.in
@@ -7,7 +7,7 @@ metadata:
type: cobinhood
spec:
revisionHistoryLimit: 5
- replicas: {{numValidators}}
+ replicas: {{numNodes}}
template:
metadata:
name: dexcon-simulation
@@ -35,4 +35,4 @@ spec:
memory: 2Gi
env:
- name: ROLE
- value: "validator"
+ value: "node"
diff --git a/simulation/kubernetes/run_scheduler.sh b/simulation/kubernetes/run_scheduler.sh
index 0ebd1a5..8b45eb5 100755
--- a/simulation/kubernetes/run_scheduler.sh
+++ b/simulation/kubernetes/run_scheduler.sh
@@ -20,11 +20,11 @@ start_simulation() {
}
main() {
- local num_validators=$1
+ local num_nodes=$1
local num_cpus=$2
- if [ "$num_validators" == "" ]; then
- num_validators=31
+ if [ "$num_nodes" == "" ]; then
+ num_nodes=31
fi
if [ "$num_cpus" == "" ]; then
@@ -33,7 +33,7 @@ main() {
# Render configuration files.
- sed "s/{{numValidators}}/$num_validators/" config.toml.in > config.toml
+ sed "s/{{numNodes}}/$num_nodes/" config.toml.in > config.toml
sed "s/{{numCPUs}}/$num_cpus/" scheduler.yaml.in > scheduler.yaml
build_binary
diff --git a/simulation/kubernetes/run_simulation.sh b/simulation/kubernetes/run_simulation.sh
index c5298d7..b9e6862 100755
--- a/simulation/kubernetes/run_simulation.sh
+++ b/simulation/kubernetes/run_simulation.sh
@@ -28,19 +28,19 @@ start_simulation() {
sleep 1
done
- kubectl apply -f validator.yaml
+ kubectl apply -f node.yaml
}
main() {
- local num_validators=$1
+ local num_nodes=$1
- if [ "$num_validators" == "" ]; then
- num_validators=7
+ if [ "$num_nodes" == "" ]; then
+ num_nodes=7
fi
# Render configuration files.
- sed "s/{{numValidators}}/$num_validators/" validator.yaml.in > validator.yaml
- sed "s/{{numValidators}}/$num_validators/" config.toml.in > config.toml
+ sed "s/{{numNodes}}/$num_nodes/" node.yaml.in > node.yaml
+ sed "s/{{numNodes}}/$num_nodes/" config.toml.in > config.toml
build_binary
build_docker_image
diff --git a/simulation/network.go b/simulation/network.go
index ce6ea65..d14f07e 100644
--- a/simulation/network.go
+++ b/simulation/network.go
@@ -70,8 +70,8 @@ const (
// infoMessage is a struct used by peerServer's /info.
type infoMessage struct {
- Status infoStatus `json:"status"`
- Peers map[types.ValidatorID]string `json:"peers"`
+ Status infoStatus `json:"status"`
+ Peers map[types.NodeID]string `json:"peers"`
}
// network implements core.Network interface and other methods for simulation
@@ -83,12 +83,12 @@ type network struct {
trans test.TransportClient
fromTransport <-chan *test.TransportEnvelope
toConsensus chan interface{}
- toValidator chan interface{}
+ toNode chan interface{}
}
-// newNetwork setup network stuffs for validators, which provides an
+// newNetwork setup network stuffs for nodes, which provides an
// implementation of core.Network based on test.TransportClient.
-func newNetwork(vID types.ValidatorID, cfg config.Networking) (n *network) {
+func newNetwork(nID types.NodeID, cfg config.Networking) (n *network) {
// Construct latency model.
latency := &test.NormalLatencyModel{
Mean: cfg.Mean,
@@ -97,7 +97,7 @@ func newNetwork(vID types.ValidatorID, cfg config.Networking) (n *network) {
// Construct basic network instance.
n = &network{
cfg: cfg,
- toValidator: make(chan interface{}, 1000),
+ toNode: make(chan interface{}, 1000),
toConsensus: make(chan interface{}, 1000),
}
n.ctx, n.ctxCancel = context.WithCancel(context.Background())
@@ -105,12 +105,12 @@ func newNetwork(vID types.ValidatorID, cfg config.Networking) (n *network) {
switch cfg.Type {
case config.NetworkTypeTCPLocal:
n.trans = test.NewTCPTransportClient(
- vID, latency, &jsonMarshaller{}, true)
+ nID, latency, &jsonMarshaller{}, true)
case config.NetworkTypeTCP:
n.trans = test.NewTCPTransportClient(
- vID, latency, &jsonMarshaller{}, false)
+ nID, latency, &jsonMarshaller{}, false)
case config.NetworkTypeFake:
- n.trans = test.NewFakeTransportClient(vID, latency)
+ n.trans = test.NewFakeTransportClient(nID, latency)
default:
panic(fmt.Errorf("unknown network type: %v", cfg.Type))
}
@@ -138,7 +138,7 @@ func (n *network) BroadcastWitnessAck(witnessAck *types.WitnessAck) {
}
}
-// broadcast message to all other validators in the network.
+// broadcast message to all other nodes in the network.
func (n *network) broadcast(message interface{}) {
if err := n.trans.Broadcast(message); err != nil {
panic(err)
@@ -147,7 +147,7 @@ func (n *network) broadcast(message interface{}) {
// SendDKGPrivateShare implements core.Network interface.
func (n *network) SendDKGPrivateShare(
- recv types.ValidatorID, prvShare *types.DKGPrivateShare) {
+ recv types.NodeID, prvShare *types.DKGPrivateShare) {
if err := n.trans.Send(recv, prvShare); err != nil {
panic(err)
}
@@ -158,10 +158,10 @@ func (n *network) ReceiveChan() <-chan interface{} {
return n.toConsensus
}
-// receiveChanForValidator returns a channel for validators' specific
+// receiveChanForNode returns a channel for nodes' specific
// messages.
-func (n *network) receiveChanForValidator() <-chan interface{} {
- return n.toValidator
+func (n *network) receiveChanForNode() <-chan interface{} {
+ return n.toNode
}
// setup transport layer.
@@ -185,14 +185,14 @@ func (n *network) setup(serverEndpoint interface{}) (err error) {
// run the main loop.
func (n *network) run() {
// The dispatcher declararion:
- // to consensus or validator, that's the question.
+ // to consensus or node, that's the question.
disp := func(e *test.TransportEnvelope) {
switch e.Msg.(type) {
case *types.Block, *types.Vote, *types.WitnessAck,
*types.DKGPrivateShare, *types.DKGPartialSignature:
n.toConsensus <- e.Msg
default:
- n.toValidator <- e.Msg
+ n.toNode <- e.Msg
}
}
MainLoop:
@@ -219,8 +219,8 @@ func (n *network) Close() (err error) {
n.ctxCancel()
close(n.toConsensus)
n.toConsensus = nil
- close(n.toValidator)
- n.toValidator = nil
+ close(n.toNode)
+ n.toNode = nil
if err = n.trans.Close(); err != nil {
return
}
@@ -233,6 +233,6 @@ func (n *network) report(msg interface{}) error {
}
// peers exports 'Peers' method of test.Transport.
-func (n *network) peers() map[types.ValidatorID]struct{} {
+func (n *network) peers() map[types.NodeID]struct{} {
return n.trans.Peers()
}
diff --git a/simulation/validator.go b/simulation/node.go
index 1662dc0..8b282c7 100644
--- a/simulation/validator.go
+++ b/simulation/node.go
@@ -29,41 +29,41 @@ import (
"github.com/dexon-foundation/dexon-consensus-core/simulation/config"
)
-// validator represents a validator in DexCon.
-type validator struct {
+// node represents a node in DexCon.
+type node struct {
app *simApp
gov *simGovernance
db blockdb.BlockDatabase
- config config.Validator
+ config config.Node
netModule *network
- ID types.ValidatorID
+ ID types.NodeID
chainID uint64
prvKey crypto.PrivateKey
sigToPub core.SigToPubFn
consensus *core.Consensus
}
-// newValidator returns a new empty validator.
-func newValidator(
+// newNode returns a new empty node.
+func newNode(
prvKey crypto.PrivateKey,
sigToPub core.SigToPubFn,
- config config.Config) *validator {
+ config config.Config) *node {
- id := types.NewValidatorID(prvKey.PublicKey())
+ id := types.NewNodeID(prvKey.PublicKey())
netModule := newNetwork(id, config.Networking)
db, err := blockdb.NewMemBackedBlockDB(
id.String() + ".blockdb")
if err != nil {
panic(err)
}
- gov := newSimGovernance(id, config.Validator.Num, config.Validator.Consensus)
- return &validator{
+ gov := newSimGovernance(id, config.Node.Num, config.Node.Consensus)
+ return &node{
ID: id,
prvKey: prvKey,
sigToPub: sigToPub,
- config: config.Validator,
+ config: config.Node,
app: newSimApp(id, netModule),
gov: gov,
db: db,
@@ -71,40 +71,40 @@ func newValidator(
}
}
-// GetID returns the ID of validator.
-func (v *validator) GetID() types.ValidatorID {
- return v.ID
+// GetID returns the ID of node.
+func (n *node) GetID() types.NodeID {
+ return n.ID
}
-// run starts the validator.
-func (v *validator) run(serverEndpoint interface{}, legacy bool) {
+// run starts the node.
+func (n *node) run(serverEndpoint interface{}, legacy bool) {
// Run network.
- if err := v.netModule.setup(serverEndpoint); err != nil {
+ if err := n.netModule.setup(serverEndpoint); err != nil {
panic(err)
}
- msgChannel := v.netModule.receiveChanForValidator()
- peers := v.netModule.peers()
- go v.netModule.run()
- v.gov.setNetwork(v.netModule)
+ msgChannel := n.netModule.receiveChanForNode()
+ peers := n.netModule.peers()
+ go n.netModule.run()
+ n.gov.setNetwork(n.netModule)
// Run consensus.
hashes := make(common.Hashes, 0, len(peers))
- for vID := range peers {
- v.gov.addValidator(vID)
- hashes = append(hashes, vID.Hash)
+ for nID := range peers {
+ n.gov.addNode(nID)
+ hashes = append(hashes, nID.Hash)
}
sort.Sort(hashes)
for i, hash := range hashes {
- if hash == v.ID.Hash {
- v.chainID = uint64(i)
+ if hash == n.ID.Hash {
+ n.chainID = uint64(i)
break
}
}
- v.consensus = core.NewConsensus(
- v.app, v.gov, v.db, v.netModule, v.prvKey, v.sigToPub)
+ n.consensus = core.NewConsensus(
+ n.app, n.gov, n.db, n.netModule, n.prvKey, n.sigToPub)
if legacy {
- go v.consensus.RunLegacy()
+ go n.consensus.RunLegacy()
} else {
- go v.consensus.Run()
+ go n.consensus.Run()
}
// Blocks forever.
@@ -117,19 +117,19 @@ MainLoop:
break MainLoop
}
case *types.DKGComplaint:
- v.gov.AddDKGComplaint(val)
+ n.gov.AddDKGComplaint(val)
case *types.DKGMasterPublicKey:
- v.gov.AddDKGMasterPublicKey(val)
+ n.gov.AddDKGMasterPublicKey(val)
default:
panic(fmt.Errorf("unexpected message from server: %v", val))
}
}
// Cleanup.
- v.consensus.Stop()
- if err := v.db.Close(); err != nil {
+ n.consensus.Stop()
+ if err := n.db.Close(); err != nil {
fmt.Println(err)
}
- v.netModule.report(&message{
+ n.netModule.report(&message{
Type: shutdownAck,
})
// TODO(mission): once we have a way to know if consensus is stopped, stop
diff --git a/simulation/peer-server.go b/simulation/peer-server.go
index 5b43be4..30cf896 100644
--- a/simulation/peer-server.go
+++ b/simulation/peer-server.go
@@ -32,7 +32,7 @@ import (
// PeerServer is the main object to collect results and monitor simulation.
type PeerServer struct {
- peers map[types.ValidatorID]struct{}
+ peers map[types.NodeID]struct{}
msgChannel chan *test.TransportEnvelope
trans test.TransportServer
peerTotalOrder PeerTotalOrder
@@ -47,22 +47,22 @@ type PeerServer struct {
func NewPeerServer() *PeerServer {
ctx, cancel := context.WithCancel(context.Background())
return &PeerServer{
- peers: make(map[types.ValidatorID]struct{}),
+ peers: make(map[types.NodeID]struct{}),
peerTotalOrder: make(PeerTotalOrder),
ctx: ctx,
ctxCancel: cancel,
}
}
-// isValidator checks if vID is in p.peers. If peer server restarts but
-// validators are not, it will cause panic if validators send message.
-func (p *PeerServer) isValidator(vID types.ValidatorID) bool {
- _, exist := p.peers[vID]
+// isNode checks if nID is in p.peers. If peer server restarts but
+// nodes are not, it will cause panic if nodes send message.
+func (p *PeerServer) isNode(nID types.NodeID) bool {
+ _, exist := p.peers[nID]
return exist
}
// handleBlockList is the handler for messages with BlockList as payload.
-func (p *PeerServer) handleBlockList(id types.ValidatorID, blocks *BlockList) {
+func (p *PeerServer) handleBlockList(id types.NodeID, blocks *BlockList) {
p.peerTotalOrderMu.Lock()
defer p.peerTotalOrderMu.Unlock()
@@ -71,7 +71,7 @@ func (p *PeerServer) handleBlockList(id types.ValidatorID, blocks *BlockList) {
return
}
// Verify the total order result.
- go func(id types.ValidatorID) {
+ go func(id types.NodeID) {
p.peerTotalOrderMu.Lock()
defer p.peerTotalOrderMu.Unlock()
@@ -82,7 +82,7 @@ func (p *PeerServer) handleBlockList(id types.ValidatorID, blocks *BlockList) {
log.Printf("The result of Total Ordering Algorithm has error.\n")
}
p.verifiedLen += uint64(length)
- if p.verifiedLen >= p.cfg.Validator.MaxBlock {
+ if p.verifiedLen >= p.cfg.Node.MaxBlock {
if err := p.trans.Broadcast(statusShutdown); err != nil {
panic(err)
}
@@ -91,7 +91,7 @@ func (p *PeerServer) handleBlockList(id types.ValidatorID, blocks *BlockList) {
}
// handleMessage is the handler for messages with Message as payload.
-func (p *PeerServer) handleMessage(id types.ValidatorID, m *message) {
+func (p *PeerServer) handleMessage(id types.NodeID, m *message) {
switch m.Type {
case shutdownAck:
delete(p.peers, id)
@@ -125,7 +125,7 @@ func (p *PeerServer) mainLoop() {
case <-p.ctx.Done():
return
case e := <-p.msgChannel:
- if !p.isValidator(e.From) {
+ if !p.isNode(e.From) {
break
}
// Handle messages based on their type.
@@ -164,7 +164,7 @@ func (p *PeerServer) Setup(
// Run the simulation.
func (p *PeerServer) Run() {
- if err := p.trans.WaitForPeers(p.cfg.Validator.Num); err != nil {
+ if err := p.trans.WaitForPeers(p.cfg.Node.Num); err != nil {
panic(err)
}
// Cache peers' info.
diff --git a/simulation/simulation.go b/simulation/simulation.go
index 74a758d..61e09c7 100644
--- a/simulation/simulation.go
+++ b/simulation/simulation.go
@@ -33,13 +33,13 @@ func Run(cfg *config.Config, legacy bool) {
err error
)
- // init is a function to init a validator.
+ // init is a function to init a node.
init := func(serverEndpoint interface{}) {
prv, err := eth.NewPrivateKey()
if err != nil {
panic(err)
}
- v := newValidator(prv, eth.SigToPub, *cfg)
+ v := newNode(prv, eth.SigToPub, *cfg)
wg.Add(1)
go func() {
defer wg.Done()
@@ -64,8 +64,8 @@ func Run(cfg *config.Config, legacy bool) {
defer wg.Done()
server.Run()
}()
- // Initialize all validators.
- for i := 0; i < cfg.Validator.Num; i++ {
+ // Initialize all nodes.
+ for i := 0; i < cfg.Node.Num; i++ {
init(serverEndpoint)
}
}
diff --git a/simulation/verification.go b/simulation/verification.go
index 0ae7450..fd41cd9 100644
--- a/simulation/verification.go
+++ b/simulation/verification.go
@@ -44,20 +44,20 @@ type totalOrderStatus struct {
// TotalOrderResult is the object maintaining peer's result of
// Total Ordering Algorithm.
type TotalOrderResult struct {
- validatorID types.ValidatorID
+ nodeID types.NodeID
hashList common.Hashes
curID int
pendingBlockList PendingBlockList
status totalOrderStatus
}
-// PeerTotalOrder stores the TotalOrderResult of each validator.
-type PeerTotalOrder = map[types.ValidatorID]*TotalOrderResult
+// PeerTotalOrder stores the TotalOrderResult of each node.
+type PeerTotalOrder = map[types.NodeID]*TotalOrderResult
// NewTotalOrderResult returns pointer to a a new TotalOrderResult instance.
-func NewTotalOrderResult(vID types.ValidatorID) *TotalOrderResult {
+func NewTotalOrderResult(nID types.NodeID) *TotalOrderResult {
totalOrder := &TotalOrderResult{
- validatorID: vID,
+ nodeID: nID,
status: totalOrderStatus{
blockSeen: make(map[common.Hash]time.Time),
},
@@ -172,15 +172,15 @@ func (totalOrder *TotalOrderResult) CalculateAverageTimestampLatency() (
}
// VerifyTotalOrder verifies if the result of Total Ordering Algorithm
-// returned by all validators are the same. However, the length of result
-// of each validators may not be the same, so only the common part is verified.
-func VerifyTotalOrder(id types.ValidatorID,
+// 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 validators.
+ // Get the common length from all nodes.
length = math.MaxInt32
for _, peerTotalOrder := range totalOrder {
if len(peerTotalOrder.hashList) < length {
@@ -218,8 +218,8 @@ func VerifyTotalOrder(id types.ValidatorID,
// LogStatus prints all the status to log.
func LogStatus(peerTotalOrder PeerTotalOrder) {
- for vID, totalOrder := range peerTotalOrder {
- log.Printf("[Validator %s]\n", vID)
+ for nID, totalOrder := range peerTotalOrder {
+ log.Printf("[Node %s]\n", nID)
log.Printf(" BPS: %.6f\n",
totalOrder.CalculateBlocksPerSecond())
log.Printf(" Confirm Latency: %.2fms\n",