Compare commits

...

5 Commits

Author SHA1 Message Date
Jae Kwon
4ec86eea39 Removed MinValidTime Round0 exception; WiggleR -> WiggleDelta 2018-05-23 17:10:04 -07:00
Jae Kwon
b4120e25ff Implement BFT time 2018-05-23 03:36:18 -07:00
Jae Kwon
5161d0aa86 Update bft-time.md 2018-05-21 12:29:45 -07:00
Jae Kwon
cafdd78b04 Update bft-time spec 2018-05-20 17:36:11 -07:00
Jae Kwon
e429aa2121 time.Now().Round(0).UTC() for all times externally communicated 2018-05-15 18:52:49 -07:00
29 changed files with 185 additions and 114 deletions

View File

@@ -132,7 +132,7 @@ vagrant_test:
### go tests
test:
@echo "--> Running go test"
@go test $(PACKAGES)
@GOCACHE=off go test $(PACKAGES)
test_race:
@echo "--> Running go test --race"

View File

@@ -31,7 +31,7 @@ func makeStateAndBlockStore(logger log.Logger) (sm.State, *BlockStore) {
func newBlockchainReactor(logger log.Logger, maxBlockHeight int64) *BlockchainReactor {
state, blockStore := makeStateAndBlockStore(logger)
// Make the blockchainReactor itself
// Make the blockchainReactor itself.
fastSync := true
var nilApp proxy.AppConnConsensus
blockExec := sm.NewBlockExecutor(dbm.NewMemDB(), log.TestingLogger(), nilApp,
@@ -40,10 +40,10 @@ func newBlockchainReactor(logger log.Logger, maxBlockHeight int64) *BlockchainRe
bcReactor := NewBlockchainReactor(state.Copy(), blockExec, blockStore, fastSync)
bcReactor.SetLogger(logger.With("module", "blockchain"))
// Next: we need to set a switch in order for peers to be added in
// Next: we need to set a switch in order for peers to be added in.
bcReactor.Switch = p2p.NewSwitch(cfg.DefaultP2PConfig())
// Lastly: let's add some blocks in
// Lastly: let's add some blocks in.
for blockHeight := int64(1); blockHeight <= maxBlockHeight; blockHeight++ {
firstBlock := makeBlock(blockHeight, state)
secondBlock := makeBlock(blockHeight+1, state)
@@ -155,7 +155,7 @@ func makeTxs(height int64) (txs []types.Tx) {
}
func makeBlock(height int64, state sm.State) *types.Block {
block, _ := state.MakeBlock(height, makeTxs(height), new(types.Commit))
block, _ := state.MakeBlock(height, makeTxs(height), new(types.Commit), 0)
return block
}

View File

@@ -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: time.Now().Round(0).UTC()}}}
)
// 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: time.Now().Round(0).UTC()}}}
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: time.Now().Round(0).UTC(),
}
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: time.Now().Round(0).UTC()}}}
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: time.Now().Round(0).UTC()}}}
bs.SaveBlock(block, partSet, seenCommit)
require.Equal(t, bs.Height(), block.Header.Height, "expecting the new height to be changed")

View File

@@ -53,7 +53,7 @@ func initFilesWithConfig(config *cfg.Config) error {
} else {
genDoc := types.GenesisDoc{
ChainID: cmn.Fmt("test-chain-%v", cmn.RandStr(6)),
GenesisTime: time.Now(),
GenesisTime: time.Now().Round(0).UTC(),
}
genDoc.Validators = []types.GenesisValidator{{
PubKey: pv.GetPubKey(),

View File

@@ -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: time.Now().Round(0).UTC(),
ChainID: "chain-" + cmn.RandStr(6),
Validators: genVals,
}

View File

@@ -394,6 +394,11 @@ 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
BlockTimeIota int `mapstructure:"blocktime_iota"`
BlockTimeWiggle int `mapstructure:"blocktime_wiggle"`
BlockTimeWiggleDelta int `mapstructure:"blocktime_wiggle_delta"`
}
// DefaultConsensusConfig returns a default configuration for the consensus service
@@ -414,6 +419,9 @@ func DefaultConsensusConfig() *ConsensusConfig {
CreateEmptyBlocksInterval: 0,
PeerGossipSleepDuration: 100,
PeerQueryMaj23SleepDuration: 2000,
BlockTimeIota: 10,
BlockTimeWiggle: 20000,
BlockTimeWiggleDelta: 1000,
}
}
@@ -430,6 +438,10 @@ func TestConsensusConfig() *ConsensusConfig {
cfg.SkipTimeoutCommit = true
cfg.PeerGossipSleepDuration = 5
cfg.PeerQueryMaj23SleepDuration = 250
// 1 second so we actually test the iota function.
// This is enforced logically in the propose function.
cfg.BlockTimeIota = 200
return cfg
}
@@ -473,6 +485,25 @@ func (cfg *ConsensusConfig) PeerQueryMaj23Sleep() time.Duration {
return time.Duration(cfg.PeerQueryMaj23SleepDuration) * time.Millisecond
}
// BlockTimeMinValidTime returns the minimum acceptable block time, as part
// of "subjective time validity". See the BFT time spec.
func (cfg *ConsensusConfig) BlockTimeMinValidTime(lastBlockTime time.Time) time.Time {
return lastBlockTime.
Add(time.Duration(cfg.BlockTimeIota) * time.Millisecond)
}
// BlockTimeMaxValidTime returns the maximum acceptable block time, as part
// of "subjective time validity". See the BFT time spec.
func (cfg *ConsensusConfig) BlockTimeMaxValidTime(now time.Time, round int) time.Time {
return now.
Add(time.Duration(cfg.BlockTimeWiggle) * time.Millisecond).
Add(
time.Duration(
int64(cfg.BlockTimeWiggleDelta)*int64(round),
) * time.Millisecond,
)
}
// WalFile returns the full path to the write-ahead log file
func (cfg *ConsensusConfig) WalFile() string {
if cfg.walFile != "" {

View File

@@ -75,7 +75,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: time.Now().Round(0).UTC(),
Type: voteType,
BlockID: types.BlockID{hash, header},
}
@@ -120,7 +120,7 @@ func startTestRound(cs *ConsensusState, height int64, round int) {
cs.startRoutines(0)
}
// Create proposal block from cs1 but sign it with vs
// Create proposal block from cs1 but sign it with vs.
func decideProposal(cs1 *ConsensusState, vs *validatorStub, height int64, round int) (proposal *types.Proposal, block *types.Block) {
block, blockParts := cs1.createProposalBlock()
if block == nil { // on error
@@ -420,7 +420,7 @@ func randGenesisDoc(numValidators int, randPower bool, minPower int64) (*types.G
sort.Sort(types.PrivValidatorsByAddress(privValidators))
return &types.GenesisDoc{
GenesisTime: time.Now(),
GenesisTime: time.Now().Round(0).UTC(),
ChainID: config.ChainID(),
Validators: validators,
}, privValidators

View File

@@ -1201,7 +1201,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 := time.Now().Round(0).UTC().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

@@ -199,7 +199,7 @@ func TestReactorRecordsVotes(t *testing.T) {
ValidatorAddress: val.Address,
Height: 2,
Round: 0,
Timestamp: time.Now().UTC(),
Timestamp: time.Now().Round(0).UTC(),
Type: types.VoteTypePrevote,
BlockID: types.BlockID{},
}

View File

@@ -386,8 +386,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", time.Now().Round(0).UTC(), "startTime", cs.StartTime)
sleepDuration := rs.StartTime.Sub(time.Now().Round(0).UTC()) // nolint: gotype, gosimple
cs.scheduleTimeout(sleepDuration, rs.Height, 0, cstypes.RoundStepNewHeight)
}
@@ -477,7 +477,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(time.Now().Round(0).UTC())
} else {
cs.StartTime = cs.config.Commit(cs.CommitTime)
}
@@ -672,7 +672,7 @@ func (cs *ConsensusState) enterNewRound(height int64, round int) {
return
}
if now := time.Now(); cs.StartTime.After(now) {
if now := time.Now().Round(0).UTC(); cs.StartTime.After(now) {
cs.Logger.Info("Need to set a buffer and log message here for sanity.", "startTime", cs.StartTime, "now", now)
}
@@ -885,7 +885,7 @@ func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts
// Mempool validated transactions
txs := cs.mempool.Reap(cs.config.MaxBlockSizeTxs)
block, parts := cs.state.MakeBlock(cs.Height, txs, commit)
block, parts := cs.state.MakeBlock(cs.Height, txs, commit, cs.config.BlockTimeIota)
evidence := cs.evpool.PendingEvidence()
block.AddEvidence(evidence)
return block, parts
@@ -941,7 +941,28 @@ func (cs *ConsensusState) defaultDoPrevote(height int64, round int) {
return
}
// Validate proposal block
// Validate proposal block timestamp ("subjective time validity").
// See the BFT time spec.
lastBlockTime := cs.state.LastBlockTime
now := time.Now().Round(0).UTC()
minValidTime := cs.config.BlockTimeMinValidTime(lastBlockTime)
if cs.ProposalBlock.Time.Before(minValidTime) {
logger.Info("enterPrevote: ProposalBlock time too low",
"blockTime", cs.ProposalBlock.Time,
"minValidTime", minValidTime)
cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{})
return
}
maxValidTime := cs.config.BlockTimeMaxValidTime(now, round)
if maxValidTime.Before(cs.ProposalBlock.Time) {
logger.Info("enterPrevote: ProposalBlock time too high",
"blockTime", cs.ProposalBlock.Time,
"maxValidTime", maxValidTime)
cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{})
return
}
// Validate proposal block.
err := cs.blockExec.ValidateBlock(cs.state, cs.ProposalBlock)
if err != nil {
// ProposalBlock is invalid, prevote nil.
@@ -950,8 +971,8 @@ func (cs *ConsensusState) defaultDoPrevote(height int64, round int) {
return
}
// Prevote cs.ProposalBlock
// NOTE: the proposal signature is validated when it is received,
// Prevote cs.ProposalBlock.
// NOTE: The proposal signature is validated when it is received,
// and the proposal block parts are validated as they are received (against the merkle hash in the proposal)
logger.Info("enterPrevote: ProposalBlock is valid")
cs.signAddVote(types.VoteTypePrevote, cs.ProposalBlock.Hash(), cs.ProposalBlockParts.Header())
@@ -1112,7 +1133,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 = time.Now().Round(0).UTC()
cs.newStep()
// Maybe finalize immediately.
@@ -1521,7 +1542,7 @@ func (cs *ConsensusState) signVote(type_ byte, hash []byte, header types.PartSet
ValidatorIndex: valIndex,
Height: cs.Height,
Round: cs.Round,
Timestamp: time.Now().UTC(),
Timestamp: time.Now().Round(0).UTC(),
Type: type_,
BlockID: types.BlockID{hash, header},
}

View 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: time.Now().Round(0).UTC(),
Type: types.VoteTypePrecommit,
BlockID: types.BlockID{[]byte("fakehash"), types.PartSetHeader{}},
}

View File

@@ -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: time.Now().Round(0).UTC(),
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: time.Now().Round(0).UTC(),
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: time.Now().Round(0).UTC(),
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: time.Now().Round(0).UTC(),
CommitTime: time.Now().Round(0).UTC(),
Validators: vset,
Proposal: proposal,
ProposalBlock: block,

View File

@@ -116,7 +116,7 @@ func (wal *baseWAL) Save(msg WALMessage) {
}
// Write the wal message
if err := wal.enc.Encode(&TimedWALMessage{time.Now(), msg}); err != nil {
if err := wal.enc.Encode(&TimedWALMessage{time.Now().Round(0).UTC(), msg}); err != nil {
cmn.PanicQ(cmn.Fmt("Error writing msg to consensus wal: %v \n\nMessage: %v", err, msg))
}

View File

@@ -16,7 +16,7 @@ import (
)
func TestWALEncoderDecoder(t *testing.T) {
now := time.Now()
now := time.Now().Round(0).UTC()
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 +93,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,56 +1,61 @@
# BFT time in Tendermint
Tendermint provides a deterministic, Byzantine fault-tolerant, source of time.
Tendermint provides a deterministic, Byzantine fault-tolerant, source of time.
In the context of Tendermint, time denotes UNIX time in milliseconds, i.e.,
corresponds to the number of milliseconds since January 1, 1970.
Time in Tendermint is defined with the Time field of the block header.
It satisfies the following property:
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 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.
Beyond satisfying time monotinicity, Tendermint also checks the following
property, but only when signing a prevote for a block:
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:
- **Subjective Time Validity**: Time is greater than MinValidTime(last_block_time) and less than or equal to MaxValidTime(now, round), where:
- 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.
```go
// iota provided by consensus config.
func MinValidTime(last_block_time) time.Time {
return lastBlockTime.
Add(time.Duration(iota) * time.Millisecond)
}
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.
// wiggle and wiggle_delta are provided by consensus config.
func MaxValidTime(now time.Time, round int) time.Time {
return now.
Add(time.Duration(wiggle) * time.Millisecond).
Add(
time.Duration(
wiggle_delta*float64(round),
) * time.Millisecond,
)
}
```
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`.
For `MinValidTime`, we accept any block that is at least `iota` greater than
the last block time. Blocks that are significantly older than the current time
can still be valid, which allows for the re-proposing of older proposals.
- Furthermore, when creating the `vote` message, the following rules for determining `vote.Time` field should hold:
For `MaxValidTime`, we accept block times greater than `now` plus some
threshold that increases linearly with the round number. The purpose of
`wiggle_delta` is for graceful degredation when +2/3 validators *aren't* within
`wiggle` of each other but are otherwise non-Byzantine in all other respects.
- 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.
Consider an example with 100 equally weighted validators, where 33 are
Byzantine, and one of the remaining 67 validators has a faulty clock that
causes it to drift back more than `wiggle` from the other 66. Without
`wiggle_delta`, if the 33 Byzantine validators were to withhold their votes, no
block would produce a Polka until the drifting one becomes the proposer.
NOTE: `wiggle_delta` should probably be less than `timeout_propose_delta` to
prevent unnecessary forward time-jumps in cases where higher rounds are reached
due to factors other than network performance -- for example, where the network
is performant but several proposers were absent in a row. `wiggle_delta` could
theoretically be set to 0 if it can be assumed that +2/3 (by voting power) of
correct validators' clocks are within `wiggle` of each other.
Subjective time validity is ignored when a Polka or Commit is found, allowing
consensus to progress locally even when the subjective time requirements are
not satisfied.

View File

@@ -25,7 +25,7 @@ func initializeValidatorState(valAddr []byte, height int64) dbm.DB {
}
state := sm.State{
LastBlockHeight: 0,
LastBlockTime: time.Now(),
LastBlockTime: time.Now().Round(0).UTC(),
Validators: valSet,
LastHeightValidatorsChanged: 1,
ConsensusParams: types.ConsensusParams{

View File

@@ -97,7 +97,7 @@ func makeVote(header *types.Header, vals *types.ValidatorSet, key crypto.PrivKey
ValidatorIndex: idx,
Height: header.Height,
Round: 1,
Timestamp: time.Now().UTC(),
Timestamp: time.Now().Round(0).UTC(),
Type: types.VoteTypePrecommit,
BlockID: types.BlockID{Hash: header.Hash()},
}
@@ -115,7 +115,7 @@ func genHeader(chainID string, height int64, txs types.Txs,
return &types.Header{
ChainID: chainID,
Height: height,
Time: time.Now(),
Time: time.Now().Round(0).UTC(),
NumTxs: int64(len(txs)),
TotalTxs: int64(len(txs)),
// LastBlockID

View File

@@ -60,7 +60,7 @@ func TestBeginBlockAbsentValidators(t *testing.T) {
prevParts := types.PartSetHeader{}
prevBlockID := types.BlockID{prevHash, prevParts}
now := time.Now().UTC()
now := time.Now().Round(0).UTC()
testCases := []struct {
desc string
lastCommitPrecommits []*types.Vote
@@ -74,7 +74,7 @@ func TestBeginBlockAbsentValidators(t *testing.T) {
for _, tc := range testCases {
lastCommit := &types.Commit{BlockID: prevBlockID, Precommits: tc.lastCommitPrecommits}
block, _ := state.MakeBlock(2, makeTxs(2), lastCommit)
block, _ := state.MakeBlock(2, makeTxs(2), lastCommit, 0)
_, err = ExecCommitBlock(proxyApp.Consensus(), block, log.TestingLogger())
require.Nil(t, err, tc.desc)
@@ -118,7 +118,7 @@ func TestBeginBlockByzantineValidators(t *testing.T) {
for _, tc := range testCases {
lastCommit := &types.Commit{BlockID: prevBlockID}
block, _ := state.MakeBlock(10, makeTxs(2), lastCommit)
block, _ := state.MakeBlock(10, makeTxs(2), lastCommit, 0)
block.Evidence.Evidence = tc.evidence
_, err = ExecCommitBlock(proxyApp.Consensus(), block, log.TestingLogger())
require.Nil(t, err, tc.desc)
@@ -150,7 +150,7 @@ func state() State {
}
func makeBlock(state State, height int64) *types.Block {
block, _ := state.MakeBlock(height, makeTxs(state.LastBlockHeight), new(types.Commit))
block, _ := state.MakeBlock(height, makeTxs(state.LastBlockHeight), new(types.Commit), 0)
return block
}

View File

@@ -102,11 +102,12 @@ func (s State) GetValidators() (last *types.ValidatorSet, current *types.Validat
// Create a block from the latest state
// MakeBlock builds a block with the given txs and commit from the current state.
func (s State) MakeBlock(height int64, txs []types.Tx, commit *types.Commit) (*types.Block, *types.PartSet) {
// build base block
func (s State) MakeBlock(height int64, txs []types.Tx, commit *types.Commit, blockTimeIota int) (*types.Block, *types.PartSet) {
// Build base block.
block := types.MakeBlock(height, txs, commit)
// fill header with state data
// Fill header with state data.
block.ChainID = s.ChainID
block.TotalTxs = s.LastBlockTotalTx + block.NumTxs
block.LastBlockID = s.LastBlockID
@@ -115,6 +116,12 @@ func (s State) MakeBlock(height int64, txs []types.Tx, commit *types.Commit) (*t
block.ConsensusHash = s.ConsensusParams.Hash()
block.LastResultsHash = s.LastResultsHash
// Ensure valid time. See BFT time spec.
minValidTime := s.LastBlockTime.Add(time.Duration(blockTimeIota) * time.Millisecond)
if block.Time.Before(minValidTime) {
block.Time = minValidTime
}
return block, block.MakePartSet(s.ConsensusParams.BlockGossip.BlockPartSizeBytes)
}

View File

@@ -13,12 +13,12 @@ import (
// Validate block
func validateBlock(stateDB dbm.DB, s State, b *types.Block) error {
// validate internal consistency
// Validate internal consistency.
if err := b.ValidateBasic(); err != nil {
return err
}
// validate basic info
// Validate basic info.
if b.ChainID != s.ChainID {
return fmt.Errorf("Wrong Block.Header.ChainID. Expected %v, got %v", s.ChainID, b.ChainID)
}
@@ -33,18 +33,21 @@ func validateBlock(stateDB dbm.DB, s State, b *types.Block) error {
}
*/
// validate prev block info
// Validate prev block info.
if !b.LastBlockID.Equals(s.LastBlockID) {
return fmt.Errorf("Wrong Block.Header.LastBlockID. Expected %v, got %v", s.LastBlockID, b.LastBlockID)
return fmt.Errorf("Wrong Block.Header.LastBlockID. Expected %v, got %v", s.LastBlockID, b.LastBlockID)
}
newTxs := int64(len(b.Data.Txs))
if b.TotalTxs != s.LastBlockTotalTx+newTxs {
return fmt.Errorf("Wrong Block.Header.TotalTxs. Expected %v, got %v", s.LastBlockTotalTx+newTxs, b.TotalTxs)
}
if !b.Time.After(s.LastBlockTime) {
return fmt.Errorf("Bad Block.Time. Expected something greater than %v (last block time), but got %v", s.LastBlockTime, b.Time)
}
// validate app info
// Validate app info.
if !bytes.Equal(b.AppHash, s.AppHash) {
return fmt.Errorf("Wrong Block.Header.AppHash. Expected %X, got %v", s.AppHash, b.AppHash)
return fmt.Errorf("Wrong Block.Header.AppHash. Expected %X, got %v", s.AppHash, b.AppHash)
}
if !bytes.Equal(b.ConsensusHash, s.ConsensusParams.Hash()) {
return fmt.Errorf("Wrong Block.Header.ConsensusHash. Expected %X, got %v", s.ConsensusParams.Hash(), b.ConsensusHash)

View File

@@ -29,8 +29,9 @@ func MakeBlock(height int64, txs []Tx, commit *Commit) *Block {
block := &Block{
Header: &Header{
Height: height,
Time: time.Now(),
Time: time.Now().Round(0), // Strip monotonic.
NumTxs: int64(len(txs)),
Nonce: cmn.RandStr(12),
},
LastCommit: commit,
Data: &Data{
@@ -177,6 +178,7 @@ type Header struct {
Height int64 `json:"height"`
Time time.Time `json:"time"`
NumTxs int64 `json:"num_txs"`
Nonce string `json:"nonce"`
// prev block info
LastBlockID BlockID `json:"last_block_id"`
@@ -209,6 +211,7 @@ func (h *Header) Hash() cmn.HexBytes {
"Height": aminoHasher(h.Height),
"Time": aminoHasher(h.Time),
"NumTxs": aminoHasher(h.NumTxs),
"Nonce": aminoHasher(h.Nonce),
"TotalTxs": aminoHasher(h.TotalTxs),
"LastBlockID": aminoHasher(h.LastBlockID),
"LastCommit": aminoHasher(h.LastCommitHash),
@@ -231,6 +234,7 @@ func (h *Header) StringIndented(indent string) string {
%s Height: %v
%s Time: %v
%s NumTxs: %v
%s Nonce: %v
%s TotalTxs: %v
%s LastBlockID: %v
%s LastCommit: %v
@@ -245,6 +249,7 @@ func (h *Header) StringIndented(indent string) string {
indent, h.Height,
indent, h.Time,
indent, h.NumTxs,
indent, h.Nonce,
indent, h.TotalTxs,
indent, h.LastBlockID,
indent, h.LastCommitHash,

View File

@@ -21,7 +21,10 @@ type EventBusSubscriber interface {
// are proxied to underlying pubsub server. All events must be published using
// EventBus to ensure correct data types.
type EventBus struct {
// JAE: This is supposed to be unnecessary as pubsub is already.
// TODO: Use BaseService as intended by overriding OnStart/Stop.
cmn.BaseService
pubsub *tmpubsub.Server
}
@@ -32,7 +35,7 @@ func NewEventBus() *EventBus {
// NewEventBusWithBufferCapacity returns a new event bus with the given buffer capacity.
func NewEventBusWithBufferCapacity(cap int) *EventBus {
// capacity could be exposed later if needed
// Capacity could be exposed later if needed.
pubsub := tmpubsub.NewServer(tmpubsub.BufferCapacity(cap))
b := &EventBus{pubsub: pubsub}
b.BaseService = *cmn.NewBaseService(nil, "EventBus", b)

View File

@@ -84,10 +84,6 @@ func (genDoc *GenesisDoc) ValidateAndComplete() error {
}
}
if genDoc.GenesisTime.IsZero() {
genDoc.GenesisTime = time.Now()
}
return nil
}

View File

@@ -309,7 +309,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(time.Now().Round(0).UTC())
lastVote.Timestamp = now
newVote.Timestamp = now
lastVoteBytes, _ := cdc.MarshalJSON(lastVote)
@@ -335,7 +335,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(time.Now().Round(0).UTC())
lastProposal.Timestamp = now
newProposal.Timestamp = now
lastProposalBytes, _ := cdc.MarshalJSON(lastProposal)

View File

@@ -229,7 +229,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: time.Now().Round(0).UTC(),
BlockID: blockID,
}
}
@@ -239,7 +239,7 @@ func newProposal(height int64, round int, partsHeader types.PartSetHeader) *type
Height: height,
Round: round,
BlockPartsHeader: partsHeader,
Timestamp: time.Now().UTC(),
Timestamp: time.Now().Round(0).UTC(),
}
}

View File

@@ -34,7 +34,7 @@ func NewProposal(height int64, round int, blockPartsHeader PartSetHeader, polRou
return &Proposal{
Height: height,
Round: round,
Timestamp: time.Now().UTC(),
Timestamp: time.Now().Round(0).UTC(),
BlockPartsHeader: blockPartsHeader,
POLRound: polRound,
POLBlockID: polBlockID,

View File

@@ -16,7 +16,7 @@ func MakeCommit(blockID BlockID, height int64, round int,
Round: round,
Type: VoteTypePrecommit,
BlockID: blockID,
Timestamp: time.Now().UTC(),
Timestamp: time.Now().Round(0).UTC(),
}
_, err := signAddVote(validators[i], vote, voteSet)

View File

@@ -327,7 +327,7 @@ func TestValidatorSetVerifyCommit(t *testing.T) {
ValidatorIndex: 0,
Height: height,
Round: 0,
Timestamp: time.Now().UTC(),
Timestamp: time.Now().Round(0).UTC(),
Type: VoteTypePrecommit,
BlockID: blockID,
}

View File

@@ -83,7 +83,7 @@ func TestAddVote(t *testing.T) {
Height: height,
Round: round,
Type: VoteTypePrevote,
Timestamp: time.Now().UTC(),
Timestamp: time.Now().Round(0).UTC(),
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: time.Now().Round(0).UTC(),
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: time.Now().Round(0).UTC(),
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: time.Now().Round(0).UTC(),
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: time.Now().Round(0).UTC(),
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: time.Now().Round(0).UTC(),
Type: VoteTypePrecommit,
BlockID: BlockID{blockHash, blockPartsHeader},
}