make ConsensusParams.EvidenceParams.MaxAge time

Refs #2565
This commit is contained in:
Anton Kaliaev 2018-10-11 15:07:05 +04:00
parent 80562669bf
commit 7e7e4c74ca
No known key found for this signature in database
GPG Key ID: 7B6881D965918214
15 changed files with 520 additions and 269 deletions

View File

@ -44,7 +44,7 @@ protoc_all: protoc_libs protoc_merkle protoc_abci protoc_grpc
## See https://stackoverflow.com/a/25518702 ## See https://stackoverflow.com/a/25518702
## Note the $< here is substituted for the %.proto ## Note the $< here is substituted for the %.proto
## Note the $@ here is substituted for the %.pb.go ## Note the $@ here is substituted for the %.pb.go
protoc $(INCLUDE) $< --gogo_out=Mgoogle/protobuf/timestamp.proto=github.com/golang/protobuf/ptypes/timestamp,plugins=grpc:. protoc $(INCLUDE) $< --gogo_out=Mgoogle/protobuf/timestamp.proto=github.com/golang/protobuf/ptypes/timestamp,Mgoogle/protobuf/duration.proto=github.com/golang/protobuf/ptypes/duration,plugins=grpc:.
######################################## ########################################
### Build ABCI ### Build ABCI

File diff suppressed because it is too large Load Diff

View File

@ -5,6 +5,7 @@ package types;
// https://github.com/gogo/protobuf/blob/master/extensions.md // https://github.com/gogo/protobuf/blob/master/extensions.md
import "github.com/gogo/protobuf/gogoproto/gogo.proto"; import "github.com/gogo/protobuf/gogoproto/gogo.proto";
import "google/protobuf/timestamp.proto"; import "google/protobuf/timestamp.proto";
import "google/protobuf/duration.proto";
import "github.com/tendermint/tendermint/libs/common/types.proto"; import "github.com/tendermint/tendermint/libs/common/types.proto";
import "github.com/tendermint/tendermint/crypto/merkle/merkle.proto"; import "github.com/tendermint/tendermint/crypto/merkle/merkle.proto";
@ -217,8 +218,8 @@ message BlockSize {
// EvidenceParams contains limits on the evidence. // EvidenceParams contains limits on the evidence.
message EvidenceParams { message EvidenceParams {
// Note: must be greater than 0 // Note: must be greater than 0 if provided
int64 max_age = 1; google.protobuf.Duration max_age = 1 [(gogoproto.stdduration)=true];
} }
message LastCommitInfo { message LastCommitInfo {

View File

@ -13,6 +13,7 @@ import golang_proto "github.com/golang/protobuf/proto"
import fmt "fmt" import fmt "fmt"
import math "math" import math "math"
import _ "github.com/gogo/protobuf/gogoproto" import _ "github.com/gogo/protobuf/gogoproto"
import _ "github.com/golang/protobuf/ptypes/duration"
import _ "github.com/golang/protobuf/ptypes/timestamp" import _ "github.com/golang/protobuf/ptypes/timestamp"
import _ "github.com/tendermint/tendermint/crypto/merkle" import _ "github.com/tendermint/tendermint/crypto/merkle"
import _ "github.com/tendermint/tendermint/libs/common" import _ "github.com/tendermint/tendermint/libs/common"

View File

@ -8,8 +8,7 @@ import (
"github.com/pkg/errors" "github.com/pkg/errors"
"github.com/tendermint/go-amino" amino "github.com/tendermint/go-amino"
cstypes "github.com/tendermint/tendermint/consensus/types" cstypes "github.com/tendermint/tendermint/consensus/types"
cmn "github.com/tendermint/tendermint/libs/common" cmn "github.com/tendermint/tendermint/libs/common"
tmevents "github.com/tendermint/tendermint/libs/events" tmevents "github.com/tendermint/tendermint/libs/events"
@ -438,6 +437,7 @@ func makeRoundStepMessages(rs *cstypes.RoundState) (nrsMsg *NewRoundStepMessage,
if rs.Step == cstypes.RoundStepCommit { if rs.Step == cstypes.RoundStepCommit {
csMsg = &CommitStepMessage{ csMsg = &CommitStepMessage{
Height: rs.Height, Height: rs.Height,
Time: rs.CommitTime,
BlockPartsHeader: rs.ProposalBlockParts.Header(), BlockPartsHeader: rs.ProposalBlockParts.Header(),
BlockParts: rs.ProposalBlockParts.BitArray(), BlockParts: rs.ProposalBlockParts.BitArray(),
} }
@ -948,14 +948,22 @@ func (ps *PeerState) ToJSON() ([]byte, error) {
return cdc.MarshalJSON(ps) return cdc.MarshalJSON(ps)
} }
// GetHeight returns an atomic snapshot of the PeerRoundState's height // GetHeight returns an atomic snapshot of the PeerRoundState's height used by
// used by the mempool to ensure peers are caught up before broadcasting new txs // the mempool to ensure peers are caught up before broadcasting new txs.
func (ps *PeerState) GetHeight() int64 { func (ps *PeerState) GetHeight() int64 {
ps.mtx.Lock() ps.mtx.Lock()
defer ps.mtx.Unlock() defer ps.mtx.Unlock()
return ps.PRS.Height return ps.PRS.Height
} }
// GetLastBlockTime returns an atomic snapshot of the PeerRoundState's last
// block time used by the evidence reactor when sending evidence.
func (ps *PeerState) GetLastBlockTime() time.Time {
ps.mtx.Lock()
defer ps.mtx.Unlock()
return ps.PRS.LastBlockTime
}
// SetHasProposal sets the given proposal as known for the peer. // SetHasProposal sets the given proposal as known for the peer.
func (ps *PeerState) SetHasProposal(proposal *types.Proposal) { func (ps *PeerState) SetHasProposal(proposal *types.Proposal) {
ps.mtx.Lock() ps.mtx.Lock()
@ -1263,6 +1271,7 @@ func (ps *PeerState) ApplyCommitStepMessage(msg *CommitStepMessage) {
ps.PRS.ProposalBlockPartsHeader = msg.BlockPartsHeader ps.PRS.ProposalBlockPartsHeader = msg.BlockPartsHeader
ps.PRS.ProposalBlockParts = msg.BlockParts ps.PRS.ProposalBlockParts = msg.BlockParts
ps.PRS.LastBlockTime = msg.Time
} }
// ApplyProposalPOLMessage updates the peer state for the new proposal POL. // ApplyProposalPOLMessage updates the peer state for the new proposal POL.
@ -1386,13 +1395,14 @@ func (m *NewRoundStepMessage) String() string {
// CommitStepMessage is sent when a block is committed. // CommitStepMessage is sent when a block is committed.
type CommitStepMessage struct { type CommitStepMessage struct {
Height int64 Height int64
Time time.Time
BlockPartsHeader types.PartSetHeader BlockPartsHeader types.PartSetHeader
BlockParts *cmn.BitArray BlockParts *cmn.BitArray
} }
// String returns a string representation. // String returns a string representation.
func (m *CommitStepMessage) String() string { func (m *CommitStepMessage) String() string {
return fmt.Sprintf("[CommitStep H:%v BP:%v BA:%v]", m.Height, m.BlockPartsHeader, m.BlockParts) return fmt.Sprintf("[CommitStep H:%v T:%v BP:%v BA:%v]", m.Height, m.Time, m.BlockPartsHeader, m.BlockParts)
} }
//------------------------------------- //-------------------------------------

View File

@ -14,6 +14,7 @@ import (
// NOTE: Read-only when returned by PeerState.GetRoundState(). // NOTE: Read-only when returned by PeerState.GetRoundState().
type PeerRoundState struct { type PeerRoundState struct {
Height int64 `json:"height"` // Height peer is at Height int64 `json:"height"` // Height peer is at
LastBlockTime time.Time `json:"last_block_time"` // Time the last block was created at.
Round int `json:"round"` // Round peer is at, -1 if unknown. Round int `json:"round"` // Round peer is at, -1 if unknown.
Step RoundStepType `json:"step"` // Step peer is at Step RoundStepType `json:"step"` // Step peer is at
StartTime time.Time `json:"start_time"` // Estimated start of round 0 at this height StartTime time.Time `json:"start_time"` // Estimated start of round 0 at this height
@ -38,7 +39,7 @@ func (prs PeerRoundState) String() string {
// StringIndented returns a string representation of the PeerRoundState // StringIndented returns a string representation of the PeerRoundState
func (prs PeerRoundState) StringIndented(indent string) string { func (prs PeerRoundState) StringIndented(indent string) string {
return fmt.Sprintf(`PeerRoundState{ return fmt.Sprintf(`PeerRoundState{
%s %v/%v/%v @%v %s %v/%v/%v @%v (last block time @%v)
%s Proposal %v -> %v %s Proposal %v -> %v
%s POL %v (round %v) %s POL %v (round %v)
%s Prevotes %v %s Prevotes %v
@ -46,7 +47,7 @@ func (prs PeerRoundState) StringIndented(indent string) string {
%s LastCommit %v (round %v) %s LastCommit %v (round %v)
%s Catchup %v (round %v) %s Catchup %v (round %v)
%s}`, %s}`,
indent, prs.Height, prs.Round, prs.Step, prs.StartTime, indent, prs.Height, prs.Round, prs.Step, prs.StartTime, prs.LastBlockTime,
indent, prs.ProposalBlockPartsHeader, prs.ProposalBlockParts, indent, prs.ProposalBlockPartsHeader, prs.ProposalBlockParts,
indent, prs.ProposalPOL, prs.ProposalPOLRound, indent, prs.ProposalPOL, prs.ProposalPOLRound,
indent, prs.Prevotes, indent, prs.Prevotes,

View File

@ -3,6 +3,7 @@ package evidence
import ( import (
"fmt" "fmt"
"sync" "sync"
"time"
clist "github.com/tendermint/tendermint/libs/clist" clist "github.com/tendermint/tendermint/libs/clist"
dbm "github.com/tendermint/tendermint/libs/db" dbm "github.com/tendermint/tendermint/libs/db"
@ -128,19 +129,27 @@ func (evpool *EvidencePool) MarkEvidenceAsCommitted(height int64, evidence []typ
// remove committed evidence from the clist // remove committed evidence from the clist
maxAge := evpool.State().ConsensusParams.EvidenceParams.MaxAge maxAge := evpool.State().ConsensusParams.EvidenceParams.MaxAge
evpool.removeEvidence(height, maxAge, blockEvidenceMap) evpool.removeEvidence(
height,
evpool.State().LastBlockTime,
maxAge,
blockEvidenceMap,
)
} }
func (evpool *EvidencePool) removeEvidence(height, maxAge int64, blockEvidenceMap map[string]struct{}) { func (evpool *EvidencePool) removeEvidence(
height int64,
lastBlockTime time.Time,
maxAge time.Duration,
blockEvidenceMap map[string]struct{},
) {
for e := evpool.evidenceList.Front(); e != nil; e = e.Next() { for e := evpool.evidenceList.Front(); e != nil; e = e.Next() {
ev := e.Value.(types.Evidence) ev := e.Value.(types.Evidence)
evAge := lastBlockTime.Sub(ev.Time())
// Remove the evidence if it's already in a block // Remove the evidence if it's already in a block
// or if it's now too old. // or if it's now too old.
if _, ok := blockEvidenceMap[evMapKey(ev)]; ok || if _, ok := blockEvidenceMap[evMapKey(ev)]; ok || evAge > maxAge {
ev.Height() < height-maxAge {
// remove from clist // remove from clist
evpool.evidenceList.Remove(e) evpool.evidenceList.Remove(e)
e.DetachPrev() e.DetachPrev()

View File

@ -91,7 +91,7 @@ func (evR *EvidenceReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
} }
} }
// SetEventSwitch implements events.Eventable. // SetEventBus implements events.Eventable.
func (evR *EvidenceReactor) SetEventBus(b *types.EventBus) { func (evR *EvidenceReactor) SetEventBus(b *types.EventBus) {
evR.eventBus = b evR.eventBus = b
} }
@ -166,10 +166,11 @@ func (evR EvidenceReactor) checkSendEvidenceMessage(peer p2p.Peer, ev types.Evid
// peerHeight - maxAge < evidenceHeight < peerHeight // peerHeight - maxAge < evidenceHeight < peerHeight
maxAge := evR.evpool.State().ConsensusParams.EvidenceParams.MaxAge maxAge := evR.evpool.State().ConsensusParams.EvidenceParams.MaxAge
peerHeight := peerState.GetHeight() peerHeight := peerState.GetHeight()
peerLastBlockTime := peerState.GetLastBlockTime()
if peerHeight < evHeight { if peerHeight < evHeight {
// peer is behind. sleep while he catches up // peer is behind. sleep while he catches up
return nil, true return nil, true
} else if peerHeight > evHeight+maxAge { } else if peerLastBlockTime.Sub(ev.Time()) > maxAge {
// evidence is too old, skip // evidence is too old, skip
// NOTE: if evidence is too old for an honest peer, // NOTE: if evidence is too old for an honest peer,
// then we're behind and either it already got committed or it never will! // then we're behind and either it already got committed or it never will!
@ -185,6 +186,7 @@ func (evR EvidenceReactor) checkSendEvidenceMessage(peer p2p.Peer, ev types.Evid
// PeerState describes the state of a peer. // PeerState describes the state of a peer.
type PeerState interface { type PeerState interface {
GetHeight() int64 GetHeight() int64
GetLastBlockTime() time.Time
} }
//----------------------------------------------------------------------------- //-----------------------------------------------------------------------------

View File

@ -14,6 +14,7 @@ import (
"github.com/tendermint/tendermint/libs/log" "github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/types" "github.com/tendermint/tendermint/types"
tmtime "github.com/tendermint/tendermint/types/time"
) )
// evidenceLogger is a TestingLogger which uses a different // evidenceLogger is a TestingLogger which uses a different
@ -132,7 +133,7 @@ func TestReactorBroadcastEvidence(t *testing.T) {
// set the peer height on each reactor // set the peer height on each reactor
for _, r := range reactors { for _, r := range reactors {
for _, peer := range r.Switch.Peers().List() { for _, peer := range r.Switch.Peers().List() {
ps := peerState{height} ps := testPeerState{height, tmtime.Now()}
peer.Set(types.PeerStateKey, ps) peer.Set(types.PeerStateKey, ps)
} }
} }
@ -143,14 +144,21 @@ func TestReactorBroadcastEvidence(t *testing.T) {
waitForEvidence(t, evList, reactors) waitForEvidence(t, evList, reactors)
} }
type peerState struct { type testPeerState struct {
height int64 height int64
time time.Time
} }
func (ps peerState) GetHeight() int64 { var _ PeerState = (*testPeerState)(nil)
func (ps testPeerState) GetHeight() int64 {
return ps.height return ps.height
} }
func (ps testPeerState) GetLastBlockTime() time.Time {
return ps.time
}
func TestReactorSelectiveBroadcast(t *testing.T) { func TestReactorSelectiveBroadcast(t *testing.T) {
config := cfg.TestConfig() config := cfg.TestConfig()
@ -165,7 +173,7 @@ func TestReactorSelectiveBroadcast(t *testing.T) {
// make reactors from statedb // make reactors from statedb
reactors := makeAndConnectEvidenceReactors(config, []dbm.DB{stateDB1, stateDB2}) reactors := makeAndConnectEvidenceReactors(config, []dbm.DB{stateDB1, stateDB2})
peer := reactors[0].Switch.Peers().List()[0] peer := reactors[0].Switch.Peers().List()[0]
ps := peerState{height2} ps := testPeerState{height2, tmtime.Now()}
peer.Set(types.PeerStateKey, ps) peer.Set(types.PeerStateKey, ps)
// send a bunch of valid evidence to the first reactor's evpool // send a bunch of valid evidence to the first reactor's evpool

View File

@ -4,6 +4,7 @@ import (
"bytes" "bytes"
"fmt" "fmt"
"testing" "testing"
"time"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
@ -386,7 +387,7 @@ func TestConsensusParamsChangesSaveLoad(t *testing.T) {
} }
} }
func makeParams(blockBytes, blockGas, evidenceAge int64) types.ConsensusParams { func makeParams(blockBytes, blockGas int64, evidenceAge time.Duration) types.ConsensusParams {
return types.ConsensusParams{ return types.ConsensusParams{
BlockSize: types.BlockSize{ BlockSize: types.BlockSize{
MaxBytes: blockBytes, MaxBytes: blockBytes,
@ -403,7 +404,8 @@ func pk() []byte {
} }
func TestApplyUpdates(t *testing.T) { func TestApplyUpdates(t *testing.T) {
initParams := makeParams(1, 2, 3) initParams := makeParams(1, 2, 3*time.Second)
newMaxAge := 66 * time.Second
cases := [...]struct { cases := [...]struct {
init types.ConsensusParams init types.ConsensusParams
@ -419,14 +421,14 @@ func TestApplyUpdates(t *testing.T) {
MaxGas: 55, MaxGas: 55,
}, },
}, },
makeParams(44, 55, 3)}, makeParams(44, 55, 3*time.Second)},
3: {initParams, 3: {initParams,
abci.ConsensusParams{ abci.ConsensusParams{
EvidenceParams: &abci.EvidenceParams{ EvidenceParams: &abci.EvidenceParams{
MaxAge: 66, MaxAge: &newMaxAge,
}, },
}, },
makeParams(1, 2, 66)}, makeParams(1, 2, 66*time.Second)},
} }
for i, tc := range cases { for i, tc := range cases {

View File

@ -168,13 +168,11 @@ func validateBlock(stateDB dbm.DB, state State, block *types.Block) error {
// - it is internally consistent // - it is internally consistent
// - it was properly signed by the alleged equivocator // - it was properly signed by the alleged equivocator
func VerifyEvidence(stateDB dbm.DB, state State, evidence types.Evidence) error { func VerifyEvidence(stateDB dbm.DB, state State, evidence types.Evidence) error {
height := state.LastBlockHeight evidenceAge := state.LastBlockTime.Sub(evidence.Time())
evidenceAge := height - evidence.Height()
maxAge := state.ConsensusParams.EvidenceParams.MaxAge maxAge := state.ConsensusParams.EvidenceParams.MaxAge
if evidenceAge > maxAge { if evidenceAge > maxAge {
return fmt.Errorf("Evidence from height %d is too old. Min height is %d", return fmt.Errorf("Evidence from %v is too old. Expecting evidence no older than %v",
evidence.Height(), height-maxAge) evidence.Time(), state.LastBlockTime.Add(-maxAge))
} }
valset, err := LoadValidators(stateDB, evidence.Height()) valset, err := LoadValidators(stateDB, evidence.Height())

View File

@ -3,8 +3,10 @@ package types
import ( import (
"bytes" "bytes"
"fmt" "fmt"
"time"
"github.com/tendermint/tendermint/crypto/tmhash" "github.com/tendermint/tendermint/crypto/tmhash"
tmtime "github.com/tendermint/tendermint/types/time"
amino "github.com/tendermint/go-amino" amino "github.com/tendermint/go-amino"
@ -54,6 +56,7 @@ func (err *ErrEvidenceOverflow) Error() string {
// Evidence represents any provable malicious activity by a validator // Evidence represents any provable malicious activity by a validator
type Evidence interface { type Evidence interface {
Height() int64 // height of the equivocation Height() int64 // height of the equivocation
Time() time.Time // when the evidence was created
Address() []byte // address of the equivocating validator Address() []byte // address of the equivocating validator
Bytes() []byte // bytes which compromise the evidence Bytes() []byte // bytes which compromise the evidence
Hash() []byte // hash of the evidence Hash() []byte // hash of the evidence
@ -102,6 +105,11 @@ func (dve *DuplicateVoteEvidence) Height() int64 {
return dve.VoteA.Height return dve.VoteA.Height
} }
// Time returns the time when the evidence was created.
func (dve *DuplicateVoteEvidence) Time() time.Time {
return dve.VoteA.Timestamp
}
// Address returns the address of the validator. // Address returns the address of the validator.
func (dve *DuplicateVoteEvidence) Address() []byte { func (dve *DuplicateVoteEvidence) Address() []byte {
return dve.PubKey.Address() return dve.PubKey.Address()
@ -188,6 +196,7 @@ func NewMockGoodEvidence(height int64, idx int, address []byte) MockGoodEvidence
} }
func (e MockGoodEvidence) Height() int64 { return e.Height_ } func (e MockGoodEvidence) Height() int64 { return e.Height_ }
func (e MockGoodEvidence) Time() time.Time { return tmtime.Now() }
func (e MockGoodEvidence) Address() []byte { return e.Address_ } func (e MockGoodEvidence) Address() []byte { return e.Address_ }
func (e MockGoodEvidence) Hash() []byte { func (e MockGoodEvidence) Hash() []byte {
return []byte(fmt.Sprintf("%d-%x", e.Height_, e.Address_)) return []byte(fmt.Sprintf("%d-%x", e.Height_, e.Address_))

View File

@ -1,6 +1,8 @@
package types package types
import ( import (
"time"
abci "github.com/tendermint/tendermint/abci/types" abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto/tmhash" "github.com/tendermint/tendermint/crypto/tmhash"
cmn "github.com/tendermint/tendermint/libs/common" cmn "github.com/tendermint/tendermint/libs/common"
@ -29,7 +31,7 @@ type BlockSize struct {
// EvidenceParams determine how we handle evidence of malfeasance // EvidenceParams determine how we handle evidence of malfeasance
type EvidenceParams struct { type EvidenceParams struct {
MaxAge int64 `json:"max_age"` // only accept new evidence more recent than this MaxAge time.Duration `json:"max_age"` // only accept new evidence more recent than this
} }
// DefaultConsensusParams returns a default ConsensusParams. // DefaultConsensusParams returns a default ConsensusParams.
@ -51,7 +53,7 @@ func DefaultBlockSize() BlockSize {
// DefaultEvidenceParams Params returns a default EvidenceParams. // DefaultEvidenceParams Params returns a default EvidenceParams.
func DefaultEvidenceParams() EvidenceParams { func DefaultEvidenceParams() EvidenceParams {
return EvidenceParams{ return EvidenceParams{
MaxAge: 100000, // 27.8 hrs at 1block/s MaxAge: 48 * time.Hour,
} }
} }
@ -109,7 +111,9 @@ func (params ConsensusParams) Update(params2 *abci.ConsensusParams) ConsensusPar
res.BlockSize.MaxGas = params2.BlockSize.MaxGas res.BlockSize.MaxGas = params2.BlockSize.MaxGas
} }
if params2.EvidenceParams != nil { if params2.EvidenceParams != nil {
res.EvidenceParams.MaxAge = params2.EvidenceParams.MaxAge if params2.EvidenceParams.MaxAge != nil {
res.EvidenceParams.MaxAge = *params2.EvidenceParams.MaxAge
}
} }
return res return res
} }

View File

@ -4,6 +4,7 @@ import (
"bytes" "bytes"
"sort" "sort"
"testing" "testing"
"time"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
abci "github.com/tendermint/tendermint/abci/types" abci "github.com/tendermint/tendermint/abci/types"
@ -15,17 +16,17 @@ func TestConsensusParamsValidation(t *testing.T) {
valid bool valid bool
}{ }{
// test block size // test block size
0: {makeParams(1, 0, 1), true}, 0: {makeParams(1, 0, 10*time.Second), true},
1: {makeParams(0, 0, 1), false}, 1: {makeParams(0, 0, 10*time.Second), false},
2: {makeParams(47*1024*1024, 0, 1), true}, 2: {makeParams(47*1024*1024, 0, 10*time.Second), true},
3: {makeParams(10, 0, 1), true}, 3: {makeParams(10, 0, 10*time.Second), true},
4: {makeParams(100*1024*1024, 0, 1), true}, 4: {makeParams(100*1024*1024, 0, 10*time.Second), true},
5: {makeParams(101*1024*1024, 0, 1), false}, 5: {makeParams(101*1024*1024, 0, 10*time.Second), false},
6: {makeParams(1024*1024*1024, 0, 1), false}, 6: {makeParams(1024*1024*1024, 0, 10*time.Second), false},
7: {makeParams(1024*1024*1024, 0, -1), false}, 7: {makeParams(1024*1024*1024, 0, -10*time.Second), false},
// test evidence age // test evidence age
8: {makeParams(1, 0, 0), false}, 8: {makeParams(1, 0, 0), false},
9: {makeParams(1, 0, -1), false}, 9: {makeParams(1, 0, -1*time.Millisecond), false},
} }
for i, tc := range testCases { for i, tc := range testCases {
if tc.valid { if tc.valid {
@ -36,7 +37,7 @@ func TestConsensusParamsValidation(t *testing.T) {
} }
} }
func makeParams(blockBytes, blockGas, evidenceAge int64) ConsensusParams { func makeParams(blockBytes, blockGas int64, evidenceAge time.Duration) ConsensusParams {
return ConsensusParams{ return ConsensusParams{
BlockSize: BlockSize{ BlockSize: BlockSize{
MaxBytes: blockBytes, MaxBytes: blockBytes,
@ -50,14 +51,9 @@ func makeParams(blockBytes, blockGas, evidenceAge int64) ConsensusParams {
func TestConsensusParamsHash(t *testing.T) { func TestConsensusParamsHash(t *testing.T) {
params := []ConsensusParams{ params := []ConsensusParams{
makeParams(4, 2, 3), makeParams(4, 2, 3*time.Second),
makeParams(1, 4, 3), makeParams(1, 4, 3*time.Second),
makeParams(1, 2, 4), makeParams(1, 2, 4*time.Second),
makeParams(2, 5, 7),
makeParams(1, 7, 6),
makeParams(9, 5, 4),
makeParams(7, 8, 9),
makeParams(4, 6, 5),
} }
hashes := make([][]byte, len(params)) hashes := make([][]byte, len(params))
@ -76,6 +72,7 @@ func TestConsensusParamsHash(t *testing.T) {
} }
func TestConsensusParamsUpdate(t *testing.T) { func TestConsensusParamsUpdate(t *testing.T) {
newMaxAge := 300 * time.Second
testCases := []struct { testCases := []struct {
params ConsensusParams params ConsensusParams
updates *abci.ConsensusParams updates *abci.ConsensusParams
@ -83,23 +80,23 @@ func TestConsensusParamsUpdate(t *testing.T) {
}{ }{
// empty updates // empty updates
{ {
makeParams(1, 2, 3), makeParams(1, 2, 3*time.Second),
&abci.ConsensusParams{}, &abci.ConsensusParams{},
makeParams(1, 2, 3), makeParams(1, 2, 3*time.Second),
}, },
// fine updates // fine updates
{ {
makeParams(1, 2, 3), makeParams(1, 2, 3*time.Second),
&abci.ConsensusParams{ &abci.ConsensusParams{
BlockSize: &abci.BlockSize{ BlockSize: &abci.BlockSize{
MaxBytes: 100, MaxBytes: 100,
MaxGas: 200, MaxGas: 200,
}, },
EvidenceParams: &abci.EvidenceParams{ EvidenceParams: &abci.EvidenceParams{
MaxAge: 300, MaxAge: &newMaxAge,
}, },
}, },
makeParams(100, 200, 300), makeParams(100, 200, 300*time.Second),
}, },
} }
for _, tc := range testCases { for _, tc := range testCases {

View File

@ -119,7 +119,7 @@ func (tm2pb) ConsensusParams(params *ConsensusParams) *abci.ConsensusParams {
MaxGas: params.BlockSize.MaxGas, MaxGas: params.BlockSize.MaxGas,
}, },
EvidenceParams: &abci.EvidenceParams{ EvidenceParams: &abci.EvidenceParams{
MaxAge: params.EvidenceParams.MaxAge, MaxAge: &params.EvidenceParams.MaxAge,
}, },
} }
} }
@ -209,13 +209,14 @@ func (pb2tm) ValidatorUpdates(vals []abci.ValidatorUpdate) ([]*Validator, error)
} }
func (pb2tm) ConsensusParams(csp *abci.ConsensusParams) ConsensusParams { func (pb2tm) ConsensusParams(csp *abci.ConsensusParams) ConsensusParams {
return ConsensusParams{ params := ConsensusParams{
BlockSize: BlockSize{ BlockSize: BlockSize{
MaxBytes: csp.BlockSize.MaxBytes, MaxBytes: csp.BlockSize.MaxBytes,
MaxGas: csp.BlockSize.MaxGas, MaxGas: csp.BlockSize.MaxGas,
}, },
EvidenceParams: EvidenceParams{
MaxAge: csp.EvidenceParams.MaxAge,
},
} }
if csp.EvidenceParams.MaxAge != nil {
params.EvidenceParams.MaxAge = *csp.EvidenceParams.MaxAge
}
return params
} }