Implement BFT time (#2203)

* Implement BFT time

* set LastValidators when creating state in state helper

for heights >= 2
This commit is contained in:
Zarko Milosevic 2018-09-01 01:33:51 +02:00 committed by Ethan Buchman
parent ffe91ae9e3
commit 7b88172f41
31 changed files with 306 additions and 112 deletions

View File

@ -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:

View File

@ -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")

View File

@ -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{{

View File

@ -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,
}

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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{},
}

View File

@ -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

View File

@ -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{}},
}

View File

@ -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,

View File

@ -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))
}
}

View File

@ -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()

View File

@ -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.

View File

@ -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,

View File

@ -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 + " ")

View File

@ -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

View File

@ -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)

View File

@ -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(),
}
}

View File

@ -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()

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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{

View File

@ -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(),

View File

@ -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,

View File

@ -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
View 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
View 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)))
}

View File

@ -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,
}

View File

@ -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},
}