Browse Source

Bucky/fix evidence halt (#34)

* consensus: createProposalBlock function

* blockExecutor.CreateProposalBlock

- factored out of consensus pkg into a method on blockExec
- new private interfaces for mempool ("txNotifier") and evpool with one function each
- consensus tests still require more mempool methods

* failing test for CreateProposalBlock

* Fix bug in include evidece into block

* evidence: change maxBytes to maxSize

* MaxEvidencePerBlock

- changed to return both the max number and the max bytes
- preparation for #2590

* changelog

* fix linter

* Fix from review

Co-Authored-By: ebuchman <ethan@coinculture.info>
release/v0.28.1
Ethan Buchman 5 years ago
committed by GitHub
parent
commit
8fd8f800d0
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 254 additions and 65 deletions
  1. +2
    -1
      CHANGELOG_PENDING.md
  2. +12
    -6
      consensus/mempool_test.go
  3. +2
    -2
      consensus/reactor_test.go
  4. +1
    -1
      consensus/replay_file.go
  5. +1
    -1
      consensus/replay_test.go
  6. +27
    -22
      consensus/state.go
  7. +4
    -4
      evidence/pool.go
  8. +10
    -10
      evidence/store.go
  9. +5
    -0
      evidence/wire.go
  10. +111
    -0
      node/node_test.go
  11. +27
    -1
      state/execution.go
  12. +5
    -4
      state/validation.go
  13. +3
    -4
      state/validation_test.go
  14. +3
    -2
      types/block.go
  15. +36
    -7
      types/evidence.go
  16. +5
    -0
      types/wire.go

+ 2
- 1
CHANGELOG_PENDING.md View File

@ -1,4 +1,4 @@
## v0.29.0
## v0.28.1
*TBD*
@ -21,3 +21,4 @@ Special thanks to external contributors on this release:
### IMPROVEMENTS:
### BUG FIXES:
- [consensus] \#? Fix consensus halt from proposing blocks with too much evidence

+ 12
- 6
consensus/mempool_test.go View File

@ -10,6 +10,7 @@ import (
"github.com/tendermint/tendermint/abci/example/code"
abci "github.com/tendermint/tendermint/abci/types"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
)
@ -17,12 +18,17 @@ func init() {
config = ResetConfig("consensus_mempool_test")
}
// for testing
func assertMempool(txn txNotifier) sm.Mempool {
return txn.(sm.Mempool)
}
func TestMempoolNoProgressUntilTxsAvailable(t *testing.T) {
config := ResetConfig("consensus_mempool_txs_available_test")
config.Consensus.CreateEmptyBlocks = false
state, privVals := randGenesisState(1, false, 10)
cs := newConsensusStateWithConfig(config, state, privVals[0], NewCounterApplication())
cs.mempool.EnableTxsAvailable()
assertMempool(cs.txNotifier).EnableTxsAvailable()
height, round := cs.Height, cs.Round
newBlockCh := subscribe(cs.eventBus, types.EventQueryNewBlock)
startTestRound(cs, height, round)
@ -40,7 +46,7 @@ func TestMempoolProgressAfterCreateEmptyBlocksInterval(t *testing.T) {
config.Consensus.CreateEmptyBlocksInterval = ensureTimeout
state, privVals := randGenesisState(1, false, 10)
cs := newConsensusStateWithConfig(config, state, privVals[0], NewCounterApplication())
cs.mempool.EnableTxsAvailable()
assertMempool(cs.txNotifier).EnableTxsAvailable()
height, round := cs.Height, cs.Round
newBlockCh := subscribe(cs.eventBus, types.EventQueryNewBlock)
startTestRound(cs, height, round)
@ -55,7 +61,7 @@ func TestMempoolProgressInHigherRound(t *testing.T) {
config.Consensus.CreateEmptyBlocks = false
state, privVals := randGenesisState(1, false, 10)
cs := newConsensusStateWithConfig(config, state, privVals[0], NewCounterApplication())
cs.mempool.EnableTxsAvailable()
assertMempool(cs.txNotifier).EnableTxsAvailable()
height, round := cs.Height, cs.Round
newBlockCh := subscribe(cs.eventBus, types.EventQueryNewBlock)
newRoundCh := subscribe(cs.eventBus, types.EventQueryNewRound)
@ -91,7 +97,7 @@ func deliverTxsRange(cs *ConsensusState, start, end int) {
for i := start; i < end; i++ {
txBytes := make([]byte, 8)
binary.BigEndian.PutUint64(txBytes, uint64(i))
err := cs.mempool.CheckTx(txBytes, nil)
err := assertMempool(cs.txNotifier).CheckTx(txBytes, nil)
if err != nil {
panic(fmt.Sprintf("Error after CheckTx: %v", err))
}
@ -141,7 +147,7 @@ func TestMempoolRmBadTx(t *testing.T) {
// Try to send the tx through the mempool.
// CheckTx should not err, but the app should return a bad abci code
// and the tx should get removed from the pool
err := cs.mempool.CheckTx(txBytes, func(r *abci.Response) {
err := assertMempool(cs.txNotifier).CheckTx(txBytes, func(r *abci.Response) {
if r.GetCheckTx().Code != code.CodeTypeBadNonce {
t.Fatalf("expected checktx to return bad nonce, got %v", r)
}
@ -153,7 +159,7 @@ func TestMempoolRmBadTx(t *testing.T) {
// check for the tx
for {
txs := cs.mempool.ReapMaxBytesMaxGas(int64(len(txBytes)), -1)
txs := assertMempool(cs.txNotifier).ReapMaxBytesMaxGas(int64(len(txBytes)), -1)
if len(txs) == 0 {
emptyMempoolCh <- struct{}{}
return


+ 2
- 2
consensus/reactor_test.go View File

@ -225,7 +225,7 @@ func TestReactorCreatesBlockWhenEmptyBlocksFalse(t *testing.T) {
defer stopConsensusNet(log.TestingLogger(), reactors, eventBuses)
// send a tx
if err := css[3].mempool.CheckTx([]byte{1, 2, 3}, nil); err != nil {
if err := assertMempool(css[3].txNotifier).CheckTx([]byte{1, 2, 3}, nil); err != nil {
//t.Fatal(err)
}
@ -448,7 +448,7 @@ func waitForAndValidateBlock(t *testing.T, n int, activeVals map[string]struct{}
err := validateBlock(newBlock, activeVals)
assert.Nil(t, err)
for _, tx := range txs {
err := css[j].mempool.CheckTx(tx, nil)
err := assertMempool(css[j].txNotifier).CheckTx(tx, nil)
assert.Nil(t, err)
}
}, css)


+ 1
- 1
consensus/replay_file.go View File

@ -137,7 +137,7 @@ func (pb *playback) replayReset(count int, newStepCh chan interface{}) error {
pb.cs.Wait()
newCS := NewConsensusState(pb.cs.config, pb.genesisState.Copy(), pb.cs.blockExec,
pb.cs.blockStore, pb.cs.mempool, pb.cs.evpool)
pb.cs.blockStore, pb.cs.txNotifier, pb.cs.evpool)
newCS.SetEventBus(pb.cs.eventBus)
newCS.startForReplay()


+ 1
- 1
consensus/replay_test.go View File

@ -87,7 +87,7 @@ func sendTxs(cs *ConsensusState, ctx context.Context) {
return
default:
tx := []byte{byte(i)}
cs.mempool.CheckTx(tx, nil)
assertMempool(cs.txNotifier).CheckTx(tx, nil)
i++
}
}


+ 27
- 22
consensus/state.go View File

@ -57,6 +57,16 @@ func (ti *timeoutInfo) String() string {
return fmt.Sprintf("%v ; %d/%d %v", ti.Duration, ti.Height, ti.Round, ti.Step)
}
// interface to the mempool
type txNotifier interface {
TxsAvailable() <-chan struct{}
}
// interface to the evidence pool
type evidencePool interface {
AddEvidence(types.Evidence) error
}
// ConsensusState handles execution of the consensus algorithm.
// It processes votes and proposals, and upon reaching agreement,
// commits blocks to the chain and executes them against the application.
@ -68,11 +78,18 @@ type ConsensusState struct {
config *cfg.ConsensusConfig
privValidator types.PrivValidator // for signing votes
// services for creating and executing blocks
blockExec *sm.BlockExecutor
// store blocks and commits
blockStore sm.BlockStore
mempool sm.Mempool
evpool sm.EvidencePool
// create and execute blocks
blockExec *sm.BlockExecutor
// notify us if txs are available
txNotifier txNotifier
// add evidence to the pool
// when it's detected
evpool evidencePool
// internal state
mtx sync.RWMutex
@ -128,15 +145,15 @@ func NewConsensusState(
state sm.State,
blockExec *sm.BlockExecutor,
blockStore sm.BlockStore,
mempool sm.Mempool,
evpool sm.EvidencePool,
txNotifier txNotifier,
evpool evidencePool,
options ...StateOption,
) *ConsensusState {
cs := &ConsensusState{
config: config,
blockExec: blockExec,
blockStore: blockStore,
mempool: mempool,
txNotifier: txNotifier,
peerMsgQueue: make(chan msgInfo, msgQueueSize),
internalMsgQueue: make(chan msgInfo, msgQueueSize),
timeoutTicker: NewTimeoutTicker(),
@ -484,7 +501,7 @@ func (cs *ConsensusState) updateToState(state sm.State) {
// If state isn't further out than cs.state, just ignore.
// This happens when SwitchToConsensus() is called in the reactor.
// We don't want to reset e.g. the Votes, but we still want to
// signal the new round step, because other services (eg. mempool)
// signal the new round step, because other services (eg. txNotifier)
// depend on having an up-to-date peer state!
if !cs.state.IsEmpty() && (state.LastBlockHeight <= cs.state.LastBlockHeight) {
cs.Logger.Info("Ignoring updateToState()", "newHeight", state.LastBlockHeight+1, "oldHeight", cs.state.LastBlockHeight+1)
@ -599,7 +616,7 @@ func (cs *ConsensusState) receiveRoutine(maxSteps int) {
var mi msgInfo
select {
case <-cs.mempool.TxsAvailable():
case <-cs.txNotifier.TxsAvailable():
cs.handleTxsAvailable()
case mi = <-cs.peerMsgQueue:
cs.wal.Write(mi)
@ -921,20 +938,8 @@ func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts
return
}
maxBytes := cs.state.ConsensusParams.BlockSize.MaxBytes
maxGas := cs.state.ConsensusParams.BlockSize.MaxGas
// bound evidence to 1/10th of the block
evidence := cs.evpool.PendingEvidence(types.MaxEvidenceBytesPerBlock(maxBytes))
// Mempool validated transactions
txs := cs.mempool.ReapMaxBytesMaxGas(types.MaxDataBytes(
maxBytes,
cs.state.Validators.Size(),
len(evidence),
), maxGas)
proposerAddr := cs.privValidator.GetPubKey().Address()
block, parts := cs.state.MakeBlock(cs.Height, txs, commit, evidence, proposerAddr)
return block, parts
return cs.blockExec.CreateProposalBlock(cs.Height, cs.state, commit, proposerAddr)
}
// Enter: `timeoutPropose` after entering Propose.


+ 4
- 4
evidence/pool.go View File

@ -57,10 +57,10 @@ func (evpool *EvidencePool) PriorityEvidence() []types.Evidence {
return evpool.evidenceStore.PriorityEvidence()
}
// PendingEvidence returns uncommitted evidence up to maxBytes.
// If maxBytes is -1, all evidence is returned.
func (evpool *EvidencePool) PendingEvidence(maxBytes int64) []types.Evidence {
return evpool.evidenceStore.PendingEvidence(maxBytes)
// PendingEvidence returns up to maxNum uncommitted evidence.
// If maxNum is -1, all evidence is returned.
func (evpool *EvidencePool) PendingEvidence(maxNum int64) []types.Evidence {
return evpool.evidenceStore.PendingEvidence(maxNum)
}
// State returns the current state of the evpool.


+ 10
- 10
evidence/store.go View File

@ -86,26 +86,26 @@ func (store *EvidenceStore) PriorityEvidence() (evidence []types.Evidence) {
return l
}
// PendingEvidence returns known uncommitted evidence up to maxBytes.
// If maxBytes is -1, all evidence is returned.
func (store *EvidenceStore) PendingEvidence(maxBytes int64) (evidence []types.Evidence) {
return store.listEvidence(baseKeyPending, maxBytes)
// PendingEvidence returns up to maxNum known, uncommitted evidence.
// If maxNum is -1, all evidence is returned.
func (store *EvidenceStore) PendingEvidence(maxNum int64) (evidence []types.Evidence) {
return store.listEvidence(baseKeyPending, maxNum)
}
// listEvidence lists the evidence for the given prefix key up to maxBytes.
// listEvidence lists up to maxNum pieces of evidence for the given prefix key.
// It is wrapped by PriorityEvidence and PendingEvidence for convenience.
// If maxBytes is -1, there's no cap on the size of returned evidence.
func (store *EvidenceStore) listEvidence(prefixKey string, maxBytes int64) (evidence []types.Evidence) {
var bytes int64
// If maxNum is -1, there's no cap on the size of returned evidence.
func (store *EvidenceStore) listEvidence(prefixKey string, maxNum int64) (evidence []types.Evidence) {
var count int64
iter := dbm.IteratePrefix(store.db, []byte(prefixKey))
defer iter.Close()
for ; iter.Valid(); iter.Next() {
val := iter.Value()
if maxBytes > 0 && bytes+int64(len(val)) > maxBytes {
if count == maxNum {
return evidence
}
bytes += int64(len(val))
count++
var ei EvidenceInfo
err := cdc.UnmarshalBinaryBare(val, &ei)


+ 5
- 0
evidence/wire.go View File

@ -13,3 +13,8 @@ func init() {
cryptoAmino.RegisterAmino(cdc)
types.RegisterEvidences(cdc)
}
// For testing purposes only
func RegisterMockEvidences() {
types.RegisterMockEvidences(cdc)
}

+ 111
- 0
node/node_test.go View File

@ -15,10 +15,14 @@ import (
"github.com/tendermint/tendermint/abci/example/kvstore"
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/crypto/ed25519"
"github.com/tendermint/tendermint/evidence"
cmn "github.com/tendermint/tendermint/libs/common"
dbm "github.com/tendermint/tendermint/libs/db"
"github.com/tendermint/tendermint/libs/log"
mempl "github.com/tendermint/tendermint/mempool"
"github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/privval"
"github.com/tendermint/tendermint/proxy"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
tmtime "github.com/tendermint/tendermint/types/time"
@ -192,3 +196,110 @@ func testFreeAddr(t *testing.T) string {
return fmt.Sprintf("127.0.0.1:%d", ln.Addr().(*net.TCPAddr).Port)
}
// create a proposal block using real and full
// mempool and evidence pool and validate it.
func TestCreateProposalBlock(t *testing.T) {
config := cfg.ResetTestRoot("node_create_proposal")
cc := proxy.NewLocalClientCreator(kvstore.NewKVStoreApplication())
proxyApp := proxy.NewAppConns(cc)
err := proxyApp.Start()
require.Nil(t, err)
defer proxyApp.Stop()
logger := log.TestingLogger()
var height int64 = 1
state, stateDB := state(1, height)
maxBytes := 16384
state.ConsensusParams.BlockSize.MaxBytes = int64(maxBytes)
proposerAddr, _ := state.Validators.GetByIndex(0)
// Make Mempool
memplMetrics := mempl.PrometheusMetrics("node_test")
mempool := mempl.NewMempool(
config.Mempool,
proxyApp.Mempool(),
state.LastBlockHeight,
mempl.WithMetrics(memplMetrics),
mempl.WithPreCheck(sm.TxPreCheck(state)),
mempl.WithPostCheck(sm.TxPostCheck(state)),
)
mempool.SetLogger(logger)
// Make EvidencePool
types.RegisterMockEvidencesGlobal()
evidence.RegisterMockEvidences()
evidenceDB := dbm.NewMemDB()
evidenceStore := evidence.NewEvidenceStore(evidenceDB)
evidencePool := evidence.NewEvidencePool(stateDB, evidenceStore)
evidencePool.SetLogger(logger)
// fill the evidence pool with more evidence
// than can fit in a block
minEvSize := 12
numEv := (maxBytes / types.MaxEvidenceBytesDenominator) / minEvSize
for i := 0; i < numEv; i++ {
ev := types.NewMockRandomGoodEvidence(1, proposerAddr, cmn.RandBytes(minEvSize))
err := evidencePool.AddEvidence(ev)
assert.NoError(t, err)
}
// fill the mempool with more txs
// than can fit in a block
txLength := 1000
for i := 0; i < maxBytes/txLength; i++ {
tx := cmn.RandBytes(txLength)
err := mempool.CheckTx(tx, nil)
assert.NoError(t, err)
}
blockExec := sm.NewBlockExecutor(
stateDB,
logger,
proxyApp.Consensus(),
mempool,
evidencePool,
)
commit := &types.Commit{}
block, _ := blockExec.CreateProposalBlock(
height,
state, commit,
proposerAddr,
)
err = blockExec.ValidateBlock(state, block)
assert.NoError(t, err)
}
func state(nVals int, height int64) (sm.State, dbm.DB) {
vals := make([]types.GenesisValidator, nVals)
for i := 0; i < nVals; i++ {
secret := []byte(fmt.Sprintf("test%d", i))
pk := ed25519.GenPrivKeyFromSecret(secret)
vals[i] = types.GenesisValidator{
pk.PubKey().Address(),
pk.PubKey(),
1000,
fmt.Sprintf("test%d", i),
}
}
s, _ := sm.MakeGenesisState(&types.GenesisDoc{
ChainID: "test-chain",
Validators: vals,
AppHash: nil,
})
// save validators to db for 2 heights
stateDB := dbm.NewMemDB()
sm.SaveState(stateDB, s)
for i := 1; i < int(height); i++ {
s.LastBlockHeight++
s.LastValidators = s.Validators.Copy()
sm.SaveState(stateDB, s)
}
return s, stateDB
}

+ 27
- 1
state/execution.go View File

@ -29,7 +29,8 @@ type BlockExecutor struct {
// events
eventBus types.BlockEventPublisher
// update these with block results after commit
// manage the mempool lock during commit
// and update both with block results after commit.
mempool Mempool
evpool EvidencePool
@ -73,6 +74,31 @@ func (blockExec *BlockExecutor) SetEventBus(eventBus types.BlockEventPublisher)
blockExec.eventBus = eventBus
}
// CreateProposalBlock calls state.MakeBlock with evidence from the evpool
// and txs from the mempool. The max bytes must be big enough to fit the commit.
// Up to 1/10th of the block space is allcoated for maximum sized evidence.
// The rest is given to txs, up to the max gas.
func (blockExec *BlockExecutor) CreateProposalBlock(
height int64,
state State, commit *types.Commit,
proposerAddr []byte,
) (*types.Block, *types.PartSet) {
maxBytes := state.ConsensusParams.BlockSize.MaxBytes
maxGas := state.ConsensusParams.BlockSize.MaxGas
// Fetch a limited amount of valid evidence
maxNumEvidence, _ := types.MaxEvidencePerBlock(maxBytes)
evidence := blockExec.evpool.PendingEvidence(maxNumEvidence)
// Fetch a limited amount of valid txs
maxDataBytes := types.MaxDataBytes(maxBytes, state.Validators.Size(), len(evidence))
txs := blockExec.mempool.ReapMaxBytesMaxGas(maxDataBytes, maxGas)
return state.MakeBlock(height, txs, commit, evidence, proposerAddr)
}
// ValidateBlock validates the given block against the given state.
// If the block is invalid, it returns an error.
// Validation does not mutate state, but does require historical information from the stateDB,


+ 5
- 4
state/validation.go View File

@ -133,10 +133,11 @@ func validateBlock(stateDB dbm.DB, state State, block *types.Block) error {
}
// Limit the amount of evidence
maxEvidenceBytes := types.MaxEvidenceBytesPerBlock(state.ConsensusParams.BlockSize.MaxBytes)
evidenceBytes := int64(len(block.Evidence.Evidence)) * types.MaxEvidenceBytes
if evidenceBytes > maxEvidenceBytes {
return types.NewErrEvidenceOverflow(maxEvidenceBytes, evidenceBytes)
maxNumEvidence, _ := types.MaxEvidencePerBlock(state.ConsensusParams.BlockSize.MaxBytes)
numEvidence := int64(len(block.Evidence.Evidence))
if numEvidence > maxNumEvidence {
return types.NewErrEvidenceOverflow(maxNumEvidence, numEvidence)
}
// Validate all evidence.


+ 3
- 4
state/validation_test.go View File

@ -109,10 +109,9 @@ func TestValidateBlockEvidence(t *testing.T) {
// A block with too much evidence fails.
maxBlockSize := state.ConsensusParams.BlockSize.MaxBytes
maxEvidenceBytes := types.MaxEvidenceBytesPerBlock(maxBlockSize)
maxEvidence := maxEvidenceBytes / types.MaxEvidenceBytes
require.True(t, maxEvidence > 2)
for i := int64(0); i < maxEvidence; i++ {
maxNumEvidence, _ := types.MaxEvidencePerBlock(maxBlockSize)
require.True(t, maxNumEvidence > 2)
for i := int64(0); i < maxNumEvidence; i++ {
block.Evidence.Evidence = append(block.Evidence.Evidence, goodEvidence)
}
block.EvidenceHash = block.Evidence.Hash()


+ 3
- 2
types/block.go View File

@ -322,16 +322,17 @@ func MaxDataBytes(maxBytes int64, valsCount, evidenceCount int) int64 {
}
// MaxDataBytesUnknownEvidence returns the maximum size of block's data when
// evidence count is unknown. MaxEvidenceBytesPerBlock will be used as the size
// evidence count is unknown. MaxEvidencePerBlock will be used for the size
// of evidence.
//
// XXX: Panics on negative result.
func MaxDataBytesUnknownEvidence(maxBytes int64, valsCount int) int64 {
_, maxEvidenceBytes := MaxEvidencePerBlock(maxBytes)
maxDataBytes := maxBytes -
MaxAminoOverheadForBlock -
MaxHeaderBytes -
int64(valsCount)*MaxVoteBytes -
MaxEvidenceBytesPerBlock(maxBytes)
maxEvidenceBytes
if maxDataBytes < 0 {
panic(fmt.Sprintf(


+ 36
- 7
types/evidence.go View File

@ -36,8 +36,8 @@ func (err *ErrEvidenceInvalid) Error() string {
// ErrEvidenceOverflow is for when there is too much evidence in a block.
type ErrEvidenceOverflow struct {
MaxBytes int64
GotBytes int64
MaxNum int64
GotNum int64
}
// NewErrEvidenceOverflow returns a new ErrEvidenceOverflow where got > max.
@ -47,7 +47,7 @@ func NewErrEvidenceOverflow(max, got int64) *ErrEvidenceOverflow {
// Error returns a string representation of the error.
func (err *ErrEvidenceOverflow) Error() string {
return fmt.Sprintf("Too much evidence: Max %d bytes, got %d bytes", err.MaxBytes, err.GotBytes)
return fmt.Sprintf("Too much evidence: Max %d, got %d", err.MaxNum, err.GotNum)
}
//-------------------------------------------
@ -72,13 +72,23 @@ func RegisterEvidences(cdc *amino.Codec) {
func RegisterMockEvidences(cdc *amino.Codec) {
cdc.RegisterConcrete(MockGoodEvidence{}, "tendermint/MockGoodEvidence", nil)
cdc.RegisterConcrete(MockRandomGoodEvidence{}, "tendermint/MockRandomGoodEvidence", nil)
cdc.RegisterConcrete(MockBadEvidence{}, "tendermint/MockBadEvidence", nil)
}
// MaxEvidenceBytesPerBlock returns the maximum evidence size per block -
// 1/10th of the maximum block size.
func MaxEvidenceBytesPerBlock(blockMaxBytes int64) int64 {
return blockMaxBytes / 10
const (
MaxEvidenceBytesDenominator = 10
)
// MaxEvidencePerBlock returns the maximum number of evidences
// allowed in the block and their maximum total size (limitted to 1/10th
// of the maximum block size).
// TODO: change to a constant, or to a fraction of the validator set size.
// See https://github.com/tendermint/tendermint/issues/2590
func MaxEvidencePerBlock(blockMaxBytes int64) (int64, int64) {
maxBytes := blockMaxBytes / MaxEvidenceBytesDenominator
maxNum := maxBytes / MaxEvidenceBytes
return maxNum, maxBytes
}
//-------------------------------------------
@ -193,6 +203,25 @@ func (dve *DuplicateVoteEvidence) ValidateBasic() error {
//-----------------------------------------------------------------
// UNSTABLE
type MockRandomGoodEvidence struct {
MockGoodEvidence
randBytes []byte
}
var _ Evidence = &MockRandomGoodEvidence{}
// UNSTABLE
func NewMockRandomGoodEvidence(height int64, address []byte, randBytes []byte) MockRandomGoodEvidence {
return MockRandomGoodEvidence{
MockGoodEvidence{height, address}, randBytes,
}
}
func (e MockRandomGoodEvidence) Hash() []byte {
return []byte(fmt.Sprintf("%d-%x", e.Height_, e.randBytes))
}
// UNSTABLE
type MockGoodEvidence struct {
Height_ int64


+ 5
- 0
types/wire.go View File

@ -20,3 +20,8 @@ func RegisterBlockAmino(cdc *amino.Codec) {
func GetCodec() *amino.Codec {
return cdc
}
// For testing purposes only
func RegisterMockEvidencesGlobal() {
RegisterMockEvidences(cdc)
}

Loading…
Cancel
Save