Browse Source

abci++: synchronize PrepareProposal with the newest version of the spec (#8094)

This change implements the logic for the PrepareProposal ABCI++ method call. The main logic for creating and issuing the PrepareProposal request lives in execution.go and is tested in a set of new tests in execution_test.go. This change also updates the mempool mock to use a mockery generated version and removes much of the plumbing for the no longer used ABCIResponses.
pull/8128/head
William Banfield 2 years ago
committed by GitHub
parent
commit
68c624f5de
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
36 changed files with 2978 additions and 917 deletions
  1. +23
    -9
      abci/example/kvstore/kvstore.go
  2. +28
    -0
      abci/types/types.go
  3. +1565
    -391
      abci/types/types.pb.go
  4. +74
    -0
      abci/types/types_test.go
  5. +15
    -4
      internal/blocksync/reactor_test.go
  6. +4
    -5
      internal/consensus/byzantine_test.go
  7. +3
    -1
      internal/consensus/common_test.go
  8. +1
    -1
      internal/consensus/mempool_test.go
  9. +1
    -1
      internal/consensus/pbts_test.go
  10. +4
    -4
      internal/consensus/replay_test.go
  11. +15
    -9
      internal/consensus/state.go
  12. +4
    -4
      internal/consensus/state_test.go
  13. +1
    -4
      internal/evidence/pool_test.go
  14. +0
    -46
      internal/mempool/mock/mempool.go
  15. +172
    -0
      internal/mempool/mocks/mempool.go
  16. +2
    -0
      internal/mempool/types.go
  17. +5
    -8
      internal/rpc/core/tx.go
  18. +55
    -16
      internal/state/execution.go
  19. +357
    -22
      internal/state/execution_test.go
  20. +6
    -9
      internal/state/helpers_test.go
  21. +2
    -7
      internal/state/state.go
  22. +125
    -103
      internal/state/state_test.go
  23. +0
    -8
      internal/state/store.go
  24. +0
    -23
      internal/state/store_test.go
  25. +4
    -8
      internal/state/test/factory/block.go
  26. +47
    -17
      internal/state/validation_test.go
  27. +6
    -14
      internal/store/store_test.go
  28. +7
    -6
      light/rpc/client.go
  29. +12
    -4
      node/node_test.go
  30. +78
    -51
      proto/tendermint/abci/types.proto.intermediate
  31. +2
    -1
      test/e2e/app/app.go
  32. +0
    -54
      types/results.go
  33. +0
    -54
      types/results_test.go
  34. +197
    -27
      types/tx.go
  35. +160
    -6
      types/tx_test.go
  36. +3
    -0
      types/vote_set.go

+ 23
- 9
abci/example/kvstore/kvstore.go View File

@ -284,7 +284,7 @@ func (app *Application) PrepareProposal(req types.RequestPrepareProposal) types.
app.mu.Lock()
defer app.mu.Unlock()
return types.ResponsePrepareProposal{BlockData: app.substPrepareTx(req.BlockData)}
return types.ResponsePrepareProposal{TxRecords: app.substPrepareTx(req.Txs)}
}
func (*Application) ProcessProposal(req types.RequestProcessProposal) types.ResponseProcessProposal {
@ -420,7 +420,7 @@ func (app *Application) updateValidator(v types.ValidatorUpdate) *types.ExecTxRe
const PreparePrefix = "prepare"
func isPrepareTx(tx []byte) bool {
return strings.HasPrefix(string(tx), PreparePrefix)
return bytes.HasPrefix(tx, []byte(PreparePrefix))
}
// execPrepareTx is noop. tx data is considered as placeholder
@ -430,16 +430,30 @@ func (app *Application) execPrepareTx(tx []byte) *types.ExecTxResult {
return &types.ExecTxResult{}
}
// substPrepareTx subst all the preparetx in the blockdata
// to null string(could be any arbitrary string).
func (app *Application) substPrepareTx(blockData [][]byte) [][]byte {
// TODO: this mechanism will change with the current spec of PrepareProposal
// We now have a special type for marking a tx as changed
// substPrepareTx substitutes all the transactions prefixed with 'prepare' in the
// proposal for transactions with the prefix strips.
// It marks all of the original transactions as 'REMOVED' so that
// Tendermint will remove them from its mempool.
func (app *Application) substPrepareTx(blockData [][]byte) []*types.TxRecord {
trs := make([]*types.TxRecord, len(blockData))
var removed []*types.TxRecord
for i, tx := range blockData {
if isPrepareTx(tx) {
blockData[i] = make([]byte, len(tx))
removed = append(removed, &types.TxRecord{
Tx: tx,
Action: types.TxRecord_REMOVED,
})
trs[i] = &types.TxRecord{
Tx: bytes.TrimPrefix(tx, []byte(PreparePrefix)),
Action: types.TxRecord_ADDED,
}
continue
}
trs[i] = &types.TxRecord{
Tx: tx,
Action: types.TxRecord_UNMODIFIED,
}
}
return blockData
return append(trs, removed...)
}

abci/types/result.go → abci/types/types.go View File


+ 1565
- 391
abci/types/types.pb.go
File diff suppressed because it is too large
View File


+ 74
- 0
abci/types/types_test.go View File

@ -0,0 +1,74 @@
package types_test
import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto/merkle"
)
func TestHashAndProveResults(t *testing.T) {
trs := []*abci.ExecTxResult{
// Note, these tests rely on the first two entries being in this order.
{Code: 0, Data: nil},
{Code: 0, Data: []byte{}},
{Code: 0, Data: []byte("one")},
{Code: 14, Data: nil},
{Code: 14, Data: []byte("foo")},
{Code: 14, Data: []byte("bar")},
}
// Nil and []byte{} should produce the same bytes
bz0, err := trs[0].Marshal()
require.NoError(t, err)
bz1, err := trs[1].Marshal()
require.NoError(t, err)
require.Equal(t, bz0, bz1)
// Make sure that we can get a root hash from results and verify proofs.
rs, err := abci.MarshalTxResults(trs)
require.NoError(t, err)
root := merkle.HashFromByteSlices(rs)
assert.NotEmpty(t, root)
_, proofs := merkle.ProofsFromByteSlices(rs)
for i, tr := range trs {
bz, err := tr.Marshal()
require.NoError(t, err)
valid := proofs[i].Verify(root, bz)
assert.NoError(t, valid, "%d", i)
}
}
func TestHashDeterministicFieldsOnly(t *testing.T) {
tr1 := abci.ExecTxResult{
Code: 1,
Data: []byte("transaction"),
Log: "nondeterministic data: abc",
Info: "nondeterministic data: abc",
GasWanted: 1000,
GasUsed: 1000,
Events: []abci.Event{},
Codespace: "nondeterministic.data.abc",
}
tr2 := abci.ExecTxResult{
Code: 1,
Data: []byte("transaction"),
Log: "nondeterministic data: def",
Info: "nondeterministic data: def",
GasWanted: 1000,
GasUsed: 1000,
Events: []abci.Event{},
Codespace: "nondeterministic.data.def",
}
r1, err := abci.MarshalTxResults([]*abci.ExecTxResult{&tr1})
require.NoError(t, err)
r2, err := abci.MarshalTxResults([]*abci.ExecTxResult{&tr2})
require.NoError(t, err)
require.Equal(t, merkle.HashFromByteSlices(r1), merkle.HashFromByteSlices(r2))
}

+ 15
- 4
internal/blocksync/reactor_test.go View File

@ -7,6 +7,7 @@ import (
"time"
"github.com/fortytw2/leaktest"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
dbm "github.com/tendermint/tm-db"
@ -15,7 +16,7 @@ import (
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/internal/consensus"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/internal/mempool/mock"
mpmocks "github.com/tendermint/tendermint/internal/mempool/mocks"
"github.com/tendermint/tendermint/internal/p2p"
"github.com/tendermint/tendermint/internal/p2p/p2ptest"
"github.com/tendermint/tendermint/internal/proxy"
@ -121,6 +122,17 @@ func (rts *reactorTestSuite) addNode(
state, err := sm.MakeGenesisState(genDoc)
require.NoError(t, err)
require.NoError(t, stateStore.Save(state))
mp := &mpmocks.Mempool{}
mp.On("Lock").Return()
mp.On("Unlock").Return()
mp.On("FlushAppConn", mock.Anything).Return(nil)
mp.On("Update",
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything).Return(nil)
eventbus := eventbus.NewDefault(logger)
require.NoError(t, eventbus.Start(ctx))
@ -129,7 +141,7 @@ func (rts *reactorTestSuite) addNode(
stateStore,
log.TestingLogger(),
rts.app[nodeID],
mock.Mempool{},
mp,
sm.EmptyEvidencePool{},
blockStore,
eventbus,
@ -159,8 +171,7 @@ func (rts *reactorTestSuite) addNode(
)
}
thisBlock, err := sf.MakeBlock(state, blockHeight, lastCommit)
require.NoError(t, err)
thisBlock := sf.MakeBlock(state, blockHeight, lastCommit)
thisParts, err := thisBlock.MakePartSet(types.BlockPartSizeBytes)
require.NoError(t, err)
blockID := types.BlockID{Hash: thisBlock.Hash(), PartSetHeader: thisParts.Header()}


+ 4
- 5
internal/consensus/byzantine_test.go View File

@ -176,7 +176,6 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
require.NotNil(t, lazyNodeState.privValidator)
var commit *types.Commit
var votes []*types.Vote
switch {
case lazyNodeState.Height == lazyNodeState.state.InitialHeight:
// We're creating a proposal for the first block.
@ -185,7 +184,6 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
case lazyNodeState.LastCommit.HasTwoThirdsMajority():
// Make the commit from LastCommit
commit = lazyNodeState.LastCommit.MakeCommit()
votes = lazyNodeState.LastCommit.GetVotes()
default: // This shouldn't happen.
lazyNodeState.logger.Error("enterPropose: Cannot propose anything: No commit for the previous block")
return
@ -202,9 +200,10 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
}
proposerAddr := lazyNodeState.privValidatorPubKey.Address()
block, blockParts, err := lazyNodeState.blockExec.CreateProposalBlock(
ctx, lazyNodeState.Height, lazyNodeState.state, commit, proposerAddr, votes,
)
block, err := lazyNodeState.blockExec.CreateProposalBlock(
ctx, lazyNodeState.Height, lazyNodeState.state, commit, proposerAddr, nil)
require.NoError(t, err)
blockParts, err := block.MakePartSet(types.BlockPartSizeBytes)
require.NoError(t, err)
// Flush the WAL. Otherwise, we may not recompute the same proposal to sign,


+ 3
- 1
internal/consensus/common_test.go View File

@ -242,7 +242,9 @@ func decideProposal(
t.Helper()
cs1.mtx.Lock()
block, blockParts, err := cs1.createProposalBlock(ctx)
block, err := cs1.createProposalBlock(ctx)
require.NoError(t, err)
blockParts, err := block.MakePartSet(types.BlockPartSizeBytes)
require.NoError(t, err)
validRound := cs1.ValidRound
chainID := cs1.state.ChainID


+ 1
- 1
internal/consensus/mempool_test.go View File

@ -310,7 +310,7 @@ func (app *CounterApplication) Commit() abci.ResponseCommit {
func (app *CounterApplication) PrepareProposal(
req abci.RequestPrepareProposal) abci.ResponsePrepareProposal {
return abci.ResponsePrepareProposal{BlockData: req.BlockData}
return abci.ResponsePrepareProposal{}
}
func (app *CounterApplication) ProcessProposal(


+ 1
- 1
internal/consensus/pbts_test.go View File

@ -203,7 +203,7 @@ func (p *pbtsTestHarness) nextHeight(ctx context.Context, t *testing.T, proposer
ensureNewRound(t, p.roundCh, p.currentHeight, p.currentRound)
b, _, err := p.observedState.createProposalBlock(ctx)
b, err := p.observedState.createProposalBlock(ctx)
require.NoError(t, err)
b.Height = p.currentHeight
b.Header.Height = p.currentHeight


+ 4
- 4
internal/consensus/replay_test.go View File

@ -380,7 +380,7 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
newValidatorTx1 := kvstore.MakeValSetChangeTx(valPubKey1ABCI, testMinPower)
err = assertMempool(t, css[0].txNotifier).CheckTx(ctx, newValidatorTx1, nil, mempool.TxInfo{})
assert.NoError(t, err)
propBlock, _, err := css[0].createProposalBlock(ctx) // changeProposer(t, cs1, vs2)
propBlock, err := css[0].createProposalBlock(ctx) // changeProposer(t, cs1, vs2)
require.NoError(t, err)
propBlockParts, err := propBlock.MakePartSet(partSize)
require.NoError(t, err)
@ -414,7 +414,7 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
updateValidatorTx1 := kvstore.MakeValSetChangeTx(updatePubKey1ABCI, 25)
err = assertMempool(t, css[0].txNotifier).CheckTx(ctx, updateValidatorTx1, nil, mempool.TxInfo{})
assert.NoError(t, err)
propBlock, _, err = css[0].createProposalBlock(ctx) // changeProposer(t, cs1, vs2)
propBlock, err = css[0].createProposalBlock(ctx) // changeProposer(t, cs1, vs2)
require.NoError(t, err)
propBlockParts, err = propBlock.MakePartSet(partSize)
require.NoError(t, err)
@ -455,7 +455,7 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
newValidatorTx3 := kvstore.MakeValSetChangeTx(newVal3ABCI, testMinPower)
err = assertMempool(t, css[0].txNotifier).CheckTx(ctx, newValidatorTx3, nil, mempool.TxInfo{})
assert.NoError(t, err)
propBlock, _, err = css[0].createProposalBlock(ctx) // changeProposer(t, cs1, vs2)
propBlock, err = css[0].createProposalBlock(ctx) // changeProposer(t, cs1, vs2)
require.NoError(t, err)
propBlockParts, err = propBlock.MakePartSet(partSize)
require.NoError(t, err)
@ -543,7 +543,7 @@ func setupSimulator(ctx context.Context, t *testing.T) *simulatorTestSuite {
removeValidatorTx3 := kvstore.MakeValSetChangeTx(newVal3ABCI, 0)
err = assertMempool(t, css[0].txNotifier).CheckTx(ctx, removeValidatorTx3, nil, mempool.TxInfo{})
assert.NoError(t, err)
propBlock, _, err = css[0].createProposalBlock(ctx) // changeProposer(t, cs1, vs2)
propBlock, err = css[0].createProposalBlock(ctx) // changeProposer(t, cs1, vs2)
require.NoError(t, err)
propBlockParts, err = propBlock.MakePartSet(partSize)
require.NoError(t, err)


+ 15
- 9
internal/consensus/state.go View File

@ -1300,8 +1300,16 @@ func (cs *State) defaultDecideProposal(ctx context.Context, height int64, round
} else {
// Create a new proposal block from state/txs from the mempool.
var err error
block, blockParts, err = cs.createProposalBlock(ctx)
if block == nil || err != nil {
block, err = cs.createProposalBlock(ctx)
if err != nil {
cs.logger.Error("unable to create proposal block", "error", err)
return
} else if block == nil {
return
}
blockParts, err = block.MakePartSet(types.BlockPartSizeBytes)
if err != nil {
cs.logger.Error("unable to create proposal block part set", "error", err)
return
}
}
@ -1360,13 +1368,12 @@ func (cs *State) isProposalComplete() bool {
//
// NOTE: keep it side-effect free for clarity.
// CONTRACT: cs.privValidator is not nil.
func (cs *State) createProposalBlock(ctx context.Context) (block *types.Block, blockParts *types.PartSet, err error) {
func (cs *State) createProposalBlock(ctx context.Context) (*types.Block, error) {
if cs.privValidator == nil {
return nil, nil, errors.New("entered createProposalBlock with privValidator being nil")
return nil, errors.New("entered createProposalBlock with privValidator being nil")
}
var commit *types.Commit
var votes []*types.Vote
switch {
case cs.Height == cs.state.InitialHeight:
// We're creating a proposal for the first block.
@ -1376,23 +1383,22 @@ func (cs *State) createProposalBlock(ctx context.Context) (block *types.Block, b
case cs.LastCommit.HasTwoThirdsMajority():
// Make the commit from LastCommit
commit = cs.LastCommit.MakeCommit()
votes = cs.LastCommit.GetVotes()
default: // This shouldn't happen.
cs.logger.Error("propose step; cannot propose anything without commit for the previous block")
return
return nil, nil
}
if cs.privValidatorPubKey == nil {
// If this node is a validator & proposer in the current round, it will
// miss the opportunity to create a block.
cs.logger.Error("propose step; empty priv validator public key", "err", errPubKeyIsNotSet)
return
return nil, nil
}
proposerAddr := cs.privValidatorPubKey.Address()
return cs.blockExec.CreateProposalBlock(ctx, cs.Height, cs.state, commit, proposerAddr, votes)
return cs.blockExec.CreateProposalBlock(ctx, cs.Height, cs.state, commit, proposerAddr, cs.LastCommit.GetVotes())
}
// Enter: `timeoutPropose` after entering Propose.


+ 4
- 4
internal/consensus/state_test.go View File

@ -220,7 +220,7 @@ func TestStateBadProposal(t *testing.T) {
proposalCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryCompleteProposal)
voteCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryVote)
propBlock, _, err := cs1.createProposalBlock(ctx) // changeProposer(t, cs1, vs2)
propBlock, err := cs1.createProposalBlock(ctx) // changeProposer(t, cs1, vs2)
require.NoError(t, err)
// make the second validator the proposer by incrementing round
@ -282,7 +282,7 @@ func TestStateOversizedBlock(t *testing.T) {
timeoutProposeCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryTimeoutPropose)
voteCh := subscribe(ctx, t, cs1.eventBus, types.EventQueryVote)
propBlock, _, err := cs1.createProposalBlock(ctx)
propBlock, err := cs1.createProposalBlock(ctx)
require.NoError(t, err)
propBlock.Data.Txs = []types.Tx{tmrand.Bytes(2001)}
propBlock.Header.DataHash = propBlock.Data.Hash()
@ -2632,7 +2632,7 @@ func TestStateTimestamp_ProposalNotMatch(t *testing.T) {
addr := pv1.Address()
voteCh := subscribeToVoter(ctx, t, cs1, addr)
propBlock, _, err := cs1.createProposalBlock(ctx)
propBlock, err := cs1.createProposalBlock(ctx)
require.NoError(t, err)
round++
incrementRound(vss[1:]...)
@ -2680,7 +2680,7 @@ func TestStateTimestamp_ProposalMatch(t *testing.T) {
addr := pv1.Address()
voteCh := subscribeToVoter(ctx, t, cs1, addr)
propBlock, _, err := cs1.createProposalBlock(ctx)
propBlock, err := cs1.createProposalBlock(ctx)
require.NoError(t, err)
round++
incrementRound(vss[1:]...)


+ 1
- 4
internal/evidence/pool_test.go View File

@ -569,10 +569,7 @@ func initializeBlockStore(db dbm.DB, state sm.State, valAddr []byte) (*store.Blo
for i := int64(1); i <= state.LastBlockHeight; i++ {
lastCommit := makeCommit(i-1, valAddr)
block, err := sf.MakeBlock(state, i, lastCommit)
if err != nil {
return nil, err
}
block := sf.MakeBlock(state, i, lastCommit)
block.Header.Time = defaultEvidenceTime.Add(time.Duration(i) * time.Minute)
block.Header.Version = version.Consensus{Block: version.BlockProtocol, App: 1}


+ 0
- 46
internal/mempool/mock/mempool.go View File

@ -1,46 +0,0 @@
package mock
import (
"context"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/internal/libs/clist"
"github.com/tendermint/tendermint/internal/mempool"
"github.com/tendermint/tendermint/types"
)
// Mempool is an empty implementation of a Mempool, useful for testing.
type Mempool struct{}
var _ Mempool = Mempool{}
func (Mempool) Lock() {}
func (Mempool) Unlock() {}
func (Mempool) Size() int { return 0 }
func (Mempool) CheckTx(context.Context, types.Tx, func(*abci.ResponseCheckTx), mempool.TxInfo) error {
return nil
}
func (Mempool) RemoveTxByKey(txKey types.TxKey) error { return nil }
func (Mempool) ReapMaxBytesMaxGas(_, _ int64) types.Txs { return types.Txs{} }
func (Mempool) ReapMaxTxs(n int) types.Txs { return types.Txs{} }
func (Mempool) Update(
_ context.Context,
_ int64,
_ types.Txs,
_ []*abci.ExecTxResult,
_ mempool.PreCheckFunc,
_ mempool.PostCheckFunc,
) error {
return nil
}
func (Mempool) Flush() {}
func (Mempool) FlushAppConn(ctx context.Context) error { return nil }
func (Mempool) TxsAvailable() <-chan struct{} { return make(chan struct{}) }
func (Mempool) EnableTxsAvailable() {}
func (Mempool) SizeBytes() int64 { return 0 }
func (Mempool) TxsFront() *clist.CElement { return nil }
func (Mempool) TxsWaitChan() <-chan struct{} { return nil }
func (Mempool) InitWAL() error { return nil }
func (Mempool) CloseWAL() {}

+ 172
- 0
internal/mempool/mocks/mempool.go View File

@ -0,0 +1,172 @@
// Code generated by mockery. DO NOT EDIT.
package mocks
import (
context "context"
abcitypes "github.com/tendermint/tendermint/abci/types"
mempool "github.com/tendermint/tendermint/internal/mempool"
mock "github.com/stretchr/testify/mock"
types "github.com/tendermint/tendermint/types"
)
// Mempool is an autogenerated mock type for the Mempool type
type Mempool struct {
mock.Mock
}
// CheckTx provides a mock function with given fields: ctx, tx, callback, txInfo
func (_m *Mempool) CheckTx(ctx context.Context, tx types.Tx, callback func(*abcitypes.ResponseCheckTx), txInfo mempool.TxInfo) error {
ret := _m.Called(ctx, tx, callback, txInfo)
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, types.Tx, func(*abcitypes.ResponseCheckTx), mempool.TxInfo) error); ok {
r0 = rf(ctx, tx, callback, txInfo)
} else {
r0 = ret.Error(0)
}
return r0
}
// EnableTxsAvailable provides a mock function with given fields:
func (_m *Mempool) EnableTxsAvailable() {
_m.Called()
}
// Flush provides a mock function with given fields:
func (_m *Mempool) Flush() {
_m.Called()
}
// FlushAppConn provides a mock function with given fields: _a0
func (_m *Mempool) FlushAppConn(_a0 context.Context) error {
ret := _m.Called(_a0)
var r0 error
if rf, ok := ret.Get(0).(func(context.Context) error); ok {
r0 = rf(_a0)
} else {
r0 = ret.Error(0)
}
return r0
}
// Lock provides a mock function with given fields:
func (_m *Mempool) Lock() {
_m.Called()
}
// ReapMaxBytesMaxGas provides a mock function with given fields: maxBytes, maxGas
func (_m *Mempool) ReapMaxBytesMaxGas(maxBytes int64, maxGas int64) types.Txs {
ret := _m.Called(maxBytes, maxGas)
var r0 types.Txs
if rf, ok := ret.Get(0).(func(int64, int64) types.Txs); ok {
r0 = rf(maxBytes, maxGas)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(types.Txs)
}
}
return r0
}
// ReapMaxTxs provides a mock function with given fields: max
func (_m *Mempool) ReapMaxTxs(max int) types.Txs {
ret := _m.Called(max)
var r0 types.Txs
if rf, ok := ret.Get(0).(func(int) types.Txs); ok {
r0 = rf(max)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(types.Txs)
}
}
return r0
}
// RemoveTxByKey provides a mock function with given fields: txKey
func (_m *Mempool) RemoveTxByKey(txKey types.TxKey) error {
ret := _m.Called(txKey)
var r0 error
if rf, ok := ret.Get(0).(func(types.TxKey) error); ok {
r0 = rf(txKey)
} else {
r0 = ret.Error(0)
}
return r0
}
// Size provides a mock function with given fields:
func (_m *Mempool) Size() int {
ret := _m.Called()
var r0 int
if rf, ok := ret.Get(0).(func() int); ok {
r0 = rf()
} else {
r0 = ret.Get(0).(int)
}
return r0
}
// SizeBytes provides a mock function with given fields:
func (_m *Mempool) SizeBytes() int64 {
ret := _m.Called()
var r0 int64
if rf, ok := ret.Get(0).(func() int64); ok {
r0 = rf()
} else {
r0 = ret.Get(0).(int64)
}
return r0
}
// TxsAvailable provides a mock function with given fields:
func (_m *Mempool) TxsAvailable() <-chan struct{} {
ret := _m.Called()
var r0 <-chan struct{}
if rf, ok := ret.Get(0).(func() <-chan struct{}); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(<-chan struct{})
}
}
return r0
}
// Unlock provides a mock function with given fields:
func (_m *Mempool) Unlock() {
_m.Called()
}
// Update provides a mock function with given fields: ctx, blockHeight, blockTxs, txResults, newPreFn, newPostFn
func (_m *Mempool) Update(ctx context.Context, blockHeight int64, blockTxs types.Txs, txResults []*abcitypes.ExecTxResult, newPreFn mempool.PreCheckFunc, newPostFn mempool.PostCheckFunc) error {
ret := _m.Called(ctx, blockHeight, blockTxs, txResults, newPreFn, newPostFn)
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, int64, types.Txs, []*abcitypes.ExecTxResult, mempool.PreCheckFunc, mempool.PostCheckFunc) error); ok {
r0 = rf(ctx, blockHeight, blockTxs, txResults, newPreFn, newPostFn)
} else {
r0 = ret.Error(0)
}
return r0
}

+ 2
- 0
internal/mempool/types.go View File

@ -23,6 +23,8 @@ const (
MaxActiveIDs = math.MaxUint16
)
//go:generate ../../scripts/mockery_generate.sh Mempool
// Mempool defines the mempool interface.
//
// Updates to the mempool need to be synchronized with committing a block so


+ 5
- 8
internal/rpc/core/tx.go View File

@ -36,19 +36,16 @@ func (env *Environment) Tx(ctx context.Context, hash bytes.HexBytes, prove bool)
return nil, fmt.Errorf("tx (%X) not found, err: %w", hash, err)
}
height := r.Height
index := r.Index
var proof types.TxProof
if prove {
block := env.BlockStore.LoadBlock(height)
proof = block.Data.Txs.Proof(int(index)) // XXX: overflow on 32-bit machines
block := env.BlockStore.LoadBlock(r.Height)
proof = block.Data.Txs.Proof(int(r.Index))
}
return &coretypes.ResultTx{
Hash: hash,
Height: height,
Index: index,
Height: r.Height,
Index: r.Index,
TxResult: r.Result,
Tx: r.Tx,
Proof: proof,
@ -127,7 +124,7 @@ func (env *Environment) TxSearch(
var proof types.TxProof
if prove {
block := env.BlockStore.LoadBlock(r.Height)
proof = block.Data.Txs.Proof(int(r.Index)) // XXX: overflow on 32-bit machines
proof = block.Data.Txs.Proof(int(r.Index))
}
apiResults = append(apiResults, &coretypes.ResultTx{


+ 55
- 16
internal/state/execution.go View File

@ -8,6 +8,7 @@ import (
abciclient "github.com/tendermint/tendermint/abci/client"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto/encoding"
"github.com/tendermint/tendermint/crypto/merkle"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/internal/mempool"
"github.com/tendermint/tendermint/libs/log"
@ -99,10 +100,11 @@ func (blockExec *BlockExecutor) Store() Store {
func (blockExec *BlockExecutor) CreateProposalBlock(
ctx context.Context,
height int64,
state State, commit *types.Commit,
state State,
commit *types.Commit,
proposerAddr []byte,
votes []*types.Vote,
) (*types.Block, *types.PartSet, error) {
) (*types.Block, error) {
maxBytes := state.ConsensusParams.Block.MaxBytes
maxGas := state.ConsensusParams.Block.MaxGas
@ -113,13 +115,18 @@ func (blockExec *BlockExecutor) CreateProposalBlock(
maxDataBytes := types.MaxDataBytes(maxBytes, evSize, state.Validators.Size())
txs := blockExec.mempool.ReapMaxBytesMaxGas(maxDataBytes, maxGas)
block := state.MakeBlock(height, txs, commit, evidence, proposerAddr)
preparedProposal, err := blockExec.appClient.PrepareProposal(
localLastCommit := buildLastCommitInfo(block, blockExec.store, state.InitialHeight)
rpp, err := blockExec.appClient.PrepareProposal(
ctx,
abci.RequestPrepareProposal{
BlockData: txs.ToSliceOfBytes(),
BlockDataSize: maxDataBytes,
Votes: types.VotesToProto(votes),
Hash: block.Hash(),
Header: *block.Header.ToProto(),
Txs: block.Txs.ToSliceOfBytes(),
LocalLastCommit: extendedCommitInfo(localLastCommit, votes),
ByzantineValidators: block.Evidence.ToABCI(),
MaxTxBytes: maxDataBytes,
},
)
if err != nil {
@ -133,19 +140,28 @@ func (blockExec *BlockExecutor) CreateProposalBlock(
// purpose for now.
panic(err)
}
newTxs := preparedProposal.GetBlockData()
var txSize int
for _, tx := range newTxs {
txSize += len(tx)
if maxDataBytes < int64(txSize) {
panic("block data exceeds max amount of allowed bytes")
}
if !rpp.ModifiedTx {
return block, nil
}
txrSet := types.NewTxRecordSet(rpp.TxRecords)
modifiedTxs := types.ToTxs(preparedProposal.GetBlockData())
if err := txrSet.Validate(maxDataBytes, block.Txs); err != nil {
return nil, err
}
return state.MakeBlock(height, modifiedTxs, commit, evidence, proposerAddr)
for _, rtx := range txrSet.RemovedTxs() {
if err := blockExec.mempool.RemoveTxByKey(rtx.Key()); err != nil {
blockExec.logger.Debug("error removing transaction from the mempool", "error", err, "tx hash", rtx.Hash())
}
}
for _, atx := range txrSet.AddedTxs() {
if err := blockExec.mempool.CheckTx(ctx, atx, nil, mempool.TxInfo{}); err != nil {
blockExec.logger.Error("error adding tx to the mempool", "error", err, "tx hash", atx.Hash())
}
}
itxs := txrSet.IncludedTxs()
return state.MakeBlock(height, itxs, commit, evidence, proposerAddr), nil
}
func (blockExec *BlockExecutor) ProcessProposal(
@ -249,7 +265,12 @@ func (blockExec *BlockExecutor) ApplyBlock(
}
// Update the state with the block and responses.
state, err = state.Update(blockID, &block.Header, ABCIResponsesResultsHash(abciResponses), finalizeBlockResponse.ConsensusParamUpdates, validatorUpdates)
rs, err := abci.MarshalTxResults(finalizeBlockResponse.TxResults)
if err != nil {
return state, fmt.Errorf("marshaling TxResults: %w", err)
}
h := merkle.HashFromByteSlices(rs)
state, err = state.Update(blockID, &block.Header, h, finalizeBlockResponse.ConsensusParamUpdates, validatorUpdates)
if err != nil {
return state, fmt.Errorf("commit failed for application: %w", err)
}
@ -410,6 +431,24 @@ func buildLastCommitInfo(block *types.Block, store Store, initialHeight int64) a
}
}
func extendedCommitInfo(c abci.CommitInfo, votes []*types.Vote) abci.ExtendedCommitInfo {
vs := make([]abci.ExtendedVoteInfo, len(c.Votes))
for i := range vs {
vs[i] = abci.ExtendedVoteInfo{
Validator: c.Votes[i].Validator,
SignedLastBlock: c.Votes[i].SignedLastBlock,
/*
TODO: Include vote extensions information when implementing vote extensions.
VoteExtension: []byte{},
*/
}
}
return abci.ExtendedCommitInfo{
Round: c.Round,
Votes: vs,
}
}
func validateValidatorUpdates(abciUpdates []abci.ValidatorUpdate,
params types.ValidatorParams) error {
for _, valUpdate := range abciUpdates {


+ 357
- 22
internal/state/execution_test.go View File

@ -18,7 +18,7 @@ import (
"github.com/tendermint/tendermint/crypto/encoding"
"github.com/tendermint/tendermint/crypto/tmhash"
"github.com/tendermint/tendermint/internal/eventbus"
mmock "github.com/tendermint/tendermint/internal/mempool/mock"
mpmocks "github.com/tendermint/tendermint/internal/mempool/mocks"
"github.com/tendermint/tendermint/internal/proxy"
"github.com/tendermint/tendermint/internal/pubsub"
sm "github.com/tendermint/tendermint/internal/state"
@ -53,10 +53,20 @@ func TestApplyBlock(t *testing.T) {
state, stateDB, _ := makeState(t, 1, 1)
stateStore := sm.NewStore(stateDB)
blockStore := store.NewBlockStore(dbm.NewMemDB())
blockExec := sm.NewBlockExecutor(stateStore, logger, proxyApp, mmock.Mempool{}, sm.EmptyEvidencePool{}, blockStore, eventBus)
block, err := sf.MakeBlock(state, 1, new(types.Commit))
require.NoError(t, err)
mp := &mpmocks.Mempool{}
mp.On("Lock").Return()
mp.On("Unlock").Return()
mp.On("FlushAppConn", mock.Anything).Return(nil)
mp.On("Update",
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything).Return(nil)
blockExec := sm.NewBlockExecutor(stateStore, logger, proxyApp, mp, sm.EmptyEvidencePool{}, blockStore, eventBus)
block := sf.MakeBlock(state, 1, new(types.Commit))
bps, err := block.MakePartSet(testPartSize)
require.NoError(t, err)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
@ -103,11 +113,22 @@ func TestFinalizeBlockDecidedLastCommit(t *testing.T) {
evpool.On("PendingEvidence", mock.Anything).Return([]types.Evidence{}, 0)
evpool.On("Update", ctx, mock.Anything, mock.Anything).Return()
evpool.On("CheckEvidence", ctx, mock.Anything).Return(nil)
mp := &mpmocks.Mempool{}
mp.On("Lock").Return()
mp.On("Unlock").Return()
mp.On("FlushAppConn", mock.Anything).Return(nil)
mp.On("Update",
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything).Return(nil)
eventBus := eventbus.NewDefault(logger)
require.NoError(t, eventBus.Start(ctx))
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), appClient, mmock.Mempool{}, evpool, blockStore, eventBus)
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), appClient, mp, evpool, blockStore, eventBus)
state, _, lastCommit := makeAndCommitGoodBlock(ctx, t, state, 1, new(types.Commit), state.NextValidators.Validators[0].Address, blockExec, privVals, nil)
for idx, isAbsent := range tc.absentCommitSigs {
@ -117,8 +138,7 @@ func TestFinalizeBlockDecidedLastCommit(t *testing.T) {
}
// block for height 2
block, err := sf.MakeBlock(state, 2, lastCommit)
require.NoError(t, err)
block := sf.MakeBlock(state, 2, lastCommit)
bps, err := block.MakePartSet(testPartSize)
require.NoError(t, err)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
@ -215,6 +235,17 @@ func TestFinalizeBlockByzantineValidators(t *testing.T) {
evpool.On("PendingEvidence", mock.AnythingOfType("int64")).Return(ev, int64(100))
evpool.On("Update", ctx, mock.AnythingOfType("state.State"), mock.AnythingOfType("types.EvidenceList")).Return()
evpool.On("CheckEvidence", ctx, mock.AnythingOfType("types.EvidenceList")).Return(nil)
mp := &mpmocks.Mempool{}
mp.On("Lock").Return()
mp.On("Unlock").Return()
mp.On("FlushAppConn", mock.Anything).Return(nil)
mp.On("Update",
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything).Return(nil)
eventBus := eventbus.NewDefault(logger)
require.NoError(t, eventBus.Start(ctx))
@ -222,10 +253,9 @@ func TestFinalizeBlockByzantineValidators(t *testing.T) {
blockStore := store.NewBlockStore(dbm.NewMemDB())
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), proxyApp,
mmock.Mempool{}, evpool, blockStore, eventBus)
mp, evpool, blockStore, eventBus)
block, err := sf.MakeBlock(state, 1, new(types.Commit))
require.NoError(t, err)
block := sf.MakeBlock(state, 1, new(types.Commit))
block.Evidence = ev
block.Header.EvidenceHash = block.Evidence.Hash()
bps, err := block.MakePartSet(testPartSize)
@ -264,14 +294,13 @@ func TestProcessProposal(t *testing.T) {
stateStore,
logger,
proxyApp,
mmock.Mempool{},
new(mpmocks.Mempool),
sm.EmptyEvidencePool{},
blockStore,
eventBus,
)
block0, err := sf.MakeBlock(state, height-1, new(types.Commit))
require.NoError(t, err)
block0 := sf.MakeBlock(state, height-1, new(types.Commit))
lastCommitSig := []types.CommitSig{}
partSet, err := block0.MakePartSet(types.BlockPartSizeBytes)
require.NoError(t, err)
@ -295,8 +324,7 @@ func TestProcessProposal(t *testing.T) {
}
lastCommit := types.NewCommit(height-1, 0, types.BlockID{}, lastCommitSig)
block1, err := sf.MakeBlock(state, height, lastCommit)
require.NoError(t, err)
block1 := sf.MakeBlock(state, height, lastCommit)
block1.Txs = txs
expectedRpp := abci.RequestProcessProposal{
@ -463,6 +491,18 @@ func TestFinalizeBlockValidatorUpdates(t *testing.T) {
state, stateDB, _ := makeState(t, 1, 1)
stateStore := sm.NewStore(stateDB)
blockStore := store.NewBlockStore(dbm.NewMemDB())
mp := &mpmocks.Mempool{}
mp.On("Lock").Return()
mp.On("Unlock").Return()
mp.On("FlushAppConn", mock.Anything).Return(nil)
mp.On("Update",
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything).Return(nil)
mp.On("ReapMaxBytesMaxGas", mock.Anything, mock.Anything).Return(types.Txs{})
eventBus := eventbus.NewDefault(logger)
require.NoError(t, eventBus.Start(ctx))
@ -471,7 +511,7 @@ func TestFinalizeBlockValidatorUpdates(t *testing.T) {
stateStore,
logger,
proxyApp,
mmock.Mempool{},
mp,
sm.EmptyEvidencePool{},
blockStore,
eventBus,
@ -483,8 +523,7 @@ func TestFinalizeBlockValidatorUpdates(t *testing.T) {
})
require.NoError(t, err)
block, err := sf.MakeBlock(state, 1, new(types.Commit))
require.NoError(t, err)
block := sf.MakeBlock(state, 1, new(types.Commit))
bps, err := block.MakePartSet(testPartSize)
require.NoError(t, err)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
@ -542,14 +581,13 @@ func TestFinalizeBlockValidatorUpdatesResultingInEmptySet(t *testing.T) {
stateStore,
log.TestingLogger(),
proxyApp,
mmock.Mempool{},
new(mpmocks.Mempool),
sm.EmptyEvidencePool{},
blockStore,
eventBus,
)
block, err := sf.MakeBlock(state, 1, new(types.Commit))
require.NoError(t, err)
block := sf.MakeBlock(state, 1, new(types.Commit))
bps, err := block.MakePartSet(testPartSize)
require.NoError(t, err)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
@ -566,6 +604,292 @@ func TestFinalizeBlockValidatorUpdatesResultingInEmptySet(t *testing.T) {
assert.NotEmpty(t, state.NextValidators.Validators)
}
func TestEmptyPrepareProposal(t *testing.T) {
const height = 2
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
logger := log.TestingLogger()
eventBus := eventbus.NewDefault(logger)
require.NoError(t, eventBus.Start(ctx))
app := abcimocks.NewBaseMock()
cc := abciclient.NewLocalClient(logger, app)
proxyApp := proxy.New(cc, logger, proxy.NopMetrics())
err := proxyApp.Start(ctx)
require.NoError(t, err)
state, stateDB, privVals := makeState(t, 1, height)
stateStore := sm.NewStore(stateDB)
mp := &mpmocks.Mempool{}
mp.On("Lock").Return()
mp.On("Unlock").Return()
mp.On("FlushAppConn", mock.Anything).Return(nil)
mp.On("Update",
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything).Return(nil)
mp.On("ReapMaxBytesMaxGas", mock.Anything, mock.Anything).Return(types.Txs{})
blockExec := sm.NewBlockExecutor(
stateStore,
logger,
proxyApp,
mp,
sm.EmptyEvidencePool{},
nil,
eventBus,
)
pa, _ := state.Validators.GetByIndex(0)
commit := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals)
_, err = blockExec.CreateProposalBlock(ctx, height, state, commit, pa, nil)
require.NoError(t, err)
}
// TestPrepareProposalRemoveTxs tests that any transactions marked as REMOVED
// are not included in the block produced by CreateProposalBlock. The test also
// ensures that any transactions removed are also removed from the mempool.
func TestPrepareProposalRemoveTxs(t *testing.T) {
const height = 2
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
logger := log.TestingLogger()
eventBus := eventbus.NewDefault(logger)
require.NoError(t, eventBus.Start(ctx))
state, stateDB, privVals := makeState(t, 1, height)
stateStore := sm.NewStore(stateDB)
evpool := &mocks.EvidencePool{}
evpool.On("PendingEvidence", mock.Anything).Return([]types.Evidence{}, int64(0))
txs := factory.MakeTenTxs(height)
mp := &mpmocks.Mempool{}
mp.On("ReapMaxBytesMaxGas", mock.Anything, mock.Anything).Return(types.Txs(txs))
trs := txsToTxRecords(types.Txs(txs))
trs[0].Action = abci.TxRecord_REMOVED
trs[1].Action = abci.TxRecord_REMOVED
mp.On("RemoveTxByKey", mock.Anything).Return(nil).Twice()
app := abcimocks.NewBaseMock()
app.On("PrepareProposal", mock.Anything).Return(abci.ResponsePrepareProposal{
ModifiedTx: true,
TxRecords: trs,
}, nil)
cc := abciclient.NewLocalClient(logger, app)
proxyApp := proxy.New(cc, logger, proxy.NopMetrics())
err := proxyApp.Start(ctx)
require.NoError(t, err)
blockExec := sm.NewBlockExecutor(
stateStore,
logger,
proxyApp,
mp,
evpool,
nil,
eventBus,
)
pa, _ := state.Validators.GetByIndex(0)
commit := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals)
block, err := blockExec.CreateProposalBlock(ctx, height, state, commit, pa, nil)
require.NoError(t, err)
require.Len(t, block.Data.Txs.ToSliceOfBytes(), len(trs)-2)
require.Equal(t, -1, block.Data.Txs.Index(types.Tx(trs[0].Tx)))
require.Equal(t, -1, block.Data.Txs.Index(types.Tx(trs[1].Tx)))
mp.AssertCalled(t, "RemoveTxByKey", types.Tx(trs[0].Tx).Key())
mp.AssertCalled(t, "RemoveTxByKey", types.Tx(trs[1].Tx).Key())
mp.AssertExpectations(t)
}
// TestPrepareProposalAddedTxsIncluded tests that any transactions marked as ADDED
// in the prepare proposal response are included in the block. The test also
// ensures that any transactions added are also checked into the mempool.
func TestPrepareProposalAddedTxsIncluded(t *testing.T) {
const height = 2
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
logger := log.TestingLogger()
eventBus := eventbus.NewDefault(logger)
require.NoError(t, eventBus.Start(ctx))
state, stateDB, privVals := makeState(t, 1, height)
stateStore := sm.NewStore(stateDB)
evpool := &mocks.EvidencePool{}
evpool.On("PendingEvidence", mock.Anything).Return([]types.Evidence{}, int64(0))
txs := factory.MakeTenTxs(height)
mp := &mpmocks.Mempool{}
mp.On("ReapMaxBytesMaxGas", mock.Anything, mock.Anything).Return(types.Txs(txs[2:]))
mp.On("CheckTx", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil).Twice()
trs := txsToTxRecords(types.Txs(txs))
trs[0].Action = abci.TxRecord_ADDED
trs[1].Action = abci.TxRecord_ADDED
app := abcimocks.NewBaseMock()
app.On("PrepareProposal", mock.Anything).Return(abci.ResponsePrepareProposal{
ModifiedTx: true,
TxRecords: trs,
}, nil)
cc := abciclient.NewLocalClient(logger, app)
proxyApp := proxy.New(cc, logger, proxy.NopMetrics())
err := proxyApp.Start(ctx)
require.NoError(t, err)
blockExec := sm.NewBlockExecutor(
stateStore,
logger,
proxyApp,
mp,
evpool,
nil,
eventBus,
)
pa, _ := state.Validators.GetByIndex(0)
commit := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals)
block, err := blockExec.CreateProposalBlock(ctx, height, state, commit, pa, nil)
require.NoError(t, err)
require.Equal(t, txs[0], block.Data.Txs[0])
require.Equal(t, txs[1], block.Data.Txs[1])
mp.AssertExpectations(t)
mp.AssertCalled(t, "CheckTx", mock.Anything, types.Tx(trs[0].Tx), mock.Anything, mock.Anything)
mp.AssertCalled(t, "CheckTx", mock.Anything, types.Tx(trs[1].Tx), mock.Anything, mock.Anything)
}
// TestPrepareProposalReorderTxs tests that CreateBlock produces a block with transactions
// in the order matching the order they are returned from PrepareProposal.
func TestPrepareProposalReorderTxs(t *testing.T) {
const height = 2
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
logger := log.TestingLogger()
eventBus := eventbus.NewDefault(logger)
require.NoError(t, eventBus.Start(ctx))
state, stateDB, privVals := makeState(t, 1, height)
stateStore := sm.NewStore(stateDB)
evpool := &mocks.EvidencePool{}
evpool.On("PendingEvidence", mock.Anything).Return([]types.Evidence{}, int64(0))
txs := factory.MakeTenTxs(height)
mp := &mpmocks.Mempool{}
mp.On("ReapMaxBytesMaxGas", mock.Anything, mock.Anything).Return(types.Txs(txs))
trs := txsToTxRecords(types.Txs(txs))
trs = trs[2:]
trs = append(trs[len(trs)/2:], trs[:len(trs)/2]...)
app := abcimocks.NewBaseMock()
app.On("PrepareProposal", mock.Anything).Return(abci.ResponsePrepareProposal{
ModifiedTx: true,
TxRecords: trs,
}, nil)
cc := abciclient.NewLocalClient(logger, app)
proxyApp := proxy.New(cc, logger, proxy.NopMetrics())
err := proxyApp.Start(ctx)
require.NoError(t, err)
blockExec := sm.NewBlockExecutor(
stateStore,
logger,
proxyApp,
mp,
evpool,
nil,
eventBus,
)
pa, _ := state.Validators.GetByIndex(0)
commit := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals)
block, err := blockExec.CreateProposalBlock(ctx, height, state, commit, pa, nil)
require.NoError(t, err)
for i, tx := range block.Data.Txs {
require.Equal(t, types.Tx(trs[i].Tx), tx)
}
mp.AssertExpectations(t)
}
// TestPrepareProposalModifiedTxFalse tests that CreateBlock correctly ignores
// the ResponsePrepareProposal TxRecords if ResponsePrepareProposal does not
// set ModifiedTx to true.
func TestPrepareProposalModifiedTxFalse(t *testing.T) {
const height = 2
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
logger := log.TestingLogger()
eventBus := eventbus.NewDefault(logger)
require.NoError(t, eventBus.Start(ctx))
state, stateDB, privVals := makeState(t, 1, height)
stateStore := sm.NewStore(stateDB)
evpool := &mocks.EvidencePool{}
evpool.On("PendingEvidence", mock.Anything).Return([]types.Evidence{}, int64(0))
txs := factory.MakeTenTxs(height)
mp := &mpmocks.Mempool{}
mp.On("ReapMaxBytesMaxGas", mock.Anything, mock.Anything).Return(types.Txs(txs))
trs := txsToTxRecords(types.Txs(txs))
trs = append(trs[len(trs)/2:], trs[:len(trs)/2]...)
trs = trs[1:]
trs[0].Action = abci.TxRecord_REMOVED
trs[1] = &abci.TxRecord{
Tx: []byte("new"),
Action: abci.TxRecord_ADDED,
}
app := abcimocks.NewBaseMock()
app.On("PrepareProposal", mock.Anything).Return(abci.ResponsePrepareProposal{
ModifiedTx: false,
TxRecords: trs,
}, nil)
cc := abciclient.NewLocalClient(logger, app)
proxyApp := proxy.New(cc, logger, proxy.NopMetrics())
err := proxyApp.Start(ctx)
require.NoError(t, err)
blockExec := sm.NewBlockExecutor(
stateStore,
logger,
proxyApp,
mp,
evpool,
nil,
eventBus,
)
pa, _ := state.Validators.GetByIndex(0)
commit := makeValidCommit(ctx, t, height, types.BlockID{}, state.Validators, privVals)
block, err := blockExec.CreateProposalBlock(ctx, height, state, commit, pa, nil)
require.NoError(t, err)
for i, tx := range block.Data.Txs {
require.Equal(t, txs[i], tx)
}
mp.AssertExpectations(t)
}
func makeBlockID(hash []byte, partSetSize uint32, partSetHash []byte) types.BlockID {
var (
h = make([]byte, tmhash.Size)
@ -581,3 +905,14 @@ func makeBlockID(hash []byte, partSetSize uint32, partSetHash []byte) types.Bloc
},
}
}
func txsToTxRecords(txs []types.Tx) []*abci.TxRecord {
trs := make([]*abci.TxRecord, len(txs))
for i, tx := range txs {
trs[i] = &abci.TxRecord{
Action: abci.TxRecord_UNMODIFIED,
Tx: tx,
}
}
return trs
}

+ 6
- 9
internal/state/helpers_test.go View File

@ -18,7 +18,6 @@ import (
sm "github.com/tendermint/tendermint/internal/state"
sf "github.com/tendermint/tendermint/internal/state/test/factory"
"github.com/tendermint/tendermint/internal/test/factory"
tmrand "github.com/tendermint/tendermint/libs/rand"
tmtime "github.com/tendermint/tendermint/libs/time"
tmstate "github.com/tendermint/tendermint/proto/tendermint/state"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
@ -63,12 +62,13 @@ func makeAndApplyGoodBlock(
evidence []types.Evidence,
) (sm.State, types.BlockID) {
t.Helper()
block, _, err := state.MakeBlock(height, factory.MakeTenTxs(height), lastCommit, evidence, proposerAddr)
block := state.MakeBlock(height, factory.MakeTenTxs(height), lastCommit, evidence, proposerAddr)
partSet, err := block.MakePartSet(types.BlockPartSizeBytes)
require.NoError(t, err)
require.NoError(t, blockExec.ValidateBlock(ctx, state, block))
blockID := types.BlockID{Hash: block.Hash(),
PartSetHeader: types.PartSetHeader{Total: 3, Hash: tmrand.Bytes(32)}}
PartSetHeader: partSet.Header()}
state, err = blockExec.ApplyBlock(ctx, state, blockID, block)
require.NoError(t, err)
@ -144,8 +144,7 @@ func makeHeaderPartsResponsesValPubKeyChange(
pubkey crypto.PubKey,
) (types.Header, types.BlockID, *tmstate.ABCIResponses) {
block, err := sf.MakeBlock(state, state.LastBlockHeight+1, new(types.Commit))
require.NoError(t, err)
block := sf.MakeBlock(state, state.LastBlockHeight+1, new(types.Commit))
abciResponses := &tmstate.ABCIResponses{}
// If the pubkey is new, remove the old and add the new.
_, val := state.NextValidators.GetByIndex(0)
@ -173,8 +172,7 @@ func makeHeaderPartsResponsesValPowerChange(
) (types.Header, types.BlockID, *tmstate.ABCIResponses) {
t.Helper()
block, err := sf.MakeBlock(state, state.LastBlockHeight+1, new(types.Commit))
require.NoError(t, err)
block := sf.MakeBlock(state, state.LastBlockHeight+1, new(types.Commit))
abciResponses := &tmstate.ABCIResponses{}
@ -202,8 +200,7 @@ func makeHeaderPartsResponsesParams(
) (types.Header, types.BlockID, *tmstate.ABCIResponses) {
t.Helper()
block, err := sf.MakeBlock(state, state.LastBlockHeight+1, new(types.Commit))
require.NoError(t, err)
block := sf.MakeBlock(state, state.LastBlockHeight+1, new(types.Commit))
pbParams := params.ToProto()
abciResponses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{ConsensusParamUpdates: &pbParams},


+ 2
- 7
internal/state/state.go View File

@ -260,7 +260,7 @@ func (state State) MakeBlock(
commit *types.Commit,
evidence []types.Evidence,
proposerAddress []byte,
) (*types.Block, *types.PartSet, error) {
) *types.Block {
// Build base block with block data.
block := types.MakeBlock(height, txs, commit, evidence)
@ -274,12 +274,7 @@ func (state State) MakeBlock(
proposerAddress,
)
bps, err := block.MakePartSet(types.BlockPartSizeBytes)
if err != nil {
return nil, nil, err
}
return block, bps, nil
return block
}
//------------------------------------------------------------------------


+ 125
- 103
internal/state/state_test.go View File

@ -18,6 +18,7 @@ import (
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/crypto/ed25519"
"github.com/tendermint/tendermint/crypto/encoding"
"github.com/tendermint/tendermint/crypto/merkle"
sm "github.com/tendermint/tendermint/internal/state"
statefactory "github.com/tendermint/tendermint/internal/state/test/factory"
tmstate "github.com/tendermint/tendermint/proto/tendermint/state"
@ -103,8 +104,7 @@ func TestABCIResponsesSaveLoad1(t *testing.T) {
state.LastBlockHeight++
// Build mock responses.
block, err := statefactory.MakeBlock(state, 2, new(types.Commit))
require.NoError(t, err)
block := statefactory.MakeBlock(state, 2, new(types.Commit))
abciResponses := new(tmstate.ABCIResponses)
dtxs := make([]*abci.ExecTxResult, 2)
@ -205,14 +205,13 @@ func TestABCIResponsesSaveLoad2(t *testing.T) {
res, err := stateStore.LoadABCIResponses(h)
if assert.NoError(t, err, "%d", i) {
t.Log(res)
responses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
TxResults: tc.expected,
},
}
sm.ABCIResponsesResultsHash(res)
sm.ABCIResponsesResultsHash(responses)
assert.Equal(t, sm.ABCIResponsesResultsHash(responses), sm.ABCIResponsesResultsHash(res), "%d", i)
e, err := abci.MarshalTxResults(tc.expected)
require.NoError(t, err)
he := merkle.HashFromByteSlices(e)
rs, err := abci.MarshalTxResults(res.FinalizeBlock.TxResults)
hrs := merkle.HashFromByteSlices(rs)
require.NoError(t, err)
assert.Equal(t, he, hrs, "%d", i)
}
}
}
@ -278,9 +277,12 @@ func TestOneValidatorChangesSaveLoad(t *testing.T) {
header, blockID, responses := makeHeaderPartsResponsesValPowerChange(t, state, power)
validatorUpdates, err = types.PB2TM.ValidatorUpdates(responses.FinalizeBlock.ValidatorUpdates)
require.NoError(t, err)
state, err = state.Update(blockID, &header, sm.ABCIResponsesResultsHash(responses), responses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err := abci.MarshalTxResults(responses.FinalizeBlock.TxResults)
require.NoError(t, err)
h := merkle.HashFromByteSlices(rs)
state, err = state.Update(blockID, &header, h, responses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
err := stateStore.Save(state)
err = stateStore.Save(state)
require.NoError(t, err)
}
@ -451,19 +453,19 @@ func TestProposerPriorityDoesNotGetResetToZero(t *testing.T) {
// NewValidatorSet calls IncrementProposerPriority but uses on a copy of val1
assert.EqualValues(t, 0, val1.ProposerPriority)
block, err := statefactory.MakeBlock(state, state.LastBlockHeight+1, new(types.Commit))
require.NoError(t, err)
block := statefactory.MakeBlock(state, state.LastBlockHeight+1, new(types.Commit))
bps, err := block.MakePartSet(testPartSize)
require.NoError(t, err)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
abciResponses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
ValidatorUpdates: nil,
},
fb := &abci.ResponseFinalizeBlock{
ValidatorUpdates: nil,
}
validatorUpdates, err := types.PB2TM.ValidatorUpdates(abciResponses.FinalizeBlock.ValidatorUpdates)
validatorUpdates, err := types.PB2TM.ValidatorUpdates(fb.ValidatorUpdates)
require.NoError(t, err)
rs, err := abci.MarshalTxResults(fb.TxResults)
require.NoError(t, err)
updatedState, err := state.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
h := merkle.HashFromByteSlices(rs)
updatedState, err := state.Update(blockID, &block.Header, h, fb.ConsensusParamUpdates, validatorUpdates)
assert.NoError(t, err)
curTotal := val1VotingPower
// one increment step and one validator: 0 + power - total_power == 0
@ -478,7 +480,10 @@ func TestProposerPriorityDoesNotGetResetToZero(t *testing.T) {
updateAddVal := abci.ValidatorUpdate{PubKey: fvp, Power: val2VotingPower}
validatorUpdates, err = types.PB2TM.ValidatorUpdates([]abci.ValidatorUpdate{updateAddVal})
assert.NoError(t, err)
updatedState2, err := updatedState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err = abci.MarshalTxResults(fb.TxResults)
require.NoError(t, err)
h = merkle.HashFromByteSlices(rs)
updatedState2, err := updatedState.Update(blockID, &block.Header, h, fb.ConsensusParamUpdates, validatorUpdates)
assert.NoError(t, err)
require.Equal(t, len(updatedState2.NextValidators.Validators), 2)
@ -517,7 +522,10 @@ func TestProposerPriorityDoesNotGetResetToZero(t *testing.T) {
// this will cause the diff of priorities (77)
// to be larger than threshold == 2*totalVotingPower (22):
updatedState3, err := updatedState2.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err = abci.MarshalTxResults(fb.TxResults)
require.NoError(t, err)
h = merkle.HashFromByteSlices(rs)
updatedState3, err := updatedState2.Update(blockID, &block.Header, h, fb.ConsensusParamUpdates, validatorUpdates)
assert.NoError(t, err)
require.Equal(t, len(updatedState3.NextValidators.Validators), 2)
@ -569,21 +577,21 @@ func TestProposerPriorityProposerAlternates(t *testing.T) {
// we only have one validator:
assert.Equal(t, val1PubKey.Address(), state.Validators.Proposer.Address)
block, err := statefactory.MakeBlock(state, state.LastBlockHeight+1, new(types.Commit))
require.NoError(t, err)
block := statefactory.MakeBlock(state, state.LastBlockHeight+1, new(types.Commit))
bps, err := block.MakePartSet(testPartSize)
require.NoError(t, err)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
// no updates:
abciResponses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
ValidatorUpdates: nil,
},
fb := &abci.ResponseFinalizeBlock{
ValidatorUpdates: nil,
}
validatorUpdates, err := types.PB2TM.ValidatorUpdates(abciResponses.FinalizeBlock.ValidatorUpdates)
validatorUpdates, err := types.PB2TM.ValidatorUpdates(fb.ValidatorUpdates)
require.NoError(t, err)
updatedState, err := state.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err := abci.MarshalTxResults(fb.TxResults)
require.NoError(t, err)
h := merkle.HashFromByteSlices(rs)
updatedState, err := state.Update(blockID, &block.Header, h, fb.ConsensusParamUpdates, validatorUpdates)
assert.NoError(t, err)
// 0 + 10 (initial prio) - 10 (avg) - 10 (mostest - total) = -10
@ -600,7 +608,10 @@ func TestProposerPriorityProposerAlternates(t *testing.T) {
validatorUpdates, err = types.PB2TM.ValidatorUpdates([]abci.ValidatorUpdate{updateAddVal})
assert.NoError(t, err)
updatedState2, err := updatedState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err = abci.MarshalTxResults(fb.TxResults)
require.NoError(t, err)
h = merkle.HashFromByteSlices(rs)
updatedState2, err := updatedState.Update(blockID, &block.Header, h, fb.ConsensusParamUpdates, validatorUpdates)
assert.NoError(t, err)
require.Equal(t, len(updatedState2.NextValidators.Validators), 2)
@ -640,10 +651,13 @@ func TestProposerPriorityProposerAlternates(t *testing.T) {
updatedVal2,
)
validatorUpdates, err = types.PB2TM.ValidatorUpdates(abciResponses.FinalizeBlock.ValidatorUpdates)
validatorUpdates, err = types.PB2TM.ValidatorUpdates(fb.ValidatorUpdates)
require.NoError(t, err)
updatedState3, err := updatedState2.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err = abci.MarshalTxResults(fb.TxResults)
require.NoError(t, err)
h = merkle.HashFromByteSlices(rs)
updatedState3, err := updatedState2.Update(blockID, &block.Header, h, fb.ConsensusParamUpdates, validatorUpdates)
assert.NoError(t, err)
assert.Equal(t, updatedState3.Validators.Proposer.Address, updatedState3.NextValidators.Proposer.Address)
@ -679,15 +693,16 @@ func TestProposerPriorityProposerAlternates(t *testing.T) {
// no changes in voting power and both validators have same voting power
// -> proposers should alternate:
oldState := updatedState3
abciResponses = &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
ValidatorUpdates: nil,
},
fb = &abci.ResponseFinalizeBlock{
ValidatorUpdates: nil,
}
validatorUpdates, err = types.PB2TM.ValidatorUpdates(abciResponses.FinalizeBlock.ValidatorUpdates)
validatorUpdates, err = types.PB2TM.ValidatorUpdates(fb.ValidatorUpdates)
require.NoError(t, err)
oldState, err = oldState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err = abci.MarshalTxResults(fb.TxResults)
require.NoError(t, err)
h = merkle.HashFromByteSlices(rs)
oldState, err = oldState.Update(blockID, &block.Header, h, fb.ConsensusParamUpdates, validatorUpdates)
assert.NoError(t, err)
expectedVal1Prio2 = 1
expectedVal2Prio2 = -1
@ -696,15 +711,16 @@ func TestProposerPriorityProposerAlternates(t *testing.T) {
for i := 0; i < 1000; i++ {
// no validator updates:
abciResponses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
ValidatorUpdates: nil,
},
fb := &abci.ResponseFinalizeBlock{
ValidatorUpdates: nil,
}
validatorUpdates, err = types.PB2TM.ValidatorUpdates(abciResponses.FinalizeBlock.ValidatorUpdates)
validatorUpdates, err = types.PB2TM.ValidatorUpdates(fb.ValidatorUpdates)
require.NoError(t, err)
updatedState, err := oldState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err := abci.MarshalTxResults(fb.TxResults)
require.NoError(t, err)
h := merkle.HashFromByteSlices(rs)
updatedState, err := oldState.Update(blockID, &block.Header, h, fb.ConsensusParamUpdates, validatorUpdates)
assert.NoError(t, err)
// alternate (and cyclic priorities):
assert.NotEqual(
@ -755,21 +771,21 @@ func TestLargeGenesisValidator(t *testing.T) {
oldState := state
for i := 0; i < 10; i++ {
// no updates:
abciResponses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
ValidatorUpdates: nil,
},
fb := &abci.ResponseFinalizeBlock{
ValidatorUpdates: nil,
}
validatorUpdates, err := types.PB2TM.ValidatorUpdates(abciResponses.FinalizeBlock.ValidatorUpdates)
validatorUpdates, err := types.PB2TM.ValidatorUpdates(fb.ValidatorUpdates)
require.NoError(t, err)
block, err := statefactory.MakeBlock(oldState, oldState.LastBlockHeight+1, new(types.Commit))
require.NoError(t, err)
block := statefactory.MakeBlock(oldState, oldState.LastBlockHeight+1, new(types.Commit))
bps, err := block.MakePartSet(testPartSize)
require.NoError(t, err)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
updatedState, err := oldState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err := abci.MarshalTxResults(fb.TxResults)
require.NoError(t, err)
h := merkle.HashFromByteSlices(rs)
updatedState, err := oldState.Update(blockID, &block.Header, h, fb.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
// no changes in voting power (ProposerPrio += VotingPower == Voting in 1st round; than shiftByAvg == 0,
// than -Total == -Voting)
@ -791,41 +807,41 @@ func TestLargeGenesisValidator(t *testing.T) {
firstAddedVal := abci.ValidatorUpdate{PubKey: fvp, Power: firstAddedValVotingPower}
validatorUpdates, err := types.PB2TM.ValidatorUpdates([]abci.ValidatorUpdate{firstAddedVal})
assert.NoError(t, err)
abciResponses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
ValidatorUpdates: []abci.ValidatorUpdate{firstAddedVal},
},
fb := &abci.ResponseFinalizeBlock{
ValidatorUpdates: []abci.ValidatorUpdate{firstAddedVal},
}
block, err := statefactory.MakeBlock(oldState, oldState.LastBlockHeight+1, new(types.Commit))
require.NoError(t, err)
block := statefactory.MakeBlock(oldState, oldState.LastBlockHeight+1, new(types.Commit))
bps, err := block.MakePartSet(testPartSize)
require.NoError(t, err)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
updatedState, err := oldState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err := abci.MarshalTxResults(fb.TxResults)
require.NoError(t, err)
h := merkle.HashFromByteSlices(rs)
updatedState, err := oldState.Update(blockID, &block.Header, h, fb.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
lastState := updatedState
for i := 0; i < 200; i++ {
// no updates:
abciResponses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
ValidatorUpdates: nil,
},
fb := &abci.ResponseFinalizeBlock{
ValidatorUpdates: nil,
}
validatorUpdates, err := types.PB2TM.ValidatorUpdates(abciResponses.FinalizeBlock.ValidatorUpdates)
validatorUpdates, err := types.PB2TM.ValidatorUpdates(fb.ValidatorUpdates)
require.NoError(t, err)
block, err := statefactory.MakeBlock(lastState, lastState.LastBlockHeight+1, new(types.Commit))
require.NoError(t, err)
block := statefactory.MakeBlock(lastState, lastState.LastBlockHeight+1, new(types.Commit))
bps, err = block.MakePartSet(testPartSize)
require.NoError(t, err)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
updatedStateInner, err := lastState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err := abci.MarshalTxResults(fb.TxResults)
require.NoError(t, err)
h := merkle.HashFromByteSlices(rs)
updatedStateInner, err := lastState.Update(blockID, &block.Header, h, fb.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
lastState = updatedStateInner
}
@ -851,18 +867,18 @@ func TestLargeGenesisValidator(t *testing.T) {
validatorUpdates, err := types.PB2TM.ValidatorUpdates([]abci.ValidatorUpdate{addedVal})
assert.NoError(t, err)
abciResponses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
ValidatorUpdates: []abci.ValidatorUpdate{addedVal},
},
fb := &abci.ResponseFinalizeBlock{
ValidatorUpdates: []abci.ValidatorUpdate{addedVal},
}
block, err := statefactory.MakeBlock(oldState, oldState.LastBlockHeight+1, new(types.Commit))
require.NoError(t, err)
block := statefactory.MakeBlock(oldState, oldState.LastBlockHeight+1, new(types.Commit))
bps, err := block.MakePartSet(testPartSize)
require.NoError(t, err)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
state, err = state.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err := abci.MarshalTxResults(fb.TxResults)
require.NoError(t, err)
h := merkle.HashFromByteSlices(rs)
state, err = state.Update(blockID, &block.Header, h, fb.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
}
require.Equal(t, 10+2, len(state.NextValidators.Validators))
@ -871,22 +887,23 @@ func TestLargeGenesisValidator(t *testing.T) {
gp, err := encoding.PubKeyToProto(genesisPubKey)
require.NoError(t, err)
removeGenesisVal := abci.ValidatorUpdate{PubKey: gp, Power: 0}
abciResponses = &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
ValidatorUpdates: []abci.ValidatorUpdate{removeGenesisVal},
},
fb = &abci.ResponseFinalizeBlock{
ValidatorUpdates: []abci.ValidatorUpdate{removeGenesisVal},
}
block, err = statefactory.MakeBlock(oldState, oldState.LastBlockHeight+1, new(types.Commit))
block = statefactory.MakeBlock(oldState, oldState.LastBlockHeight+1, new(types.Commit))
require.NoError(t, err)
bps, err = block.MakePartSet(testPartSize)
require.NoError(t, err)
blockID = types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
validatorUpdates, err = types.PB2TM.ValidatorUpdates(abciResponses.FinalizeBlock.ValidatorUpdates)
validatorUpdates, err = types.PB2TM.ValidatorUpdates(fb.ValidatorUpdates)
require.NoError(t, err)
updatedState, err = state.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err = abci.MarshalTxResults(fb.TxResults)
require.NoError(t, err)
h = merkle.HashFromByteSlices(rs)
updatedState, err = state.Update(blockID, &block.Header, h, fb.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
// only the first added val (not the genesis val) should be left
assert.Equal(t, 11, len(updatedState.NextValidators.Validators))
@ -897,21 +914,21 @@ func TestLargeGenesisValidator(t *testing.T) {
count := 0
isProposerUnchanged := true
for isProposerUnchanged {
abciResponses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
ValidatorUpdates: nil,
},
fb = &abci.ResponseFinalizeBlock{
ValidatorUpdates: nil,
}
validatorUpdates, err = types.PB2TM.ValidatorUpdates(abciResponses.FinalizeBlock.ValidatorUpdates)
require.NoError(t, err)
block, err = statefactory.MakeBlock(curState, curState.LastBlockHeight+1, new(types.Commit))
validatorUpdates, err = types.PB2TM.ValidatorUpdates(fb.ValidatorUpdates)
require.NoError(t, err)
block = statefactory.MakeBlock(curState, curState.LastBlockHeight+1, new(types.Commit))
bps, err := block.MakePartSet(testPartSize)
require.NoError(t, err)
blockID = types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
curState, err = curState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err := abci.MarshalTxResults(fb.TxResults)
require.NoError(t, err)
h := merkle.HashFromByteSlices(rs)
curState, err = curState.Update(blockID, &block.Header, h, fb.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
if !bytes.Equal(curState.Validators.Proposer.Address, curState.NextValidators.Proposer.Address) {
isProposerUnchanged = false
@ -927,23 +944,23 @@ func TestLargeGenesisValidator(t *testing.T) {
proposers := make([]*types.Validator, numVals)
for i := 0; i < 100; i++ {
// no updates:
abciResponses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
ValidatorUpdates: nil,
},
fb := &abci.ResponseFinalizeBlock{
ValidatorUpdates: nil,
}
validatorUpdates, err := types.PB2TM.ValidatorUpdates(abciResponses.FinalizeBlock.ValidatorUpdates)
validatorUpdates, err := types.PB2TM.ValidatorUpdates(fb.ValidatorUpdates)
require.NoError(t, err)
block, err := statefactory.MakeBlock(updatedState, updatedState.LastBlockHeight+1, new(types.Commit))
require.NoError(t, err)
block := statefactory.MakeBlock(updatedState, updatedState.LastBlockHeight+1, new(types.Commit))
bps, err := block.MakePartSet(testPartSize)
require.NoError(t, err)
blockID := types.BlockID{Hash: block.Hash(), PartSetHeader: bps.Header()}
updatedState, err = updatedState.Update(blockID, &block.Header, sm.ABCIResponsesResultsHash(abciResponses), abciResponses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err := abci.MarshalTxResults(fb.TxResults)
require.NoError(t, err)
h := merkle.HashFromByteSlices(rs)
updatedState, err = updatedState.Update(blockID, &block.Header, h, fb.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
if i > numVals { // expect proposers to cycle through after the first iteration (of numVals blocks):
if proposers[i%numVals] == nil {
@ -1002,7 +1019,10 @@ func TestManyValidatorChangesSaveLoad(t *testing.T) {
var validatorUpdates []*types.Validator
validatorUpdates, err = types.PB2TM.ValidatorUpdates(responses.FinalizeBlock.ValidatorUpdates)
require.NoError(t, err)
state, err = state.Update(blockID, &header, sm.ABCIResponsesResultsHash(responses), responses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err := abci.MarshalTxResults(responses.FinalizeBlock.TxResults)
require.NoError(t, err)
h := merkle.HashFromByteSlices(rs)
state, err = state.Update(blockID, &header, h, responses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
nextHeight := state.LastBlockHeight + 1
err = stateStore.Save(state)
@ -1035,8 +1055,7 @@ func TestStateMakeBlock(t *testing.T) {
proposerAddress := state.Validators.GetProposer().Address
stateVersion := state.Version.Consensus
block, err := statefactory.MakeBlock(state, 2, new(types.Commit))
require.NoError(t, err)
block := statefactory.MakeBlock(state, 2, new(types.Commit))
// test we set some fields
assert.Equal(t, stateVersion, block.Version)
@ -1080,10 +1099,13 @@ func TestConsensusParamsChangesSaveLoad(t *testing.T) {
header, blockID, responses := makeHeaderPartsResponsesParams(t, state, &cp)
validatorUpdates, err = types.PB2TM.ValidatorUpdates(responses.FinalizeBlock.ValidatorUpdates)
require.NoError(t, err)
state, err = state.Update(blockID, &header, sm.ABCIResponsesResultsHash(responses), responses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
rs, err := abci.MarshalTxResults(responses.FinalizeBlock.TxResults)
require.NoError(t, err)
h := merkle.HashFromByteSlices(rs)
state, err = state.Update(blockID, &header, h, responses.FinalizeBlock.ConsensusParamUpdates, validatorUpdates)
require.NoError(t, err)
err := stateStore.Save(state)
err = stateStore.Save(state)
require.NoError(t, err)
}


+ 0
- 8
internal/state/store.go View File

@ -396,14 +396,6 @@ func (store dbStore) reverseBatchDelete(batch dbm.Batch, start, end []byte) ([]b
//------------------------------------------------------------------------
// ABCIResponsesResultsHash returns the root hash of a Merkle tree of
// ResponseDeliverTx responses (see ABCIResults.Hash)
//
// See merkle.SimpleHashFromByteSlices
func ABCIResponsesResultsHash(ar *tmstate.ABCIResponses) []byte {
return types.NewResults(ar.FinalizeBlock.TxResults).Hash()
}
// LoadABCIResponses loads the ABCIResponses for the given height from the
// database. If not found, ErrNoABCIResponsesForHeight is returned.
//


+ 0
- 23
internal/state/store_test.go View File

@ -7,7 +7,6 @@ import (
"os"
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
dbm "github.com/tendermint/tm-db"
@ -299,25 +298,3 @@ func TestPruneStates(t *testing.T) {
})
}
}
func TestABCIResponsesResultsHash(t *testing.T) {
responses := &tmstate.ABCIResponses{
FinalizeBlock: &abci.ResponseFinalizeBlock{
TxResults: []*abci.ExecTxResult{
{Code: 32, Data: []byte("Hello"), Log: "Huh?"},
},
},
}
root := sm.ABCIResponsesResultsHash(responses)
// root should be Merkle tree root of FinalizeBlock tx responses
results := types.NewResults(responses.FinalizeBlock.TxResults)
assert.Equal(t, root, results.Hash())
// test we can prove first tx in FinalizeBlock
proof := results.ProveResult(0)
bz, err := results[0].Marshal()
require.NoError(t, err)
assert.NoError(t, proof.Verify(root, bz))
}

+ 4
- 8
internal/state/test/factory/block.go View File

@ -42,19 +42,14 @@ func MakeBlocks(ctx context.Context, t *testing.T, n int, state *sm.State, privV
return blocks
}
func MakeBlock(state sm.State, height int64, c *types.Commit) (*types.Block, error) {
block, _, err := state.MakeBlock(
func MakeBlock(state sm.State, height int64, c *types.Commit) *types.Block {
return state.MakeBlock(
height,
factory.MakeTenTxs(state.LastBlockHeight),
c,
nil,
state.Validators.GetProposer().Address,
)
if err != nil {
return nil, err
}
return block, nil
}
func makeBlockAndPartSet(
@ -82,7 +77,8 @@ func makeBlockAndPartSet(
lastBlockMeta.BlockID, []types.CommitSig{vote.CommitSig()})
}
block, partSet, err := state.MakeBlock(height, []types.Tx{}, lastCommit, nil, state.Validators.GetProposer().Address)
block := state.MakeBlock(height, []types.Tx{}, lastCommit, nil, state.Validators.GetProposer().Address)
partSet, err := block.MakePartSet(types.BlockPartSizeBytes)
require.NoError(t, err)
return block, partSet


+ 47
- 17
internal/state/validation_test.go View File

@ -15,7 +15,7 @@ import (
"github.com/tendermint/tendermint/crypto/ed25519"
"github.com/tendermint/tendermint/crypto/tmhash"
"github.com/tendermint/tendermint/internal/eventbus"
memmock "github.com/tendermint/tendermint/internal/mempool/mock"
mpmocks "github.com/tendermint/tendermint/internal/mempool/mocks"
"github.com/tendermint/tendermint/internal/proxy"
sm "github.com/tendermint/tendermint/internal/state"
"github.com/tendermint/tendermint/internal/state/mocks"
@ -42,12 +42,24 @@ func TestValidateBlockHeader(t *testing.T) {
state, stateDB, privVals := makeState(t, 3, 1)
stateStore := sm.NewStore(stateDB)
mp := &mpmocks.Mempool{}
mp.On("Lock").Return()
mp.On("Unlock").Return()
mp.On("FlushAppConn", mock.Anything).Return(nil)
mp.On("Update",
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything).Return(nil)
blockStore := store.NewBlockStore(dbm.NewMemDB())
blockExec := sm.NewBlockExecutor(
stateStore,
logger,
proxyApp,
memmock.Mempool{},
mp,
sm.EmptyEvidencePool{},
blockStore,
eventBus,
@ -98,10 +110,9 @@ func TestValidateBlockHeader(t *testing.T) {
Invalid blocks don't pass
*/
for _, tc := range testCases {
block, err := statefactory.MakeBlock(state, height, lastCommit)
require.NoError(t, err)
block := statefactory.MakeBlock(state, height, lastCommit)
tc.malleateBlock(block)
err = blockExec.ValidateBlock(ctx, state, block)
err := blockExec.ValidateBlock(ctx, state, block)
t.Logf("%s: %v", tc.name, err)
require.Error(t, err, tc.name)
}
@ -114,10 +125,9 @@ func TestValidateBlockHeader(t *testing.T) {
}
nextHeight := validationTestsStopHeight
block, err := statefactory.MakeBlock(state, nextHeight, lastCommit)
require.NoError(t, err)
block := statefactory.MakeBlock(state, nextHeight, lastCommit)
state.InitialHeight = nextHeight + 1
err = blockExec.ValidateBlock(ctx, state, block)
err := blockExec.ValidateBlock(ctx, state, block)
require.Error(t, err, "expected an error when state is ahead of block")
assert.Contains(t, err.Error(), "lower than initial height")
}
@ -135,12 +145,24 @@ func TestValidateBlockCommit(t *testing.T) {
state, stateDB, privVals := makeState(t, 1, 1)
stateStore := sm.NewStore(stateDB)
mp := &mpmocks.Mempool{}
mp.On("Lock").Return()
mp.On("Unlock").Return()
mp.On("FlushAppConn", mock.Anything).Return(nil)
mp.On("Update",
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything).Return(nil)
blockStore := store.NewBlockStore(dbm.NewMemDB())
blockExec := sm.NewBlockExecutor(
stateStore,
logger,
proxyApp,
memmock.Mempool{},
mp,
sm.EmptyEvidencePool{},
blockStore,
eventBus,
@ -174,8 +196,7 @@ func TestValidateBlockCommit(t *testing.T) {
state.LastBlockID,
[]types.CommitSig{wrongHeightVote.CommitSig()},
)
block, err := statefactory.MakeBlock(state, height, wrongHeightCommit)
require.NoError(t, err)
block := statefactory.MakeBlock(state, height, wrongHeightCommit)
err = blockExec.ValidateBlock(ctx, state, block)
_, isErrInvalidCommitHeight := err.(types.ErrInvalidCommitHeight)
require.True(t, isErrInvalidCommitHeight, "expected ErrInvalidCommitHeight at height %d but got: %v", height, err)
@ -183,8 +204,7 @@ func TestValidateBlockCommit(t *testing.T) {
/*
#2589: test len(block.LastCommit.Signatures) == state.LastValidators.Size()
*/
block, err = statefactory.MakeBlock(state, height, wrongSigsCommit)
require.NoError(t, err)
block = statefactory.MakeBlock(state, height, wrongSigsCommit)
err = blockExec.ValidateBlock(ctx, state, block)
_, isErrInvalidCommitSignatures := err.(types.ErrInvalidCommitSignatures)
require.True(t, isErrInvalidCommitSignatures,
@ -274,13 +294,24 @@ func TestValidateBlockEvidence(t *testing.T) {
eventBus := eventbus.NewDefault(logger)
require.NoError(t, eventBus.Start(ctx))
mp := &mpmocks.Mempool{}
mp.On("Lock").Return()
mp.On("Unlock").Return()
mp.On("FlushAppConn", mock.Anything).Return(nil)
mp.On("Update",
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything,
mock.Anything).Return(nil)
state.ConsensusParams.Evidence.MaxBytes = 1000
blockExec := sm.NewBlockExecutor(
stateStore,
log.TestingLogger(),
proxyApp,
memmock.Mempool{},
mp,
evpool,
blockStore,
eventBus,
@ -304,10 +335,9 @@ func TestValidateBlockEvidence(t *testing.T) {
evidence = append(evidence, newEv)
currentBytes += int64(len(newEv.Bytes()))
}
block, _, err := state.MakeBlock(height, testfactory.MakeTenTxs(height), lastCommit, evidence, proposerAddr)
require.NoError(t, err)
block := state.MakeBlock(height, testfactory.MakeTenTxs(height), lastCommit, evidence, proposerAddr)
err = blockExec.ValidateBlock(ctx, state, block)
err := blockExec.ValidateBlock(ctx, state, block)
if assert.Error(t, err) {
_, ok := err.(*types.ErrEvidenceOverflow)
require.True(t, ok, "expected error to be of type ErrEvidenceOverflow at height %d but got %v", height, err)


+ 6
- 14
internal/store/store_test.go View File

@ -86,11 +86,8 @@ func TestMain(m *testing.M) {
stdlog.Fatal(err)
}
block, err = factory.MakeBlock(state, 1, new(types.Commit))
block = factory.MakeBlock(state, 1, new(types.Commit))
if err != nil {
stdlog.Fatal(err)
}
partSet, err = block.MakePartSet(2)
if err != nil {
stdlog.Fatal(err)
@ -121,8 +118,7 @@ func TestBlockStoreSaveLoadBlock(t *testing.T) {
}
// save a block
block, err := factory.MakeBlock(state, bs.Height()+1, new(types.Commit))
require.NoError(t, err)
block := factory.MakeBlock(state, bs.Height()+1, new(types.Commit))
validPartSet, err := block.MakePartSet(2)
require.NoError(t, err)
seenCommit := makeTestCommit(10, tmtime.Now())
@ -326,8 +322,7 @@ func TestLoadBaseMeta(t *testing.T) {
bs := NewBlockStore(dbm.NewMemDB())
for h := int64(1); h <= 10; h++ {
block, err := factory.MakeBlock(state, h, new(types.Commit))
require.NoError(t, err)
block := factory.MakeBlock(state, h, new(types.Commit))
partSet, err := block.MakePartSet(2)
require.NoError(t, err)
seenCommit := makeTestCommit(h, tmtime.Now())
@ -394,8 +389,7 @@ func TestPruneBlocks(t *testing.T) {
// make more than 1000 blocks, to test batch deletions
for h := int64(1); h <= 1500; h++ {
block, err := factory.MakeBlock(state, h, new(types.Commit))
require.NoError(t, err)
block := factory.MakeBlock(state, h, new(types.Commit))
partSet, err := block.MakePartSet(2)
require.NoError(t, err)
seenCommit := makeTestCommit(h, tmtime.Now())
@ -502,8 +496,7 @@ func TestBlockFetchAtHeight(t *testing.T) {
defer cleanup()
require.NoError(t, err)
require.Equal(t, bs.Height(), int64(0), "initially the height should be zero")
block, err := factory.MakeBlock(state, bs.Height()+1, new(types.Commit))
require.NoError(t, err)
block := factory.MakeBlock(state, bs.Height()+1, new(types.Commit))
partSet, err := block.MakePartSet(2)
require.NoError(t, err)
@ -545,8 +538,7 @@ func TestSeenAndCanonicalCommit(t *testing.T) {
// are persisted.
for h := int64(3); h <= 5; h++ {
blockCommit := makeTestCommit(h-1, tmtime.Now())
block, err := factory.MakeBlock(state, h, blockCommit)
require.NoError(t, err)
block := factory.MakeBlock(state, h, blockCommit)
partSet, err := block.MakePartSet(2)
require.NoError(t, err)
seenCommit := makeTestCommit(h, tmtime.Now())


+ 7
- 6
light/rpc/client.go View File

@ -458,16 +458,17 @@ func (c *Client) BlockResults(ctx context.Context, height *int64) (*coretypes.Re
return nil, err
}
// Build a Merkle tree of proto-encoded FinalizeBlock tx results and get a hash.
results := types.NewResults(res.TxsResults)
// Build a Merkle tree out of the slice.
rH := merkle.HashFromByteSlices([][]byte{bbeBytes, results.Hash()})
rs, err := abci.MarshalTxResults(res.TxsResults)
if err != nil {
return nil, err
}
mh := merkle.HashFromByteSlices(append([][]byte{bbeBytes}, rs...))
// Verify block results.
if !bytes.Equal(rH, trustedBlock.LastResultsHash) {
if !bytes.Equal(mh, trustedBlock.LastResultsHash) {
return nil, fmt.Errorf("last results %X does not match with trusted last results %X",
rH, trustedBlock.LastResultsHash)
mh, trustedBlock.LastResultsHash)
}
return res, nil


+ 12
- 4
node/node_test.go View File

@ -336,7 +336,7 @@ func TestCreateProposalBlock(t *testing.T) {
)
commit := types.NewCommit(height-1, 0, types.BlockID{}, nil)
block, _, err := blockExec.CreateProposalBlock(
block, err := blockExec.CreateProposalBlock(
ctx,
height,
state, commit,
@ -415,7 +415,7 @@ func TestMaxTxsProposalBlockSize(t *testing.T) {
)
commit := types.NewCommit(height-1, 0, types.BlockID{}, nil)
block, _, err := blockExec.CreateProposalBlock(
block, err := blockExec.CreateProposalBlock(
ctx,
height,
state, commit,
@ -497,10 +497,16 @@ func TestMaxProposalBlockSize(t *testing.T) {
},
}
// save the updated validator set for use by the block executor.
state.LastBlockHeight = math.MaxInt64 - 3
state.LastHeightValidatorsChanged = math.MaxInt64 - 1
state.NextValidators = state.Validators.Copy()
require.NoError(t, stateStore.Save(state))
timestamp := time.Date(math.MaxInt64, 0, 0, 0, 0, 0, math.MaxInt64, time.UTC)
// change state in order to produce the largest accepted header
state.LastBlockID = blockID
state.LastBlockHeight = math.MaxInt64 - 1
state.LastBlockHeight = math.MaxInt64 - 2
state.LastBlockTime = timestamp
state.LastResultsHash = tmhash.Sum([]byte("last_results_hash"))
state.AppHash = tmhash.Sum([]byte("app_hash"))
@ -530,7 +536,7 @@ func TestMaxProposalBlockSize(t *testing.T) {
commit.Signatures = append(commit.Signatures, cs)
}
block, partSet, err := blockExec.CreateProposalBlock(
block, err := blockExec.CreateProposalBlock(
ctx,
math.MaxInt64,
state, commit,
@ -538,6 +544,8 @@ func TestMaxProposalBlockSize(t *testing.T) {
nil,
)
require.NoError(t, err)
partSet, err := block.MakePartSet(types.BlockPartSizeBytes)
require.NoError(t, err)
// this ensures that the header is at max size
block.Header.Time = timestamp


+ 78
- 51
proto/tendermint/abci/types.proto.intermediate View File

@ -8,9 +8,8 @@ import "tendermint/types/params.proto";
import "google/protobuf/timestamp.proto";
import "gogoproto/gogo.proto";
// This file is a temporary workaround to enable development during the ABCI++
// project. This file should be deleted and any references to it removed when
// project. This file should be deleted and any references to it removed when
// the ongoing work on ABCI++ is completed.
//
// For the duration of ABCI++, this file should be able to build the `abci/types/types.pb.go`
@ -23,25 +22,25 @@ import "gogoproto/gogo.proto";
message Request {
oneof value {
RequestEcho echo = 1;
RequestFlush flush = 2;
RequestInfo info = 3;
RequestInitChain init_chain = 4;
RequestQuery query = 5;
RequestBeginBlock begin_block = 6 [deprecated = true];
RequestCheckTx check_tx = 7;
RequestDeliverTx deliver_tx = 8 [deprecated = true];
RequestEndBlock end_block = 9 [deprecated = true];
RequestCommit commit = 10;
RequestListSnapshots list_snapshots = 11;
RequestOfferSnapshot offer_snapshot = 12;
RequestLoadSnapshotChunk load_snapshot_chunk = 13;
RequestApplySnapshotChunk apply_snapshot_chunk = 14;
RequestPrepareProposal prepare_proposal = 15;
RequestProcessProposal process_proposal = 16;
RequestEcho echo = 1;
RequestFlush flush = 2;
RequestInfo info = 3;
RequestInitChain init_chain = 4;
RequestQuery query = 5;
RequestBeginBlock begin_block = 6 [deprecated = true];
RequestCheckTx check_tx = 7;
RequestDeliverTx deliver_tx = 8 [deprecated = true];
RequestEndBlock end_block = 9 [deprecated = true];
RequestCommit commit = 10;
RequestListSnapshots list_snapshots = 11;
RequestOfferSnapshot offer_snapshot = 12;
RequestLoadSnapshotChunk load_snapshot_chunk = 13;
RequestApplySnapshotChunk apply_snapshot_chunk = 14;
RequestPrepareProposal prepare_proposal = 15;
RequestProcessProposal process_proposal = 16;
RequestExtendVote extend_vote = 17;
RequestVerifyVoteExtension verify_vote_extension = 18;
RequestFinalizeBlock finalize_block = 19;
RequestFinalizeBlock finalize_block = 19;
}
}
@ -135,18 +134,17 @@ message RequestVerifyVoteExtension {
}
message RequestPrepareProposal {
// block_data is an array of transactions that will be included in a block,
bytes hash = 1;
tendermint.types.Header header = 2 [(gogoproto.nullable) = false];
// txs is an array of transactions that will be included in a block,
// sent to the app for possible modifications.
// applications can not exceed the size of the data passed to it.
repeated bytes block_data = 1;
// If an application decides to populate block_data with extra information, they can not exceed this value.
int64 block_data_size = 2;
// votes includes all votes from the previous block. This contains vote extension data that can be used in proposal
// preparation. The votes here will then form the last commit that gets sent in the proposed block.
repeated tendermint.types.Vote votes = 3;
repeated bytes txs = 3;
ExtendedCommitInfo local_last_commit = 4 [(gogoproto.nullable) = false];
repeated Evidence byzantine_validators = 5 [(gogoproto.nullable) = false];
// the modified transactions cannot exceed this size.
int64 max_tx_bytes = 6;
}
message RequestProcessProposal {
bytes hash = 1;
tendermint.types.Header header = 2 [(gogoproto.nullable) = false];
@ -168,26 +166,26 @@ message RequestFinalizeBlock {
message Response {
oneof value {
ResponseException exception = 1;
ResponseEcho echo = 2;
ResponseFlush flush = 3;
ResponseInfo info = 4;
ResponseInitChain init_chain = 5;
ResponseQuery query = 6;
ResponseBeginBlock begin_block = 7 [deprecated = true];
ResponseCheckTx check_tx = 8;
ResponseDeliverTx deliver_tx = 9 [deprecated = true];
ResponseEndBlock end_block = 10 [deprecated = true];
ResponseCommit commit = 11;
ResponseListSnapshots list_snapshots = 12;
ResponseOfferSnapshot offer_snapshot = 13;
ResponseLoadSnapshotChunk load_snapshot_chunk = 14;
ResponseApplySnapshotChunk apply_snapshot_chunk = 15;
ResponsePrepareProposal prepare_proposal = 16;
ResponseProcessProposal process_proposal = 17;
ResponseException exception = 1;
ResponseEcho echo = 2;
ResponseFlush flush = 3;
ResponseInfo info = 4;
ResponseInitChain init_chain = 5;
ResponseQuery query = 6;
ResponseBeginBlock begin_block = 7 [deprecated = true];
ResponseCheckTx check_tx = 8;
ResponseDeliverTx deliver_tx = 9 [deprecated = true];
ResponseEndBlock end_block = 10 [deprecated = true];
ResponseCommit commit = 11;
ResponseListSnapshots list_snapshots = 12;
ResponseOfferSnapshot offer_snapshot = 13;
ResponseLoadSnapshotChunk load_snapshot_chunk = 14;
ResponseApplySnapshotChunk apply_snapshot_chunk = 15;
ResponsePrepareProposal prepare_proposal = 16;
ResponseProcessProposal process_proposal = 17;
ResponseExtendVote extend_vote = 18;
ResponseVerifyVoteExtension verify_vote_extension = 19;
ResponseFinalizeBlock finalize_block = 20;
ResponseFinalizeBlock finalize_block = 20;
}
}
@ -330,7 +328,12 @@ message ResponseVerifyVoteExtension {
}
message ResponsePrepareProposal {
repeated bytes block_data = 1;
bool modified_tx = 1;
repeated TxRecord tx_records = 2;
bytes app_hash = 3;
repeated ExecTxResult tx_results = 4;
repeated ValidatorUpdate validator_updates = 5;
tendermint.types.ConsensusParams consensus_param_updates = 6;
}
message ResponseProcessProposal {
@ -359,6 +362,11 @@ message CommitInfo {
repeated VoteInfo votes = 2 [(gogoproto.nullable) = false];
}
message ExtendedCommitInfo {
int32 round = 1;
repeated ExtendedVoteInfo votes = 2 [(gogoproto.nullable) = false];
}
// Event allows application developers to attach additional information to
// ResponseBeginBlock, ResponseEndBlock, ResponseCheckTx and ResponseDeliverTx.
// Later, transactions may be queried using these events.
@ -384,7 +392,7 @@ message ExecTxResult {
string info = 4; // nondeterministic
int64 gas_wanted = 5;
int64 gas_used = 6;
repeated Event events = 7
repeated Event events = 7
[(gogoproto.nullable) = false, (gogoproto.jsontag) = "events,omitempty"]; // nondeterministic
string codespace = 8;
}
@ -393,12 +401,25 @@ message ExecTxResult {
//
// One usage is indexing transaction results.
message TxResult {
int64 height = 1;
uint32 index = 2;
bytes tx = 3;
int64 height = 1;
uint32 index = 2;
bytes tx = 3;
ExecTxResult result = 4 [(gogoproto.nullable) = false];
}
message TxRecord {
TxAction action = 1;
bytes tx = 2;
// TxAction contains App-provided information on what to do with a transaction that is part of a raw proposal
enum TxAction {
UNKNOWN = 0; // Unknown action
UNMODIFIED = 1; // The Application did not modify this transaction.
ADDED = 2; // The Application added this transaction.
REMOVED = 3; // The Application wants this transaction removed from the proposal and the mempool.
}
}
//----------------------------------------
// Blockchain Types
@ -423,6 +444,12 @@ message VoteInfo {
reserved 4; // Placeholder for app_signed_extension in v0.37
}
message ExtendedVoteInfo {
Validator validator = 1 [(gogoproto.nullable) = false];
bool signed_last_block = 2;
bytes vote_extension = 3;
}
enum EvidenceType {
UNKNOWN = 0;
DUPLICATE_VOTE = 1;


+ 2
- 1
test/e2e/app/app.go View File

@ -305,7 +305,8 @@ func (app *Application) ApplySnapshotChunk(req abci.RequestApplySnapshotChunk) a
}
func (app *Application) PrepareProposal(req abci.RequestPrepareProposal) abci.ResponsePrepareProposal {
return abci.ResponsePrepareProposal{BlockData: req.BlockData}
// None of the transactions are modified by this application.
return abci.ResponsePrepareProposal{ModifiedTx: false}
}
// ProcessProposal implements part of the Application interface.


+ 0
- 54
types/results.go View File

@ -1,54 +0,0 @@
package types
import (
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto/merkle"
)
// ABCIResults wraps the deliver tx results to return a proof.
type ABCIResults []*abci.ExecTxResult
// NewResults strips non-deterministic fields from ResponseDeliverTx responses
// and returns ABCIResults.
func NewResults(responses []*abci.ExecTxResult) ABCIResults {
res := make(ABCIResults, len(responses))
for i, d := range responses {
res[i] = deterministicExecTxResult(d)
}
return res
}
// Hash returns a merkle hash of all results.
func (a ABCIResults) Hash() []byte {
return merkle.HashFromByteSlices(a.toByteSlices())
}
// ProveResult returns a merkle proof of one result from the set
func (a ABCIResults) ProveResult(i int) merkle.Proof {
_, proofs := merkle.ProofsFromByteSlices(a.toByteSlices())
return *proofs[i]
}
func (a ABCIResults) toByteSlices() [][]byte {
l := len(a)
bzs := make([][]byte, l)
for i := 0; i < l; i++ {
bz, err := a[i].Marshal()
if err != nil {
panic(err)
}
bzs[i] = bz
}
return bzs
}
// deterministicExecTxResult strips non-deterministic fields from
// ResponseDeliverTx and returns another ResponseDeliverTx.
func deterministicExecTxResult(response *abci.ExecTxResult) *abci.ExecTxResult {
return &abci.ExecTxResult{
Code: response.Code,
Data: response.Data,
GasWanted: response.GasWanted,
GasUsed: response.GasUsed,
}
}

+ 0
- 54
types/results_test.go View File

@ -1,54 +0,0 @@
package types
import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types"
)
func TestABCIResults(t *testing.T) {
a := &abci.ExecTxResult{Code: 0, Data: nil}
b := &abci.ExecTxResult{Code: 0, Data: []byte{}}
c := &abci.ExecTxResult{Code: 0, Data: []byte("one")}
d := &abci.ExecTxResult{Code: 14, Data: nil}
e := &abci.ExecTxResult{Code: 14, Data: []byte("foo")}
f := &abci.ExecTxResult{Code: 14, Data: []byte("bar")}
// Nil and []byte{} should produce the same bytes
bzA, err := a.Marshal()
require.NoError(t, err)
bzB, err := b.Marshal()
require.NoError(t, err)
require.Equal(t, bzA, bzB)
// a and b should be the same, don't go in results.
results := ABCIResults{a, c, d, e, f}
// Make sure each result serializes differently
last := []byte{}
assert.Equal(t, last, bzA) // first one is empty
for i, res := range results[1:] {
bz, err := res.Marshal()
require.NoError(t, err)
assert.NotEqual(t, last, bz, "%d", i)
last = bz
}
// Make sure that we can get a root hash from results and verify proofs.
root := results.Hash()
assert.NotEmpty(t, root)
for i, res := range results {
bz, err := res.Marshal()
require.NoError(t, err)
proof := results.ProveResult(i)
valid := proof.Verify(root, bz)
assert.NoError(t, valid, "%d", i)
}
}

+ 197
- 27
types/tx.go View File

@ -5,7 +5,9 @@ import (
"crypto/sha256"
"errors"
"fmt"
"sort"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto/merkle"
"github.com/tendermint/tendermint/crypto/tmhash"
tmbytes "github.com/tendermint/tendermint/libs/bytes"
@ -32,13 +34,8 @@ type Txs []Tx
// Hash returns the Merkle root hash of the transaction hashes.
// i.e. the leaves of the tree are the hashes of the txs.
func (txs Txs) Hash() []byte {
// These allocations will be removed once Txs is switched to [][]byte,
// ref #2603. This is because golang does not allow type casting slices without unsafe
txBzs := make([][]byte, len(txs))
for i := 0; i < len(txs); i++ {
txBzs[i] = txs[i].Hash()
}
return merkle.HashFromByteSlices(txBzs)
hl := txs.hashList()
return merkle.HashFromByteSlices(hl)
}
// Index returns the index of this transaction in the list, or -1 if not found
@ -61,16 +58,9 @@ func (txs Txs) IndexByHash(hash []byte) int {
return -1
}
// Proof returns a simple merkle proof for this node.
// Panics if i < 0 or i >= len(txs)
// TODO: optimize this!
func (txs Txs) Proof(i int) TxProof {
l := len(txs)
bzs := make([][]byte, l)
for i := 0; i < l; i++ {
bzs[i] = txs[i].Hash()
}
root, proofs := merkle.ProofsFromByteSlices(bzs)
hl := txs.hashList()
root, proofs := merkle.ProofsFromByteSlices(hl)
return TxProof{
RootHash: root,
@ -79,11 +69,23 @@ func (txs Txs) Proof(i int) TxProof {
}
}
func (txs Txs) hashList() [][]byte {
hl := make([][]byte, len(txs))
for i := 0; i < len(txs); i++ {
hl[i] = txs[i].Hash()
}
return hl
}
// Txs is a slice of transactions. Sorting a Txs value orders the transactions
// lexicographically.
func (txs Txs) Len() int { return len(txs) }
func (txs Txs) Swap(i, j int) { txs[i], txs[j] = txs[j], txs[i] }
func (txs Txs) Less(i, j int) bool {
return bytes.Compare(txs[i], txs[j]) == -1
}
// ToSliceOfBytes converts a Txs to slice of byte slices.
//
// NOTE: This method should become obsolete once Txs is switched to [][]byte.
// ref: #2603
// TODO This function is to disappear when TxRecord is introduced
func (txs Txs) ToSliceOfBytes() [][]byte {
txBzs := make([][]byte, len(txs))
for i := 0; i < len(txs); i++ {
@ -92,14 +94,182 @@ func (txs Txs) ToSliceOfBytes() [][]byte {
return txBzs
}
// ToTxs converts a raw slice of byte slices into a Txs type.
// TODO This function is to disappear when TxRecord is introduced
func ToTxs(txs [][]byte) Txs {
txBzs := make(Txs, len(txs))
for i := 0; i < len(txs); i++ {
txBzs[i] = txs[i]
// TxRecordSet contains indexes into an underlying set of transactions.
// These indexes are useful for validating and working with a list of TxRecords
// from the PrepareProposal response.
//
// Only one copy of the original data is referenced by all of the indexes but a
// transaction may appear in multiple indexes.
type TxRecordSet struct {
// all holds the complete list of all transactions from the original list of
// TxRecords.
all Txs
// included is an index of the transactions that will be included in the block
// and is constructed from the list of both added and unmodified transactions.
// included maintains the original order that the transactions were present
// in the list of TxRecords.
included Txs
// added, unmodified, removed, and unknown are indexes for each of the actions
// that may be supplied with a transaction.
//
// Because each transaction only has one action, it can be referenced by
// at most 3 indexes in this data structure: the action-specific index, the
// included index, and the all index.
added Txs
unmodified Txs
removed Txs
unknown Txs
}
// NewTxRecordSet constructs a new set from the given transaction records.
// The contents of the input transactions are shared by the set, and must not
// be modified during the lifetime of the set.
func NewTxRecordSet(trs []*abci.TxRecord) TxRecordSet {
txrSet := TxRecordSet{
all: make([]Tx, len(trs)),
}
return txBzs
for i, tr := range trs {
txrSet.all[i] = Tx(tr.Tx)
// The following set of assignments do not allocate new []byte, they create
// pointers to the already allocated slice.
switch tr.GetAction() {
case abci.TxRecord_UNKNOWN:
txrSet.unknown = append(txrSet.unknown, txrSet.all[i])
case abci.TxRecord_UNMODIFIED:
txrSet.unmodified = append(txrSet.unmodified, txrSet.all[i])
txrSet.included = append(txrSet.included, txrSet.all[i])
case abci.TxRecord_ADDED:
txrSet.added = append(txrSet.added, txrSet.all[i])
txrSet.included = append(txrSet.included, txrSet.all[i])
case abci.TxRecord_REMOVED:
txrSet.removed = append(txrSet.removed, txrSet.all[i])
}
}
return txrSet
}
// IncludedTxs returns the transactions marked for inclusion in a block. This
// list maintains the order that the transactions were included in the list of
// TxRecords that were used to construct the TxRecordSet.
func (t TxRecordSet) IncludedTxs() []Tx {
return t.included
}
// AddedTxs returns the transactions added by the application.
func (t TxRecordSet) AddedTxs() []Tx {
return t.added
}
// RemovedTxs returns the transactions marked for removal by the application.
func (t TxRecordSet) RemovedTxs() []Tx {
return t.removed
}
// Validate checks that the record set was correctly constructed from the original
// list of transactions.
func (t TxRecordSet) Validate(maxSizeBytes int64, otxs Txs) error {
if len(t.unknown) > 0 {
return fmt.Errorf("%d transactions marked unknown (first unknown hash: %x)", len(t.unknown), t.unknown[0].Hash())
}
// The following validation logic performs a set of sorts on the data in the TxRecordSet indexes.
// It sorts the original transaction list, otxs, once.
// It sorts the new transaction list twice: once when sorting 'all', the total list,
// and once by sorting the set of the added, removed, and unmodified transactions indexes,
// which, when combined, comprise the complete list of modified transactions.
//
// Each of the added, removed, and unmodified indices is then iterated and once
// and each value index is checked against the sorted original list for containment.
// Asymptotically, this yields a total runtime of O(N*log(N) + 2*M*log(M) + M*log(N)).
// in the input size of the original list, N, and the input size of the new list, M, respectively.
// Performance gains are likely possible, but this was preferred for readability and maintainability.
// Sort a copy of the complete transaction slice so we can check for
// duplication. The copy is so we do not change the original ordering.
// Only the slices are copied, the transaction contents are shared.
allCopy := sortedCopy(t.all)
var size int64
for i, cur := range allCopy {
size += int64(len(cur))
if size > maxSizeBytes {
return fmt.Errorf("transaction data size %d exceeds maximum %d", size, maxSizeBytes)
}
// allCopy is sorted, so any duplicated data will be adjacent.
if i+1 < len(allCopy) && bytes.Equal(cur, allCopy[i+1]) {
return fmt.Errorf("found duplicate transaction with hash: %x", cur.Hash())
}
}
// create copies of each of the action-specific indexes so that order of the original
// indexes can be preserved.
addedCopy := sortedCopy(t.added)
removedCopy := sortedCopy(t.removed)
unmodifiedCopy := sortedCopy(t.unmodified)
// make a defensive copy of otxs so that the order of
// the caller's data is not altered.
otxsCopy := sortedCopy(otxs)
if ix, ok := containsAll(otxsCopy, unmodifiedCopy); !ok {
return fmt.Errorf("new transaction incorrectly marked as removed, transaction hash: %x", unmodifiedCopy[ix].Hash())
}
if ix, ok := containsAll(otxsCopy, removedCopy); !ok {
return fmt.Errorf("new transaction incorrectly marked as removed, transaction hash: %x", removedCopy[ix].Hash())
}
if ix, ok := containsAny(otxsCopy, addedCopy); ok {
return fmt.Errorf("existing transaction incorrectly marked as added, transaction hash: %x", addedCopy[ix].Hash())
}
return nil
}
func sortedCopy(txs Txs) Txs {
cp := make(Txs, len(txs))
copy(cp, txs)
sort.Sort(cp)
return cp
}
// containsAny checks that list a contains one of the transactions in list
// b. If a match is found, the index in b of the matching transaction is returned.
// Both lists must be sorted.
func containsAny(a, b []Tx) (int, bool) {
for i, cur := range b {
if _, ok := contains(a, cur); ok {
return i, true
}
}
return -1, false
}
// containsAll checks that super contains all of the transactions in the sub
// list. If not all values in sub are present in super, the index in sub of the
// first Tx absent from super is returned.
func containsAll(super, sub Txs) (int, bool) {
for i, cur := range sub {
if _, ok := contains(super, cur); !ok {
return i, false
}
}
return -1, true
}
// contains checks that the sorted list, set contains elem. If set does contain elem, then the
// index in set of elem is returned.
func contains(set []Tx, elem Tx) (int, bool) {
n := sort.Search(len(set), func(i int) bool {
return bytes.Compare(elem, set[i]) <= 0
})
if n == len(set) || !bytes.Equal(elem, set[n]) {
return -1, false
}
return n, true
}
// TxProof represents a Merkle proof of the presence of a transaction in the Merkle tree.


+ 160
- 6
types/tx_test.go View File

@ -2,12 +2,13 @@ package types
import (
"bytes"
mrand "math/rand"
"math/rand"
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types"
ctest "github.com/tendermint/tendermint/internal/libs/test"
tmrand "github.com/tendermint/tendermint/libs/rand"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
@ -21,11 +22,6 @@ func makeTxs(cnt, size int) Txs {
return txs
}
func randInt(low, high int) int {
off := mrand.Int() % (high - low)
return low + off
}
func TestTxIndex(t *testing.T) {
for i := 0; i < 20; i++ {
txs := makeTxs(15, 60)
@ -52,6 +48,160 @@ func TestTxIndexByHash(t *testing.T) {
}
}
func TestValidateTxRecordSet(t *testing.T) {
t.Run("should error on total transaction size exceeding max data size", func(t *testing.T) {
trs := []*abci.TxRecord{
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{1, 2, 3, 4, 5}),
},
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{6, 7, 8, 9, 10}),
},
}
txrSet := NewTxRecordSet(trs)
err := txrSet.Validate(9, []Tx{})
require.Error(t, err)
})
t.Run("should error on duplicate transactions with the same action", func(t *testing.T) {
trs := []*abci.TxRecord{
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{1, 2, 3, 4, 5}),
},
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{100}),
},
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{1, 2, 3, 4, 5}),
},
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{200}),
},
}
txrSet := NewTxRecordSet(trs)
err := txrSet.Validate(100, []Tx{})
require.Error(t, err)
})
t.Run("should error on duplicate transactions with mixed actions", func(t *testing.T) {
trs := []*abci.TxRecord{
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{1, 2, 3, 4, 5}),
},
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{100}),
},
{
Action: abci.TxRecord_REMOVED,
Tx: Tx([]byte{1, 2, 3, 4, 5}),
},
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{200}),
},
}
txrSet := NewTxRecordSet(trs)
err := txrSet.Validate(100, []Tx{})
require.Error(t, err)
})
t.Run("should error on new transactions marked UNMODIFIED", func(t *testing.T) {
trs := []*abci.TxRecord{
{
Action: abci.TxRecord_UNMODIFIED,
Tx: Tx([]byte{1, 2, 3, 4, 5}),
},
}
txrSet := NewTxRecordSet(trs)
err := txrSet.Validate(100, []Tx{})
require.Error(t, err)
})
t.Run("should error on new transactions marked REMOVED", func(t *testing.T) {
trs := []*abci.TxRecord{
{
Action: abci.TxRecord_REMOVED,
Tx: Tx([]byte{1, 2, 3, 4, 5}),
},
}
txrSet := NewTxRecordSet(trs)
err := txrSet.Validate(100, []Tx{})
require.Error(t, err)
})
t.Run("should error on existing transaction marked as ADDED", func(t *testing.T) {
trs := []*abci.TxRecord{
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{5, 4, 3, 2, 1}),
},
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{6}),
},
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{1, 2, 3, 4, 5}),
},
}
txrSet := NewTxRecordSet(trs)
err := txrSet.Validate(100, []Tx{{0}, {1, 2, 3, 4, 5}})
require.Error(t, err)
})
t.Run("should error if any transaction marked as UNKNOWN", func(t *testing.T) {
trs := []*abci.TxRecord{
{
Action: abci.TxRecord_UNKNOWN,
Tx: Tx([]byte{1, 2, 3, 4, 5}),
},
}
txrSet := NewTxRecordSet(trs)
err := txrSet.Validate(100, []Tx{})
require.Error(t, err)
})
t.Run("TxRecordSet preserves order", func(t *testing.T) {
trs := []*abci.TxRecord{
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{100}),
},
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{99}),
},
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{55}),
},
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{12}),
},
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{66}),
},
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{9}),
},
{
Action: abci.TxRecord_ADDED,
Tx: Tx([]byte{17}),
},
}
txrSet := NewTxRecordSet(trs)
err := txrSet.Validate(100, []Tx{})
require.NoError(t, err)
for i, tx := range txrSet.IncludedTxs() {
require.Equal(t, Tx(trs[i].Tx), tx)
}
})
}
func TestValidTxProof(t *testing.T) {
cases := []struct {
txs Txs
@ -150,3 +300,7 @@ func assertBadProof(t *testing.T, root []byte, bad []byte, good TxProof) {
}
}
}
func randInt(low, high int) int {
return rand.Intn(high-low) + low
}

+ 3
- 0
types/vote_set.go View File

@ -227,6 +227,9 @@ func (voteSet *VoteSet) getVote(valIndex int32, blockKey string) (vote *Vote, ok
}
func (voteSet *VoteSet) GetVotes() []*Vote {
if voteSet == nil {
return nil
}
return voteSet.votes
}


Loading…
Cancel
Save