mirror of
https://github.com/fluencelabs/tendermint
synced 2025-04-24 22:32:15 +00:00
Implement BFT time (#2203)
* Implement BFT time * set LastValidators when creating state in state helper for heights >= 2
This commit is contained in:
parent
ffe91ae9e3
commit
7b88172f41
@ -20,6 +20,7 @@ BREAKING CHANGES:
|
||||
- Remove PubKey from `Validator` and introduce `ValidatorUpdate`
|
||||
- InitChain and EndBlock use ValidatorUpdate
|
||||
- Update field names and types in BeginBlock
|
||||
- [state] Implement BFT time
|
||||
- [p2p] update secret connection to use a little endian encoded nonce
|
||||
|
||||
FEATURES:
|
||||
|
@ -6,7 +6,6 @@ import (
|
||||
"runtime/debug"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
@ -14,6 +13,7 @@ import (
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
|
||||
"github.com/tendermint/tendermint/types"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
func TestLoadBlockStoreStateJSON(t *testing.T) {
|
||||
@ -70,7 +70,7 @@ var (
|
||||
part1 = partSet.GetPart(0)
|
||||
part2 = partSet.GetPart(1)
|
||||
seenCommit1 = &types.Commit{Precommits: []*types.Vote{{Height: 10,
|
||||
Timestamp: time.Now().UTC()}}}
|
||||
Timestamp: tmtime.Now()}}}
|
||||
)
|
||||
|
||||
// TODO: This test should be simplified ...
|
||||
@ -91,7 +91,7 @@ func TestBlockStoreSaveLoadBlock(t *testing.T) {
|
||||
block := makeBlock(bs.Height()+1, state)
|
||||
validPartSet := block.MakePartSet(2)
|
||||
seenCommit := &types.Commit{Precommits: []*types.Vote{{Height: 10,
|
||||
Timestamp: time.Now().UTC()}}}
|
||||
Timestamp: tmtime.Now()}}}
|
||||
bs.SaveBlock(block, partSet, seenCommit)
|
||||
require.Equal(t, bs.Height(), block.Header.Height, "expecting the new height to be changed")
|
||||
|
||||
@ -103,7 +103,7 @@ func TestBlockStoreSaveLoadBlock(t *testing.T) {
|
||||
Height: 1,
|
||||
NumTxs: 100,
|
||||
ChainID: "block_test",
|
||||
Time: time.Now(),
|
||||
Time: tmtime.Now(),
|
||||
}
|
||||
header2 := header1
|
||||
header2.Height = 4
|
||||
@ -111,7 +111,7 @@ func TestBlockStoreSaveLoadBlock(t *testing.T) {
|
||||
// End of setup, test data
|
||||
|
||||
commitAtH10 := &types.Commit{Precommits: []*types.Vote{{Height: 10,
|
||||
Timestamp: time.Now().UTC()}}}
|
||||
Timestamp: tmtime.Now()}}}
|
||||
tuples := []struct {
|
||||
block *types.Block
|
||||
parts *types.PartSet
|
||||
@ -335,7 +335,7 @@ func TestBlockFetchAtHeight(t *testing.T) {
|
||||
|
||||
partSet := block.MakePartSet(2)
|
||||
seenCommit := &types.Commit{Precommits: []*types.Vote{{Height: 10,
|
||||
Timestamp: time.Now().UTC()}}}
|
||||
Timestamp: tmtime.Now()}}}
|
||||
|
||||
bs.SaveBlock(block, partSet, seenCommit)
|
||||
require.Equal(t, bs.Height(), block.Header.Height, "expecting the new height to be changed")
|
||||
|
@ -2,7 +2,6 @@ package commands
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/spf13/cobra"
|
||||
|
||||
@ -11,6 +10,7 @@ import (
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
"github.com/tendermint/tendermint/privval"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
// InitFilesCmd initialises a fresh Tendermint Core instance.
|
||||
@ -54,7 +54,7 @@ func initFilesWithConfig(config *cfg.Config) error {
|
||||
} else {
|
||||
genDoc := types.GenesisDoc{
|
||||
ChainID: fmt.Sprintf("test-chain-%v", cmn.RandStr(6)),
|
||||
GenesisTime: time.Now(),
|
||||
GenesisTime: tmtime.Now(),
|
||||
ConsensusParams: types.DefaultConsensusParams(),
|
||||
}
|
||||
genDoc.Validators = []types.GenesisValidator{{
|
||||
|
@ -6,7 +6,6 @@ import (
|
||||
"os"
|
||||
"path/filepath"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/spf13/cobra"
|
||||
|
||||
@ -15,6 +14,7 @@ import (
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
"github.com/tendermint/tendermint/privval"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
var (
|
||||
@ -112,7 +112,7 @@ func testnetFiles(cmd *cobra.Command, args []string) error {
|
||||
|
||||
// Generate genesis doc from generated validators
|
||||
genDoc := &types.GenesisDoc{
|
||||
GenesisTime: time.Now(),
|
||||
GenesisTime: tmtime.Now(),
|
||||
ChainID: "chain-" + cmn.RandStr(6),
|
||||
Validators: genVals,
|
||||
}
|
||||
|
@ -469,6 +469,9 @@ type ConsensusConfig struct {
|
||||
// Reactor sleep duration parameters are in milliseconds
|
||||
PeerGossipSleepDuration int `mapstructure:"peer_gossip_sleep_duration"`
|
||||
PeerQueryMaj23SleepDuration int `mapstructure:"peer_query_maj23_sleep_duration"`
|
||||
|
||||
// Block time parameters in milliseconds. Corresponds to the minimum time increment between consecutive blocks.
|
||||
BlockTimeIota int `mapstructure:"blocktime_iota"`
|
||||
}
|
||||
|
||||
// DefaultConsensusConfig returns a default configuration for the consensus service
|
||||
@ -487,6 +490,7 @@ func DefaultConsensusConfig() *ConsensusConfig {
|
||||
CreateEmptyBlocksInterval: 0,
|
||||
PeerGossipSleepDuration: 100,
|
||||
PeerQueryMaj23SleepDuration: 2000,
|
||||
BlockTimeIota: 1000,
|
||||
}
|
||||
}
|
||||
|
||||
@ -503,9 +507,17 @@ func TestConsensusConfig() *ConsensusConfig {
|
||||
cfg.SkipTimeoutCommit = true
|
||||
cfg.PeerGossipSleepDuration = 5
|
||||
cfg.PeerQueryMaj23SleepDuration = 250
|
||||
cfg.BlockTimeIota = 10
|
||||
return cfg
|
||||
}
|
||||
|
||||
// MinValidVoteTime returns the minimum acceptable block time.
|
||||
// See the [BFT time spec](https://godoc.org/github.com/tendermint/tendermint/docs/spec/consensus/bft-time.md).
|
||||
func (cfg *ConsensusConfig) MinValidVoteTime(lastBlockTime time.Time) time.Time {
|
||||
return lastBlockTime.
|
||||
Add(time.Duration(cfg.BlockTimeIota) * time.Millisecond)
|
||||
}
|
||||
|
||||
// WaitForTxs returns true if the consensus should wait for transactions before entering the propose step
|
||||
func (cfg *ConsensusConfig) WaitForTxs() bool {
|
||||
return !cfg.CreateEmptyBlocks || cfg.CreateEmptyBlocksInterval > 0
|
||||
|
@ -25,6 +25,7 @@ import (
|
||||
"github.com/tendermint/tendermint/privval"
|
||||
sm "github.com/tendermint/tendermint/state"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
|
||||
"github.com/tendermint/tendermint/abci/example/counter"
|
||||
"github.com/tendermint/tendermint/abci/example/kvstore"
|
||||
@ -75,7 +76,7 @@ func (vs *validatorStub) signVote(voteType byte, hash []byte, header types.PartS
|
||||
ValidatorAddress: vs.PrivValidator.GetAddress(),
|
||||
Height: vs.Height,
|
||||
Round: vs.Round,
|
||||
Timestamp: time.Now().UTC(),
|
||||
Timestamp: tmtime.Now(),
|
||||
Type: voteType,
|
||||
BlockID: types.BlockID{hash, header},
|
||||
}
|
||||
@ -423,7 +424,7 @@ func randGenesisDoc(numValidators int, randPower bool, minPower int64) (*types.G
|
||||
sort.Sort(types.PrivValidatorsByAddress(privValidators))
|
||||
|
||||
return &types.GenesisDoc{
|
||||
GenesisTime: time.Now(),
|
||||
GenesisTime: tmtime.Now(),
|
||||
ChainID: config.ChainID(),
|
||||
Validators: validators,
|
||||
}, privValidators
|
||||
|
@ -17,6 +17,7 @@ import (
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
sm "github.com/tendermint/tendermint/state"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
const (
|
||||
@ -1165,7 +1166,7 @@ func (ps *PeerState) ApplyNewRoundStepMessage(msg *NewRoundStepMessage) {
|
||||
psCatchupCommitRound := ps.PRS.CatchupCommitRound
|
||||
psCatchupCommit := ps.PRS.CatchupCommit
|
||||
|
||||
startTime := time.Now().Add(-1 * time.Duration(msg.SecondsSinceStartTime) * time.Second)
|
||||
startTime := tmtime.Now().Add(-1 * time.Duration(msg.SecondsSinceStartTime) * time.Second)
|
||||
ps.PRS.Height = msg.Height
|
||||
ps.PRS.Round = msg.Round
|
||||
ps.PRS.Step = msg.Step
|
||||
|
@ -20,6 +20,7 @@ import (
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
mempl "github.com/tendermint/tendermint/mempool"
|
||||
sm "github.com/tendermint/tendermint/state"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
|
||||
cfg "github.com/tendermint/tendermint/config"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
@ -321,7 +322,7 @@ func TestReactorRecordsVotes(t *testing.T) {
|
||||
ValidatorAddress: val.Address,
|
||||
Height: 2,
|
||||
Round: 0,
|
||||
Timestamp: time.Now().UTC(),
|
||||
Timestamp: tmtime.Now(),
|
||||
Type: types.VoteTypePrevote,
|
||||
BlockID: types.BlockID{},
|
||||
}
|
||||
|
@ -12,6 +12,7 @@ import (
|
||||
fail "github.com/ebuchman/fail-test"
|
||||
cmn "github.com/tendermint/tendermint/libs/common"
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
|
||||
cfg "github.com/tendermint/tendermint/config"
|
||||
cstypes "github.com/tendermint/tendermint/consensus/types"
|
||||
@ -422,8 +423,8 @@ func (cs *ConsensusState) updateRoundStep(round int, step cstypes.RoundStepType)
|
||||
|
||||
// enterNewRound(height, 0) at cs.StartTime.
|
||||
func (cs *ConsensusState) scheduleRound0(rs *cstypes.RoundState) {
|
||||
//cs.Logger.Info("scheduleRound0", "now", time.Now(), "startTime", cs.StartTime)
|
||||
sleepDuration := rs.StartTime.Sub(time.Now()) // nolint: gotype, gosimple
|
||||
//cs.Logger.Info("scheduleRound0", "now", tmtime.Now(), "startTime", cs.StartTime)
|
||||
sleepDuration := rs.StartTime.Sub(tmtime.Now()) // nolint: gotype, gosimple
|
||||
cs.scheduleTimeout(sleepDuration, rs.Height, 0, cstypes.RoundStepNewHeight)
|
||||
}
|
||||
|
||||
@ -516,7 +517,7 @@ func (cs *ConsensusState) updateToState(state sm.State) {
|
||||
// to be gathered for the first block.
|
||||
// And alternative solution that relies on clocks:
|
||||
// cs.StartTime = state.LastBlockTime.Add(timeoutCommit)
|
||||
cs.StartTime = cs.config.Commit(time.Now())
|
||||
cs.StartTime = cs.config.Commit(tmtime.Now())
|
||||
} else {
|
||||
cs.StartTime = cs.config.Commit(cs.CommitTime)
|
||||
}
|
||||
@ -729,7 +730,7 @@ func (cs *ConsensusState) enterNewRound(height int64, round int) {
|
||||
return
|
||||
}
|
||||
|
||||
if now := time.Now(); cs.StartTime.After(now) {
|
||||
if now := tmtime.Now(); cs.StartTime.After(now) {
|
||||
logger.Info("Need to set a buffer and log message here for sanity.", "startTime", cs.StartTime, "now", now)
|
||||
}
|
||||
|
||||
@ -1195,7 +1196,7 @@ func (cs *ConsensusState) enterCommit(height int64, commitRound int) {
|
||||
// keep cs.Round the same, commitRound points to the right Precommits set.
|
||||
cs.updateRoundStep(cs.Round, cstypes.RoundStepCommit)
|
||||
cs.CommitRound = commitRound
|
||||
cs.CommitTime = time.Now()
|
||||
cs.CommitTime = tmtime.Now()
|
||||
cs.newStep()
|
||||
|
||||
// Maybe finalize immediately.
|
||||
@ -1660,12 +1661,13 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerID p2p.ID) (added bool,
|
||||
func (cs *ConsensusState) signVote(type_ byte, hash []byte, header types.PartSetHeader) (*types.Vote, error) {
|
||||
addr := cs.privValidator.GetAddress()
|
||||
valIndex, _ := cs.Validators.GetByAddress(addr)
|
||||
|
||||
vote := &types.Vote{
|
||||
ValidatorAddress: addr,
|
||||
ValidatorIndex: valIndex,
|
||||
Height: cs.Height,
|
||||
Round: cs.Round,
|
||||
Timestamp: time.Now().UTC(),
|
||||
Timestamp: cs.voteTime(),
|
||||
Type: type_,
|
||||
BlockID: types.BlockID{hash, header},
|
||||
}
|
||||
@ -1673,6 +1675,23 @@ func (cs *ConsensusState) signVote(type_ byte, hash []byte, header types.PartSet
|
||||
return vote, err
|
||||
}
|
||||
|
||||
func (cs *ConsensusState) voteTime() time.Time {
|
||||
now := tmtime.Now()
|
||||
minVoteTime := now
|
||||
// TODO: We should remove next line in case we don't vote for v in case cs.ProposalBlock == nil,
|
||||
// even if cs.LockedBlock != nil. See https://github.com/tendermint/spec.
|
||||
if cs.LockedBlock != nil {
|
||||
minVoteTime = cs.config.MinValidVoteTime(cs.LockedBlock.Time)
|
||||
} else if cs.ProposalBlock != nil {
|
||||
minVoteTime = cs.config.MinValidVoteTime(cs.ProposalBlock.Time)
|
||||
}
|
||||
|
||||
if now.After(minVoteTime) {
|
||||
return now
|
||||
}
|
||||
return minVoteTime
|
||||
}
|
||||
|
||||
// sign the vote and publish on internalMsgQueue
|
||||
func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header types.PartSetHeader) *types.Vote {
|
||||
// if we don't have a key or we're not in the validator set, do nothing
|
||||
|
@ -3,10 +3,10 @@ package types
|
||||
import (
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
cfg "github.com/tendermint/tendermint/config"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
var config *cfg.Config // NOTE: must be reset for each _test.go file
|
||||
@ -55,7 +55,7 @@ func makeVoteHR(t *testing.T, height int64, round int, privVals []types.PrivVali
|
||||
ValidatorIndex: valIndex,
|
||||
Height: height,
|
||||
Round: round,
|
||||
Timestamp: time.Now().UTC(),
|
||||
Timestamp: tmtime.Now(),
|
||||
Type: types.VoteTypePrecommit,
|
||||
BlockID: types.BlockID{[]byte("fakehash"), types.PartSetHeader{}},
|
||||
}
|
||||
|
@ -2,12 +2,12 @@ package types
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
amino "github.com/tendermint/go-amino"
|
||||
"github.com/tendermint/go-amino"
|
||||
"github.com/tendermint/tendermint/crypto/ed25519"
|
||||
cmn "github.com/tendermint/tendermint/libs/common"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
func BenchmarkRoundStateDeepCopy(b *testing.B) {
|
||||
@ -27,7 +27,7 @@ func BenchmarkRoundStateDeepCopy(b *testing.B) {
|
||||
for i := 0; i < nval; i++ {
|
||||
precommits[i] = &types.Vote{
|
||||
ValidatorAddress: types.Address(cmn.RandBytes(20)),
|
||||
Timestamp: time.Now(),
|
||||
Timestamp: tmtime.Now(),
|
||||
BlockID: blockID,
|
||||
Signature: sig,
|
||||
}
|
||||
@ -40,7 +40,7 @@ func BenchmarkRoundStateDeepCopy(b *testing.B) {
|
||||
block := &types.Block{
|
||||
Header: types.Header{
|
||||
ChainID: cmn.RandStr(12),
|
||||
Time: time.Now(),
|
||||
Time: tmtime.Now(),
|
||||
LastBlockID: blockID,
|
||||
LastCommitHash: cmn.RandBytes(20),
|
||||
DataHash: cmn.RandBytes(20),
|
||||
@ -62,7 +62,7 @@ func BenchmarkRoundStateDeepCopy(b *testing.B) {
|
||||
parts := block.MakePartSet(4096)
|
||||
// Random Proposal
|
||||
proposal := &types.Proposal{
|
||||
Timestamp: time.Now(),
|
||||
Timestamp: tmtime.Now(),
|
||||
BlockPartsHeader: types.PartSetHeader{
|
||||
Hash: cmn.RandBytes(20),
|
||||
},
|
||||
@ -73,8 +73,8 @@ func BenchmarkRoundStateDeepCopy(b *testing.B) {
|
||||
// TODO: hvs :=
|
||||
|
||||
rs := &RoundState{
|
||||
StartTime: time.Now(),
|
||||
CommitTime: time.Now(),
|
||||
StartTime: tmtime.Now(),
|
||||
CommitTime: tmtime.Now(),
|
||||
Validators: vset,
|
||||
Proposal: proposal,
|
||||
ProposalBlock: block,
|
||||
|
@ -14,6 +14,7 @@ import (
|
||||
auto "github.com/tendermint/tendermint/libs/autofile"
|
||||
cmn "github.com/tendermint/tendermint/libs/common"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
const (
|
||||
@ -119,7 +120,7 @@ func (wal *baseWAL) Write(msg WALMessage) {
|
||||
}
|
||||
|
||||
// Write the wal message
|
||||
if err := wal.enc.Encode(&TimedWALMessage{time.Now(), msg}); err != nil {
|
||||
if err := wal.enc.Encode(&TimedWALMessage{tmtime.Now(), msg}); err != nil {
|
||||
panic(fmt.Sprintf("Error writing msg to consensus wal: %v \n\nMessage: %v", err, msg))
|
||||
}
|
||||
}
|
||||
|
@ -10,13 +10,14 @@ import (
|
||||
|
||||
"github.com/tendermint/tendermint/consensus/types"
|
||||
tmtypes "github.com/tendermint/tendermint/types"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func TestWALEncoderDecoder(t *testing.T) {
|
||||
now := time.Now()
|
||||
now := tmtime.Now()
|
||||
msgs := []TimedWALMessage{
|
||||
TimedWALMessage{Time: now, Msg: EndHeightMessage{0}},
|
||||
TimedWALMessage{Time: now, Msg: timeoutInfo{Duration: time.Second, Height: 1, Round: 1, Step: types.RoundStepPropose}},
|
||||
@ -93,7 +94,7 @@ func benchmarkWalDecode(b *testing.B, n int) {
|
||||
enc := NewWALEncoder(buf)
|
||||
|
||||
data := nBytes(n)
|
||||
enc.Encode(&TimedWALMessage{Msg: data, Time: time.Now().Round(time.Second)})
|
||||
enc.Encode(&TimedWALMessage{Msg: data, Time: time.Now().Round(time.Second).UTC()})
|
||||
|
||||
encoded := buf.Bytes()
|
||||
|
||||
|
@ -1,53 +1,54 @@
|
||||
# BFT time in Tendermint
|
||||
# BFT time in Tendermint
|
||||
|
||||
Tendermint provides a deterministic, Byzantine fault-tolerant, source of time.
|
||||
Time in Tendermint is defined with the Time field of the block header.
|
||||
Tendermint provides a deterministic, Byzantine fault-tolerant, source of time.
|
||||
Time in Tendermint is defined with the Time field of the block header.
|
||||
|
||||
It satisfies the following properties:
|
||||
|
||||
- Time Monotonicity: Time is monotonically increasing, i.e., given
|
||||
a header H1 for height h1 and a header H2 for height `h2 = h1 + 1`, `H1.Time < H2.Time`.
|
||||
- Time Validity: Given a set of Commit votes that forms the `block.LastCommit` field, a range of
|
||||
valid values for the Time field of the block header is defined only by
|
||||
Precommit messages (from the LastCommit field) sent by correct processes, i.e.,
|
||||
a faulty process cannot arbitrarily increase the Time value.
|
||||
- Time Monotonicity: Time is monotonically increasing, i.e., given
|
||||
a header H1 for height h1 and a header H2 for height `h2 = h1 + 1`, `H1.Time < H2.Time`.
|
||||
- Time Validity: Given a set of Commit votes that forms the `block.LastCommit` field, a range of
|
||||
valid values for the Time field of the block header is defined only by
|
||||
Precommit messages (from the LastCommit field) sent by correct processes, i.e.,
|
||||
a faulty process cannot arbitrarily increase the Time value.
|
||||
|
||||
In the context of Tendermint, time is of type int64 and denotes UNIX time in milliseconds, i.e.,
|
||||
corresponds to the number of milliseconds since January 1, 1970. Before defining rules that need to be enforced by the
|
||||
In the context of Tendermint, time is of type int64 and denotes UNIX time in milliseconds, i.e.,
|
||||
corresponds to the number of milliseconds since January 1, 1970. Before defining rules that need to be enforced by the
|
||||
Tendermint consensus protocol, so the properties above holds, we introduce the following definition:
|
||||
|
||||
- median of a set of `Vote` messages is equal to the median of `Vote.Time` fields of the corresponding `Vote` messages,
|
||||
where the value of `Vote.Time` is counted number of times proportional to the process voting power. As in Tendermint
|
||||
the voting power is not uniform (one process one vote), a vote message is actually an aggregator of the same votes whose
|
||||
number is equal to the voting power of the process that has casted the corresponding votes message.
|
||||
- median of a Commit is equal to the median of `Vote.Time` fields of the `Vote` messages,
|
||||
where the value of `Vote.Time` is counted number of times proportional to the process voting power. As in Tendermint
|
||||
the voting power is not uniform (one process one vote), a vote message is actually an aggregator of the same votes whose
|
||||
number is equal to the voting power of the process that has casted the corresponding votes message.
|
||||
|
||||
Let's consider the following example:
|
||||
- we have four processes p1, p2, p3 and p4, with the following voting power distribution (p1, 23), (p2, 27), (p3, 10)
|
||||
and (p4, 10). The total voting power is 70 (`N = 3f+1`, where `N` is the total voting power, and `f` is the maximum voting
|
||||
power of the faulty processes), so we assume that the faulty processes have at most 23 of voting power.
|
||||
Furthermore, we have the following vote messages in some LastCommit field (we ignore all fields except Time field):
|
||||
- (p1, 100), (p2, 98), (p3, 1000), (p4, 500). We assume that p3 and p4 are faulty processes. Let's assume that the
|
||||
`block.LastCommit` message contains votes of processes p2, p3 and p4. Median is then chosen the following way:
|
||||
the value 98 is counted 27 times, the value 1000 is counted 10 times and the value 500 is counted also 10 times.
|
||||
So the median value will be the value 98. No matter what set of messages with at least `2f+1` voting power we
|
||||
choose, the median value will always be between the values sent by correct processes.
|
||||
|
||||
- we have four processes p1, p2, p3 and p4, with the following voting power distribution (p1, 23), (p2, 27), (p3, 10)
|
||||
and (p4, 10). The total voting power is 70 (`N = 3f+1`, where `N` is the total voting power, and `f` is the maximum voting
|
||||
power of the faulty processes), so we assume that the faulty processes have at most 23 of voting power.
|
||||
Furthermore, we have the following vote messages in some LastCommit field (we ignore all fields except Time field): - (p1, 100), (p2, 98), (p3, 1000), (p4, 500). We assume that p3 and p4 are faulty processes. Let's assume that the
|
||||
`block.LastCommit` message contains votes of processes p2, p3 and p4. Median is then chosen the following way:
|
||||
the value 98 is counted 27 times, the value 1000 is counted 10 times and the value 500 is counted also 10 times.
|
||||
So the median value will be the value 98. No matter what set of messages with at least `2f+1` voting power we
|
||||
choose, the median value will always be between the values sent by correct processes.
|
||||
We ensure Time Monotonicity and Time Validity properties by the following rules:
|
||||
|
||||
- let rs denotes `RoundState` (consensus internal state) of some process. Then
|
||||
`rs.ProposalBlock.Header.Time == median(rs.LastCommit) &&
|
||||
rs.Proposal.Timestamp == rs.ProposalBlock.Header.Time`.
|
||||
|
||||
We ensure Time Monotonicity and Time Validity properties by the following rules:
|
||||
- Furthermore, when creating the `vote` message, the following rules for determining `vote.Time` field should hold:
|
||||
|
||||
- let rs denotes `RoundState` (consensus internal state) of some process. Then
|
||||
`rs.ProposalBlock.Header.Time == median(rs.LastCommit) && rs.Proposal.Timestamp == rs.ProposalBlock.Header.Time`.
|
||||
- if `rs.LockedBlock` is defined then
|
||||
`vote.Time = max(rs.LockedBlock.Timestamp + config.BlockTimeIota, time.Now())`, where `time.Now()`
|
||||
denotes local Unix time in milliseconds, and `config.BlockTimeIota` is a configuration parameter that corresponds
|
||||
to the minimum timestamp increment of the next block.
|
||||
|
||||
- else if `rs.Proposal` is defined then
|
||||
`vote.Time = max(rs.Proposal.Timestamp + config.BlockTimeIota, time.Now())`,
|
||||
|
||||
- otherwise, `vote.Time = time.Now())`. In this case vote is for `nil` so it is not taken into account for
|
||||
the timestamp of the next block.
|
||||
|
||||
- Furthermore, when creating the `vote` message, the following rules for determining `vote.Time` field should hold:
|
||||
|
||||
- if `rs.Proposal` is defined then
|
||||
`vote.Time = max(rs.Proposal.Timestamp + 1, time.Now())`, where `time.Now()`
|
||||
denotes local Unix time in milliseconds.
|
||||
|
||||
- if `rs.Proposal` is not defined and `rs.Votes` contains +2/3 of the corresponding vote messages (votes for the
|
||||
current height and round, and with the corresponding type (`Prevote` or `Precommit`)), then
|
||||
|
||||
`vote.Time = max(median(getVotes(rs.Votes, vote.Height, vote.Round, vote.Type)), time.Now())`,
|
||||
|
||||
where `getVotes` function returns the votes for particular `Height`, `Round` and `Type`.
|
||||
The second rule is relevant for the case when a process jumps to a higher round upon receiving +2/3 votes for a higher
|
||||
round, but the corresponding `Proposal` message for the higher round hasn't been received yet.
|
||||
|
@ -3,13 +3,13 @@ package evidence
|
||||
import (
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
dbm "github.com/tendermint/tendermint/libs/db"
|
||||
sm "github.com/tendermint/tendermint/state"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
var mockState = sm.State{}
|
||||
@ -25,7 +25,7 @@ func initializeValidatorState(valAddr []byte, height int64) dbm.DB {
|
||||
}
|
||||
state := sm.State{
|
||||
LastBlockHeight: 0,
|
||||
LastBlockTime: time.Now(),
|
||||
LastBlockTime: tmtime.Now(),
|
||||
Validators: valSet,
|
||||
NextValidators: valSet.CopyIncrementAccum(1),
|
||||
LastHeightValidatorsChanged: 1,
|
||||
|
@ -90,7 +90,7 @@ func (l tmfmtLogger) Log(keyvals ...interface{}) error {
|
||||
// D - first character of the level, uppercase (ASCII only)
|
||||
// [05-02|11:06:44.322] - our time format (see https://golang.org/src/time/format.go)
|
||||
// Stopping ... - message
|
||||
enc.buf.WriteString(fmt.Sprintf("%c[%s] %-44s ", lvl[0]-32, time.Now().UTC().Format("01-02|15:04:05.000"), msg))
|
||||
enc.buf.WriteString(fmt.Sprintf("%c[%s] %-44s ", lvl[0]-32, time.Now().Format("01-02|15:04:05.000"), msg))
|
||||
|
||||
if module != unknown {
|
||||
enc.buf.WriteString("module=" + module + " ")
|
||||
|
@ -1,13 +1,12 @@
|
||||
package lite
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
crypto "github.com/tendermint/tendermint/crypto"
|
||||
"github.com/tendermint/tendermint/crypto"
|
||||
"github.com/tendermint/tendermint/crypto/ed25519"
|
||||
"github.com/tendermint/tendermint/crypto/secp256k1"
|
||||
|
||||
"github.com/tendermint/tendermint/types"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
// privKeys is a helper type for testing.
|
||||
@ -97,7 +96,7 @@ func makeVote(header *types.Header, valset *types.ValidatorSet, key crypto.PrivK
|
||||
ValidatorIndex: idx,
|
||||
Height: header.Height,
|
||||
Round: 1,
|
||||
Timestamp: time.Now().Round(0).UTC(),
|
||||
Timestamp: tmtime.Now(),
|
||||
Type: types.VoteTypePrecommit,
|
||||
BlockID: types.BlockID{Hash: header.Hash()},
|
||||
}
|
||||
@ -119,7 +118,7 @@ func genHeader(chainID string, height int64, txs types.Txs,
|
||||
return &types.Header{
|
||||
ChainID: chainID,
|
||||
Height: height,
|
||||
Time: time.Now().Round(0).UTC(),
|
||||
Time: tmtime.Now(),
|
||||
NumTxs: int64(len(txs)),
|
||||
TotalTxs: int64(len(txs)),
|
||||
// LastBlockID
|
||||
|
@ -12,6 +12,7 @@ import (
|
||||
"github.com/tendermint/tendermint/crypto/ed25519"
|
||||
cmn "github.com/tendermint/tendermint/libs/common"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
// TODO: type ?
|
||||
@ -324,7 +325,7 @@ func checkVotesOnlyDifferByTimestamp(lastSignBytes, newSignBytes []byte) (time.T
|
||||
}
|
||||
|
||||
// set the times to the same value and check equality
|
||||
now := types.CanonicalTime(time.Now())
|
||||
now := types.CanonicalTime(tmtime.Now())
|
||||
lastVote.Timestamp = now
|
||||
newVote.Timestamp = now
|
||||
lastVoteBytes, _ := cdc.MarshalJSON(lastVote)
|
||||
@ -350,7 +351,7 @@ func checkProposalsOnlyDifferByTimestamp(lastSignBytes, newSignBytes []byte) (ti
|
||||
}
|
||||
|
||||
// set the times to the same value and check equality
|
||||
now := types.CanonicalTime(time.Now())
|
||||
now := types.CanonicalTime(tmtime.Now())
|
||||
lastProposal.Timestamp = now
|
||||
newProposal.Timestamp = now
|
||||
lastProposalBytes, _ := cdc.MarshalJSON(lastProposal)
|
||||
|
@ -12,6 +12,7 @@ import (
|
||||
"github.com/stretchr/testify/require"
|
||||
"github.com/tendermint/tendermint/crypto/ed25519"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
func TestGenLoadValidator(t *testing.T) {
|
||||
@ -235,7 +236,7 @@ func newVote(addr types.Address, idx int, height int64, round int, typ byte, blo
|
||||
Height: height,
|
||||
Round: round,
|
||||
Type: typ,
|
||||
Timestamp: time.Now().UTC(),
|
||||
Timestamp: tmtime.Now(),
|
||||
BlockID: blockID,
|
||||
}
|
||||
}
|
||||
@ -245,6 +246,6 @@ func newProposal(height int64, round int, partsHeader types.PartSetHeader) *type
|
||||
Height: height,
|
||||
Round: round,
|
||||
BlockPartsHeader: partsHeader,
|
||||
Timestamp: time.Now().UTC(),
|
||||
Timestamp: tmtime.Now(),
|
||||
}
|
||||
}
|
||||
|
@ -10,12 +10,13 @@ package main
|
||||
import (
|
||||
"bufio"
|
||||
"fmt"
|
||||
"github.com/tendermint/go-amino"
|
||||
cs "github.com/tendermint/tendermint/consensus"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
"io"
|
||||
"os"
|
||||
"strings"
|
||||
|
||||
"github.com/tendermint/go-amino"
|
||||
cs "github.com/tendermint/tendermint/consensus"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
var cdc = amino.NewCodec()
|
||||
|
@ -15,6 +15,7 @@ import (
|
||||
cmn "github.com/tendermint/tendermint/libs/common"
|
||||
dbm "github.com/tendermint/tendermint/libs/db"
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
|
||||
"github.com/tendermint/tendermint/proxy"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
@ -62,7 +63,7 @@ func TestBeginBlockValidators(t *testing.T) {
|
||||
prevParts := types.PartSetHeader{}
|
||||
prevBlockID := types.BlockID{prevHash, prevParts}
|
||||
|
||||
now := time.Now().UTC()
|
||||
now := tmtime.Now()
|
||||
vote0 := &types.Vote{ValidatorIndex: 0, Timestamp: now, Type: types.VoteTypePrecommit}
|
||||
vote1 := &types.Vote{ValidatorIndex: 1, Timestamp: now}
|
||||
|
||||
@ -81,6 +82,7 @@ func TestBeginBlockValidators(t *testing.T) {
|
||||
|
||||
// block for height 2
|
||||
block, _ := state.MakeBlock(2, makeTxs(2), lastCommit, nil, state.Validators.GetProposer().Address)
|
||||
|
||||
_, err = ExecCommitBlock(proxyApp.Consensus(), block, log.TestingLogger(), state.Validators, stateDB)
|
||||
require.Nil(t, err, tc.desc)
|
||||
|
||||
@ -119,7 +121,7 @@ func TestBeginBlockByzantineValidators(t *testing.T) {
|
||||
ev1 := types.NewMockGoodEvidence(height1, idx1, val1)
|
||||
ev2 := types.NewMockGoodEvidence(height2, idx2, val2)
|
||||
|
||||
now := time.Now()
|
||||
now := tmtime.Now()
|
||||
valSet := state.Validators
|
||||
testCases := []struct {
|
||||
desc string
|
||||
@ -320,6 +322,7 @@ func state(nVals, height int) (State, dbm.DB) {
|
||||
|
||||
for i := 1; i < height; i++ {
|
||||
s.LastBlockHeight++
|
||||
s.LastValidators = s.Validators.Copy()
|
||||
SaveState(stateDB, s)
|
||||
}
|
||||
return s, stateDB
|
||||
|
@ -7,6 +7,7 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/tendermint/tendermint/types"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
// database keys
|
||||
@ -115,6 +116,16 @@ func (state State) MakeBlock(
|
||||
// Fill rest of header with state data.
|
||||
block.ChainID = state.ChainID
|
||||
|
||||
// Set time
|
||||
if height == 1 {
|
||||
block.Time = tmtime.Now()
|
||||
if block.Time.Before(state.LastBlockTime) {
|
||||
block.Time = state.LastBlockTime // state.LastBlockTime for height == 1 is genesis time
|
||||
}
|
||||
} else {
|
||||
block.Time = MedianTime(commit, state.LastValidators)
|
||||
}
|
||||
|
||||
block.LastBlockID = state.LastBlockID
|
||||
block.TotalTxs = state.LastBlockTotalTx + block.NumTxs
|
||||
|
||||
@ -131,6 +142,26 @@ func (state State) MakeBlock(
|
||||
return block, block.MakePartSet(state.ConsensusParams.BlockGossip.BlockPartSizeBytes)
|
||||
}
|
||||
|
||||
// MedianTime computes a median time for a given Commit (based on Timestamp field of votes messages) and the
|
||||
// corresponding validator set. The computed time is always between timestamps of
|
||||
// the votes sent by honest processes, i.e., a faulty processes can not arbitrarily increase or decrease the
|
||||
// computed value.
|
||||
func MedianTime(commit *types.Commit, validators *types.ValidatorSet) time.Time {
|
||||
|
||||
weightedTimes := make([]*tmtime.WeightedTime, len(commit.Precommits))
|
||||
totalVotingPower := int64(0)
|
||||
|
||||
for i, vote := range commit.Precommits {
|
||||
if vote != nil {
|
||||
_, validator := validators.GetByIndex(vote.ValidatorIndex)
|
||||
totalVotingPower += validator.VotingPower
|
||||
weightedTimes[i] = tmtime.NewWeightedTime(vote.Timestamp, validator.VotingPower)
|
||||
}
|
||||
}
|
||||
|
||||
return tmtime.WeightedMedian(weightedTimes, totalVotingPower)
|
||||
}
|
||||
|
||||
//------------------------------------------------------------------------
|
||||
// Genesis
|
||||
|
||||
|
@ -34,13 +34,6 @@ func validateBlock(stateDB dbm.DB, state State, block *types.Block) error {
|
||||
block.Height,
|
||||
)
|
||||
}
|
||||
/* TODO: Determine bounds for Time
|
||||
See blockchain/reactor "stopSyncingDurationMinutes"
|
||||
|
||||
if !block.Time.After(lastBlockTime) {
|
||||
return errors.New("Invalid Block.Header.Time")
|
||||
}
|
||||
*/
|
||||
|
||||
// Validate prev block info.
|
||||
if !block.LastBlockID.Equals(state.LastBlockID) {
|
||||
@ -112,6 +105,26 @@ func validateBlock(stateDB dbm.DB, state State, block *types.Block) error {
|
||||
}
|
||||
}
|
||||
|
||||
// Validate block Time
|
||||
if block.Height > 1 {
|
||||
if !block.Time.After(state.LastBlockTime) {
|
||||
return fmt.Errorf(
|
||||
"Block time %v not greater than last block time %v",
|
||||
block.Time,
|
||||
state.LastBlockTime,
|
||||
)
|
||||
}
|
||||
|
||||
medianTime := MedianTime(block.LastCommit, state.LastValidators)
|
||||
if !block.Time.Equal(medianTime) {
|
||||
return fmt.Errorf(
|
||||
"Invalid block time. Expected %v, got %v",
|
||||
medianTime,
|
||||
block.Time,
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
// Validate all evidence.
|
||||
// TODO: Each check requires loading an old validator set.
|
||||
// We should cap the amount of evidence per block
|
||||
|
@ -43,7 +43,6 @@ func MakeBlock(height int64, txs []Tx, lastCommit *Commit, evidence []Evidence)
|
||||
block := &Block{
|
||||
Header: Header{
|
||||
Height: height,
|
||||
Time: time.Now(),
|
||||
NumTxs: int64(len(txs)),
|
||||
},
|
||||
Data: Data{
|
||||
|
@ -4,11 +4,11 @@ import (
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"github.com/tendermint/tendermint/crypto/ed25519"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
func TestGenesisBad(t *testing.T) {
|
||||
@ -110,7 +110,7 @@ func TestGenesisValidatorHash(t *testing.T) {
|
||||
|
||||
func randomGenesisDoc() *GenesisDoc {
|
||||
return &GenesisDoc{
|
||||
GenesisTime: time.Now().UTC(),
|
||||
GenesisTime: tmtime.Now(),
|
||||
ChainID: "abc",
|
||||
Validators: []GenesisValidator{{ed25519.GenPrivKey().PubKey(), 10, "myval"}},
|
||||
ConsensusParams: DefaultConsensusParams(),
|
||||
|
@ -6,6 +6,7 @@ import (
|
||||
"time"
|
||||
|
||||
cmn "github.com/tendermint/tendermint/libs/common"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
var (
|
||||
@ -34,7 +35,7 @@ func NewProposal(height int64, round int, blockPartsHeader PartSetHeader, polRou
|
||||
return &Proposal{
|
||||
Height: height,
|
||||
Round: round,
|
||||
Timestamp: time.Now().Round(0).UTC(),
|
||||
Timestamp: tmtime.Now(),
|
||||
BlockPartsHeader: blockPartsHeader,
|
||||
POLRound: polRound,
|
||||
POLBlockID: polBlockID,
|
||||
|
@ -1,6 +1,8 @@
|
||||
package types
|
||||
|
||||
import "time"
|
||||
import (
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
func MakeCommit(blockID BlockID, height int64, round int,
|
||||
voteSet *VoteSet,
|
||||
@ -16,7 +18,7 @@ func MakeCommit(blockID BlockID, height int64, round int,
|
||||
Round: round,
|
||||
Type: VoteTypePrecommit,
|
||||
BlockID: blockID,
|
||||
Timestamp: time.Now().UTC(),
|
||||
Timestamp: tmtime.Now(),
|
||||
}
|
||||
|
||||
_, err := signAddVote(validators[i], vote, voteSet)
|
||||
|
49
types/time/time.go
Normal file
49
types/time/time.go
Normal file
@ -0,0 +1,49 @@
|
||||
package time
|
||||
|
||||
import (
|
||||
"sort"
|
||||
"time"
|
||||
)
|
||||
|
||||
// Now returns UTC time rounded since the zero time.
|
||||
func Now() time.Time {
|
||||
return time.Now().Round(0).UTC()
|
||||
}
|
||||
|
||||
type WeightedTime struct {
|
||||
Time time.Time
|
||||
Weight int64
|
||||
}
|
||||
|
||||
func NewWeightedTime(time time.Time, weight int64) *WeightedTime {
|
||||
return &WeightedTime{
|
||||
Time: time,
|
||||
Weight: weight,
|
||||
}
|
||||
}
|
||||
|
||||
// WeightedMedian computes weighted median time for a given array of WeightedTime and the total voting power.
|
||||
func WeightedMedian(weightedTimes []*WeightedTime, totalVotingPower int64) (res time.Time) {
|
||||
median := totalVotingPower / 2
|
||||
|
||||
sort.Slice(weightedTimes, func(i, j int) bool {
|
||||
if weightedTimes[i] == nil {
|
||||
return false
|
||||
}
|
||||
if weightedTimes[j] == nil {
|
||||
return true
|
||||
}
|
||||
return weightedTimes[i].Time.UnixNano() < weightedTimes[j].Time.UnixNano()
|
||||
})
|
||||
|
||||
for _, weightedTime := range weightedTimes {
|
||||
if weightedTime != nil {
|
||||
if median <= weightedTime.Weight {
|
||||
res = weightedTime.Time
|
||||
break
|
||||
}
|
||||
median -= weightedTime.Weight
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
56
types/time/time_test.go
Normal file
56
types/time/time_test.go
Normal file
@ -0,0 +1,56 @@
|
||||
package time
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestWeightedMedian(t *testing.T) {
|
||||
m := make([]*WeightedTime, 3)
|
||||
|
||||
t1 := Now()
|
||||
t2 := t1.Add(5 * time.Second)
|
||||
t3 := t1.Add(10 * time.Second)
|
||||
|
||||
m[2] = NewWeightedTime(t1, 33) // faulty processes
|
||||
m[0] = NewWeightedTime(t2, 40) // correct processes
|
||||
m[1] = NewWeightedTime(t3, 27) // correct processes
|
||||
totalVotingPower := int64(100)
|
||||
|
||||
median := WeightedMedian(m, totalVotingPower)
|
||||
assert.Equal(t, t2, median)
|
||||
// median always returns value between values of correct processes
|
||||
assert.Equal(t, true, (median.After(t1) || median.Equal(t1)) &&
|
||||
(median.Before(t3) || median.Equal(t3)))
|
||||
|
||||
m[1] = NewWeightedTime(t1, 40) // correct processes
|
||||
m[2] = NewWeightedTime(t2, 27) // correct processes
|
||||
m[0] = NewWeightedTime(t3, 33) // faulty processes
|
||||
totalVotingPower = int64(100)
|
||||
|
||||
median = WeightedMedian(m, totalVotingPower)
|
||||
assert.Equal(t, t2, median)
|
||||
// median always returns value between values of correct processes
|
||||
assert.Equal(t, true, (median.After(t1) || median.Equal(t1)) &&
|
||||
(median.Before(t2) || median.Equal(t2)))
|
||||
|
||||
m = make([]*WeightedTime, 8)
|
||||
t4 := t1.Add(15 * time.Second)
|
||||
t5 := t1.Add(60 * time.Second)
|
||||
|
||||
m[3] = NewWeightedTime(t1, 10) // correct processes
|
||||
m[1] = NewWeightedTime(t2, 10) // correct processes
|
||||
m[5] = NewWeightedTime(t2, 10) // correct processes
|
||||
m[4] = NewWeightedTime(t3, 23) // faulty processes
|
||||
m[0] = NewWeightedTime(t4, 20) // correct processes
|
||||
m[7] = NewWeightedTime(t5, 10) // faulty processes
|
||||
totalVotingPower = int64(83)
|
||||
|
||||
median = WeightedMedian(m, totalVotingPower)
|
||||
assert.Equal(t, t3, median)
|
||||
// median always returns value between values of correct processes
|
||||
assert.Equal(t, true, (median.After(t1) || median.Equal(t1)) &&
|
||||
(median.Before(t4) || median.Equal(t4)))
|
||||
}
|
@ -7,13 +7,13 @@ import (
|
||||
"strings"
|
||||
"testing"
|
||||
"testing/quick"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
crypto "github.com/tendermint/tendermint/crypto"
|
||||
"github.com/tendermint/tendermint/crypto"
|
||||
"github.com/tendermint/tendermint/crypto/ed25519"
|
||||
cmn "github.com/tendermint/tendermint/libs/common"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
func TestValidatorSetBasic(t *testing.T) {
|
||||
@ -384,7 +384,7 @@ func TestValidatorSetVerifyCommit(t *testing.T) {
|
||||
ValidatorIndex: 0,
|
||||
Height: height,
|
||||
Round: 0,
|
||||
Timestamp: time.Now().UTC(),
|
||||
Timestamp: tmtime.Now(),
|
||||
Type: VoteTypePrecommit,
|
||||
BlockID: blockID,
|
||||
}
|
||||
|
@ -3,11 +3,11 @@ package types
|
||||
import (
|
||||
"bytes"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
crypto "github.com/tendermint/tendermint/crypto"
|
||||
"github.com/tendermint/tendermint/crypto"
|
||||
cmn "github.com/tendermint/tendermint/libs/common"
|
||||
tst "github.com/tendermint/tendermint/libs/test"
|
||||
tmtime "github.com/tendermint/tendermint/types/time"
|
||||
)
|
||||
|
||||
// NOTE: privValidators are in order
|
||||
@ -83,7 +83,7 @@ func TestAddVote(t *testing.T) {
|
||||
Height: height,
|
||||
Round: round,
|
||||
Type: VoteTypePrevote,
|
||||
Timestamp: time.Now().UTC(),
|
||||
Timestamp: tmtime.Now(),
|
||||
BlockID: BlockID{nil, PartSetHeader{}},
|
||||
}
|
||||
_, err := signAddVote(val0, vote, voteSet)
|
||||
@ -113,7 +113,7 @@ func Test2_3Majority(t *testing.T) {
|
||||
Height: height,
|
||||
Round: round,
|
||||
Type: VoteTypePrevote,
|
||||
Timestamp: time.Now().UTC(),
|
||||
Timestamp: tmtime.Now(),
|
||||
BlockID: BlockID{nil, PartSetHeader{}},
|
||||
}
|
||||
// 6 out of 10 voted for nil.
|
||||
@ -169,7 +169,7 @@ func Test2_3MajorityRedux(t *testing.T) {
|
||||
ValidatorIndex: -1, // NOTE: must fill in
|
||||
Height: height,
|
||||
Round: round,
|
||||
Timestamp: time.Now().UTC(),
|
||||
Timestamp: tmtime.Now(),
|
||||
Type: VoteTypePrevote,
|
||||
BlockID: BlockID{blockHash, blockPartsHeader},
|
||||
}
|
||||
@ -264,7 +264,7 @@ func TestBadVotes(t *testing.T) {
|
||||
ValidatorIndex: -1,
|
||||
Height: height,
|
||||
Round: round,
|
||||
Timestamp: time.Now().UTC(),
|
||||
Timestamp: tmtime.Now(),
|
||||
Type: VoteTypePrevote,
|
||||
BlockID: BlockID{nil, PartSetHeader{}},
|
||||
}
|
||||
@ -326,7 +326,7 @@ func TestConflicts(t *testing.T) {
|
||||
ValidatorIndex: -1,
|
||||
Height: height,
|
||||
Round: round,
|
||||
Timestamp: time.Now().UTC(),
|
||||
Timestamp: tmtime.Now(),
|
||||
Type: VoteTypePrevote,
|
||||
BlockID: BlockID{nil, PartSetHeader{}},
|
||||
}
|
||||
@ -455,7 +455,7 @@ func TestMakeCommit(t *testing.T) {
|
||||
ValidatorIndex: -1,
|
||||
Height: height,
|
||||
Round: round,
|
||||
Timestamp: time.Now().UTC(),
|
||||
Timestamp: tmtime.Now(),
|
||||
Type: VoteTypePrecommit,
|
||||
BlockID: BlockID{blockHash, blockPartsHeader},
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user