mirror of
https://github.com/tendermint/tendermint.git
synced 2026-01-14 08:42:48 +00:00
Compare commits
2 Commits
cal/proces
...
wb/abci++-
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
d25198288a | ||
|
|
f861062ee2 |
@@ -43,6 +43,7 @@ type Client interface {
|
||||
OfferSnapshotAsync(types.RequestOfferSnapshot) *ReqRes
|
||||
LoadSnapshotChunkAsync(types.RequestLoadSnapshotChunk) *ReqRes
|
||||
ApplySnapshotChunkAsync(types.RequestApplySnapshotChunk) *ReqRes
|
||||
ProcessProposalAsync(types.RequestProcessProposal) *ReqRes
|
||||
|
||||
FlushSync() error
|
||||
EchoSync(msg string) (*types.ResponseEcho, error)
|
||||
@@ -60,6 +61,7 @@ type Client interface {
|
||||
OfferSnapshotSync(types.RequestOfferSnapshot) (*types.ResponseOfferSnapshot, error)
|
||||
LoadSnapshotChunkSync(types.RequestLoadSnapshotChunk) (*types.ResponseLoadSnapshotChunk, error)
|
||||
ApplySnapshotChunkSync(types.RequestApplySnapshotChunk) (*types.ResponseApplySnapshotChunk, error)
|
||||
ProcessProposalSync(types.RequestProcessProposal) (*types.ResponseProcessProposal, error)
|
||||
}
|
||||
|
||||
//----------------------------------------
|
||||
|
||||
@@ -309,6 +309,16 @@ func (cli *grpcClient) PrepareProposalAsync(params types.RequestPrepareProposal)
|
||||
return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_PrepareProposal{PrepareProposal: res}})
|
||||
}
|
||||
|
||||
func (cli *grpcClient) ProcessProposalAsync(params types.RequestProcessProposal) *ReqRes {
|
||||
req := types.ToRequestProcessProposal(params)
|
||||
res, err := cli.client.ProcessProposal(context.Background(), req.GetProcessProposal(), grpc.WaitForReady(true))
|
||||
if err != nil {
|
||||
cli.StopForError(err)
|
||||
}
|
||||
|
||||
return cli.finishAsyncCall(req, &types.Response{Value: &types.Response_ProcessProposal{ProcessProposal: res}})
|
||||
}
|
||||
|
||||
// finishAsyncCall creates a ReqRes for an async call, and immediately populates it
|
||||
// with the response. We don't complete it until it's been ordered via the channel.
|
||||
func (cli *grpcClient) finishAsyncCall(req *types.Request, res *types.Response) *ReqRes {
|
||||
@@ -432,3 +442,8 @@ func (cli *grpcClient) PrepareProposalSync(
|
||||
reqres := cli.PrepareProposalAsync(params)
|
||||
return cli.finishSyncCall(reqres).GetPrepareProposal(), cli.Error()
|
||||
}
|
||||
|
||||
func (cli *grpcClient) ProcessProposalSync(params types.RequestProcessProposal) (*types.ResponseProcessProposal, error) {
|
||||
reqres := cli.ProcessProposalAsync(params)
|
||||
return cli.finishSyncCall(reqres).GetProcessProposal(), cli.Error()
|
||||
}
|
||||
|
||||
@@ -218,6 +218,17 @@ func (app *localClient) PrepareProposalAsync(req types.RequestPrepareProposal) *
|
||||
)
|
||||
}
|
||||
|
||||
func (app *localClient) ProcessProposalAsync(req types.RequestProcessProposal) *ReqRes {
|
||||
app.mtx.Lock()
|
||||
defer app.mtx.Unlock()
|
||||
|
||||
res := app.Application.ProcessProposal(req)
|
||||
return app.callback(
|
||||
types.ToRequestProcessProposal(req),
|
||||
types.ToResponseProcessProposal(res),
|
||||
)
|
||||
}
|
||||
|
||||
//-------------------------------------------------------
|
||||
|
||||
func (app *localClient) FlushSync() error {
|
||||
@@ -342,6 +353,14 @@ func (app *localClient) PrepareProposalSync(req types.RequestPrepareProposal) (*
|
||||
return &res, nil
|
||||
}
|
||||
|
||||
func (app *localClient) ProcessProposalSync(req types.RequestProcessProposal) (*types.ResponseProcessProposal, error) {
|
||||
app.mtx.Lock()
|
||||
defer app.mtx.Unlock()
|
||||
|
||||
res := app.Application.ProcessProposal(req)
|
||||
return &res, nil
|
||||
}
|
||||
|
||||
//-------------------------------------------------------
|
||||
|
||||
func (app *localClient) callback(req *types.Request, res *types.Response) *ReqRes {
|
||||
|
||||
@@ -614,6 +614,45 @@ func (_m *Client) PrepareProposalSync(_a0 types.RequestPrepareProposal) (*types.
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// ProcessProposalAsync provides a mock function with given fields: _a0
|
||||
func (_m *Client) ProcessProposalAsync(_a0 types.RequestProcessProposal) *abcicli.ReqRes {
|
||||
ret := _m.Called(_a0)
|
||||
|
||||
var r0 *abcicli.ReqRes
|
||||
if rf, ok := ret.Get(0).(func(types.RequestProcessProposal) *abcicli.ReqRes); ok {
|
||||
r0 = rf(_a0)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*abcicli.ReqRes)
|
||||
}
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// ProcessProposalSync provides a mock function with given fields: _a0
|
||||
func (_m *Client) ProcessProposalSync(_a0 types.RequestProcessProposal) (*types.ResponseProcessProposal, error) {
|
||||
ret := _m.Called(_a0)
|
||||
|
||||
var r0 *types.ResponseProcessProposal
|
||||
if rf, ok := ret.Get(0).(func(types.RequestProcessProposal) *types.ResponseProcessProposal); ok {
|
||||
r0 = rf(_a0)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*types.ResponseProcessProposal)
|
||||
}
|
||||
}
|
||||
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(1).(func(types.RequestProcessProposal) error); ok {
|
||||
r1 = rf(_a0)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// QueryAsync provides a mock function with given fields: _a0
|
||||
func (_m *Client) QueryAsync(_a0 types.RequestQuery) *abcicli.ReqRes {
|
||||
ret := _m.Called(_a0)
|
||||
|
||||
@@ -283,6 +283,10 @@ func (cli *socketClient) PrepareProposalAsync(req types.RequestPrepareProposal)
|
||||
return cli.queueRequest(types.ToRequestPrepareProposal(req))
|
||||
}
|
||||
|
||||
func (cli *socketClient) ProcessProposalAsync(req types.RequestProcessProposal) *ReqRes {
|
||||
return cli.queueRequest(types.ToRequestProcessProposal(req))
|
||||
}
|
||||
|
||||
//----------------------------------------
|
||||
|
||||
func (cli *socketClient) FlushSync() error {
|
||||
@@ -430,6 +434,15 @@ func (cli *socketClient) PrepareProposalSync(req types.RequestPrepareProposal) (
|
||||
return reqres.Response.GetPrepareProposal(), cli.Error()
|
||||
}
|
||||
|
||||
func (cli *socketClient) ProcessProposalSync(req types.RequestProcessProposal) (*types.ResponseProcessProposal, error) {
|
||||
reqres := cli.queueRequest(types.ToRequestProcessProposal(req))
|
||||
if err := cli.FlushSync(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return reqres.Response.GetProcessProposal(), cli.Error()
|
||||
}
|
||||
|
||||
//----------------------------------------
|
||||
|
||||
func (cli *socketClient) queueRequest(req *types.Request) *ReqRes {
|
||||
@@ -507,6 +520,8 @@ func resMatchesReq(req *types.Request, res *types.Response) (ok bool) {
|
||||
_, ok = res.Value.(*types.Response_OfferSnapshot)
|
||||
case *types.Request_PrepareProposal:
|
||||
_, ok = res.Value.(*types.Response_PrepareProposal)
|
||||
case *types.Request_ProcessProposal:
|
||||
_, ok = res.Value.(*types.Response_ProcessProposal)
|
||||
}
|
||||
return ok
|
||||
}
|
||||
|
||||
@@ -170,3 +170,13 @@ func (app *Application) Query(reqQuery types.RequestQuery) (resQuery types.Respo
|
||||
|
||||
return resQuery
|
||||
}
|
||||
|
||||
func (app *Application) ProcessProposal(
|
||||
req types.RequestProcessProposal) types.ResponseProcessProposal {
|
||||
for _, tx := range req.Txs {
|
||||
if len(tx) == 0 {
|
||||
return types.ResponseProcessProposal{Status: types.ResponseProcessProposal_REJECT}
|
||||
}
|
||||
}
|
||||
return types.ResponseProcessProposal{Status: types.ResponseProcessProposal_ACCEPT}
|
||||
}
|
||||
|
||||
@@ -179,6 +179,16 @@ func (app *PersistentKVStoreApplication) PrepareProposal(
|
||||
return types.ResponsePrepareProposal{TxRecords: app.substPrepareTx(req.Txs, req.MaxTxBytes)}
|
||||
}
|
||||
|
||||
func (app *PersistentKVStoreApplication) ProcessProposal(
|
||||
req types.RequestProcessProposal) types.ResponseProcessProposal {
|
||||
for _, tx := range req.Txs {
|
||||
if len(tx) == 0 {
|
||||
return types.ResponseProcessProposal{Status: types.ResponseProcessProposal_REJECT}
|
||||
}
|
||||
}
|
||||
return types.ResponseProcessProposal{Status: types.ResponseProcessProposal_ACCEPT}
|
||||
}
|
||||
|
||||
//---------------------------------------------
|
||||
// update validators
|
||||
|
||||
|
||||
@@ -233,6 +233,9 @@ func (s *SocketServer) handleRequest(req *types.Request, responses chan<- *types
|
||||
case *types.Request_PrepareProposal:
|
||||
res := s.app.PrepareProposal(*r.PrepareProposal)
|
||||
responses <- types.ToResponsePrepareProposal(res)
|
||||
case *types.Request_ProcessProposal:
|
||||
res := s.app.ProcessProposal(*r.ProcessProposal)
|
||||
responses <- types.ToResponseProcessProposal(res)
|
||||
case *types.Request_LoadSnapshotChunk:
|
||||
res := s.app.LoadSnapshotChunk(*r.LoadSnapshotChunk)
|
||||
responses <- types.ToResponseLoadSnapshotChunk(res)
|
||||
|
||||
@@ -4,7 +4,8 @@ import (
|
||||
context "golang.org/x/net/context"
|
||||
)
|
||||
|
||||
//go:generate mockery --case underscore --name Application
|
||||
//go:generate ../../scripts/mockery_generate.sh Application
|
||||
|
||||
// Application is an interface that enables any finite, deterministic state machine
|
||||
// to be driven by a blockchain-based replication engine via the ABCI.
|
||||
// All methods take a RequestXxx argument and return a ResponseXxx argument,
|
||||
@@ -21,6 +22,7 @@ type Application interface {
|
||||
// Consensus Connection
|
||||
InitChain(RequestInitChain) ResponseInitChain // Initialize blockchain w validators/other info from TendermintCore
|
||||
PrepareProposal(RequestPrepareProposal) ResponsePrepareProposal
|
||||
ProcessProposal(RequestProcessProposal) ResponseProcessProposal
|
||||
BeginBlock(RequestBeginBlock) ResponseBeginBlock // Signals the beginning of a block
|
||||
DeliverTx(RequestDeliverTx) ResponseDeliverTx // Deliver a tx for full processing
|
||||
EndBlock(RequestEndBlock) ResponseEndBlock // Signals the end of a block, returns changes to the validator set
|
||||
@@ -113,6 +115,11 @@ func (BaseApplication) PrepareProposal(req RequestPrepareProposal) ResponsePrepa
|
||||
return ResponsePrepareProposal{TxRecords: trs}
|
||||
}
|
||||
|
||||
func (BaseApplication) ProcessProposal(req RequestProcessProposal) ResponseProcessProposal {
|
||||
return ResponseProcessProposal{
|
||||
Status: ResponseProcessProposal_ACCEPT}
|
||||
}
|
||||
|
||||
//-------------------------------------------------------
|
||||
|
||||
// GRPCApplication is a GRPC wrapper for Application
|
||||
@@ -206,3 +213,9 @@ func (app *GRPCApplication) PrepareProposal(
|
||||
res := app.app.PrepareProposal(*req)
|
||||
return &res, nil
|
||||
}
|
||||
|
||||
func (app *GRPCApplication) ProcessProposal(
|
||||
ctx context.Context, req *RequestProcessProposal) (*ResponseProcessProposal, error) {
|
||||
res := app.app.ProcessProposal(*req)
|
||||
return &res, nil
|
||||
}
|
||||
|
||||
@@ -165,6 +165,12 @@ func ToRequestPrepareProposal(req RequestPrepareProposal) *Request {
|
||||
}
|
||||
}
|
||||
|
||||
func ToRequestProcessProposal(req RequestProcessProposal) *Request {
|
||||
return &Request{
|
||||
Value: &Request_ProcessProposal{&req},
|
||||
}
|
||||
}
|
||||
|
||||
//----------------------------------------
|
||||
|
||||
func ToResponseException(errStr string) *Response {
|
||||
@@ -268,3 +274,9 @@ func ToResponsePrepareProposal(res ResponsePrepareProposal) *Response {
|
||||
Value: &Response_PrepareProposal{&res},
|
||||
}
|
||||
}
|
||||
|
||||
func ToResponseProcessProposal(res ResponseProcessProposal) *Response {
|
||||
return &Response{
|
||||
Value: &Response_ProcessProposal{&res},
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
// Code generated by mockery v2.14.0. DO NOT EDIT.
|
||||
// Code generated by mockery. DO NOT EDIT.
|
||||
|
||||
package mocks
|
||||
|
||||
@@ -180,6 +180,20 @@ func (_m *Application) PrepareProposal(_a0 types.RequestPrepareProposal) types.R
|
||||
return r0
|
||||
}
|
||||
|
||||
// ProcessProposal provides a mock function with given fields: _a0
|
||||
func (_m *Application) ProcessProposal(_a0 types.RequestProcessProposal) types.ResponseProcessProposal {
|
||||
ret := _m.Called(_a0)
|
||||
|
||||
var r0 types.ResponseProcessProposal
|
||||
if rf, ok := ret.Get(0).(func(types.RequestProcessProposal) types.ResponseProcessProposal); ok {
|
||||
r0 = rf(_a0)
|
||||
} else {
|
||||
r0 = ret.Get(0).(types.ResponseProcessProposal)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// Query provides a mock function with given fields: _a0
|
||||
func (_m *Application) Query(_a0 types.RequestQuery) types.ResponseQuery {
|
||||
ret := _m.Called(_a0)
|
||||
|
||||
@@ -84,6 +84,17 @@ func (m BaseMock) PrepareProposal(input types.RequestPrepareProposal) types.Resp
|
||||
return ret
|
||||
}
|
||||
|
||||
func (m BaseMock) ProcessProposal(input types.RequestProcessProposal) types.ResponseProcessProposal {
|
||||
var ret types.ResponseProcessProposal
|
||||
defer func() {
|
||||
if r := recover(); r != nil {
|
||||
ret = m.base.ProcessProposal(input)
|
||||
}
|
||||
}()
|
||||
ret = m.Application.ProcessProposal(input)
|
||||
return ret
|
||||
}
|
||||
|
||||
// Commit the state and return the application Merkle root hash
|
||||
func (m BaseMock) Commit() types.ResponseCommit {
|
||||
var ret types.ResponseCommit
|
||||
|
||||
@@ -41,6 +41,16 @@ func (r ResponseQuery) IsErr() bool {
|
||||
return r.Code != CodeTypeOK
|
||||
}
|
||||
|
||||
// IsAccepted returns true if Code is ACCEPT
|
||||
func (r ResponseProcessProposal) IsAccepted() bool {
|
||||
return r.Status == ResponseProcessProposal_ACCEPT
|
||||
}
|
||||
|
||||
// IsStatusUnknown returns true if Code is UNKNOWN
|
||||
func (r ResponseProcessProposal) IsStatusUnknown() bool {
|
||||
return r.Status == ResponseProcessProposal_UNKNOWN
|
||||
}
|
||||
|
||||
//---------------------------------------------------------------------------
|
||||
// override JSON marshaling so we emit defaults (ie. disable omitempty)
|
||||
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@@ -13,6 +13,7 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/go-kit/log/term"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"path"
|
||||
@@ -462,12 +463,16 @@ func loadPrivValidator(config *cfg.Config) *privval.FilePV {
|
||||
}
|
||||
|
||||
func randState(nValidators int) (*State, []*validatorStub) {
|
||||
return randStateWithApp(nValidators, counter.NewApplication(true))
|
||||
}
|
||||
|
||||
func randStateWithApp(nValidators int, app abci.Application) (*State, []*validatorStub) {
|
||||
// Get State
|
||||
state, privVals := randGenesisState(nValidators, false, 10)
|
||||
|
||||
vss := make([]*validatorStub, nValidators)
|
||||
|
||||
cs := newState(state, privVals[0], counter.NewApplication(true))
|
||||
cs := newState(state, privVals[0], app)
|
||||
|
||||
for i := 0; i < nValidators; i++ {
|
||||
vss[i] = newValidatorStub(privVals[i], int32(i))
|
||||
@@ -682,6 +687,33 @@ func ensureVote(voteCh <-chan tmpubsub.Message, height int64, round int32,
|
||||
}
|
||||
}
|
||||
|
||||
func ensurePrevoteMatch(t *testing.T, voteCh <-chan tmpubsub.Message, height int64, round int32, hash []byte) {
|
||||
t.Helper()
|
||||
ensureVoteMatch(t, voteCh, height, round, hash, tmproto.PrevoteType)
|
||||
}
|
||||
|
||||
func ensureVoteMatch(t *testing.T, voteCh <-chan tmpubsub.Message, height int64, round int32, hash []byte, voteType tmproto.SignedMsgType) {
|
||||
t.Helper()
|
||||
select {
|
||||
case <-time.After(ensureTimeout):
|
||||
t.Fatal("Timeout expired while waiting for NewVote event")
|
||||
case msg := <-voteCh:
|
||||
voteEvent, ok := msg.Data().(types.EventDataVote)
|
||||
require.True(t, ok, "expected a EventDataVote, got %T. Wrong subscription channel?",
|
||||
msg.Data())
|
||||
|
||||
vote := voteEvent.Vote
|
||||
assert.Equal(t, height, vote.Height, "expected height %d, but got %d", height, vote.Height)
|
||||
assert.Equal(t, round, vote.Round, "expected round %d, but got %d", round, vote.Round)
|
||||
assert.Equal(t, voteType, vote.Type, "expected type %s, but got %s", voteType, vote.Type)
|
||||
if hash == nil {
|
||||
require.Nil(t, vote.BlockID.Hash, "Expected prevote to be for nil, got %X", vote.BlockID.Hash)
|
||||
} else {
|
||||
require.True(t, bytes.Equal(vote.BlockID.Hash, hash), "Expected prevote to be for %X, got %X", hash, vote.BlockID.Hash)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func ensurePrecommitTimeout(ch <-chan tmpubsub.Message) {
|
||||
select {
|
||||
case <-time.After(ensureTimeout):
|
||||
|
||||
@@ -274,3 +274,8 @@ func (app *CounterApplication) PrepareProposal(
|
||||
}
|
||||
return abci.ResponsePrepareProposal{TxRecords: trs}
|
||||
}
|
||||
|
||||
func (app *CounterApplication) ProcessProposal(
|
||||
req abci.RequestProcessProposal) abci.ResponseProcessProposal {
|
||||
return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT}
|
||||
}
|
||||
|
||||
@@ -1278,11 +1278,36 @@ func (cs *State) defaultDoPrevote(height int64, round int32) {
|
||||
return
|
||||
}
|
||||
|
||||
// Validate proposal block
|
||||
// Validate proposal block, from Tendermint's perspective
|
||||
err := cs.blockExec.ValidateBlock(cs.state, cs.ProposalBlock)
|
||||
if err != nil {
|
||||
// ProposalBlock is invalid, prevote nil.
|
||||
logger.Error("prevote step: ProposalBlock is invalid", "err", err)
|
||||
logger.Error("prevote step: consensus deems this block invalid; prevoting nil",
|
||||
"err", err)
|
||||
cs.signAddVote(tmproto.PrevoteType, nil, types.PartSetHeader{})
|
||||
return
|
||||
}
|
||||
|
||||
/*
|
||||
Before prevoting on the block received from the proposer for the current round and height,
|
||||
we request the Application, via `ProcessProposal` ABCI call, to confirm that the block is
|
||||
valid. If the Application does not accept the block, Tendermint prevotes `nil`.
|
||||
|
||||
WARNING: misuse of block rejection by the Application can seriously compromise Tendermint's
|
||||
liveness properties. Please see `PrepareProosal`-`ProcessProposal` coherence and determinism
|
||||
properties in the ABCI++ specification.
|
||||
*/
|
||||
isAppValid, err := cs.blockExec.ProcessProposal(cs.ProposalBlock, cs.state)
|
||||
if err != nil {
|
||||
panic(fmt.Sprintf(
|
||||
"state machine returned an error (%v) when calling ProcessProposal", err,
|
||||
))
|
||||
}
|
||||
|
||||
// Vote nil if the Application rejected the block
|
||||
if !isAppValid {
|
||||
logger.Error("prevote step: state machine rejected a proposed block; this should not happen:"+
|
||||
"the proposer may be misbehaving; prevoting nil", "err", err)
|
||||
cs.signAddVote(tmproto.PrevoteType, nil, types.PartSetHeader{})
|
||||
return
|
||||
}
|
||||
|
||||
@@ -8,11 +8,15 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/tendermint/tendermint/abci/example/counter"
|
||||
abci "github.com/tendermint/tendermint/abci/types"
|
||||
abcimocks "github.com/tendermint/tendermint/abci/types/mocks"
|
||||
cstypes "github.com/tendermint/tendermint/consensus/types"
|
||||
"github.com/tendermint/tendermint/crypto/tmhash"
|
||||
tmbytes "github.com/tendermint/tendermint/libs/bytes"
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
tmpubsub "github.com/tendermint/tendermint/libs/pubsub"
|
||||
tmrand "github.com/tendermint/tendermint/libs/rand"
|
||||
@@ -1368,6 +1372,55 @@ func TestSetValidBlockOnDelayedProposal(t *testing.T) {
|
||||
assert.True(t, rs.ValidRound == round)
|
||||
}
|
||||
|
||||
func TestProcessProposalAccept(t *testing.T) {
|
||||
for _, testCase := range []struct {
|
||||
name string
|
||||
accept bool
|
||||
expectedNilPrevote bool
|
||||
}{
|
||||
{
|
||||
name: "accepted block is prevoted",
|
||||
accept: true,
|
||||
expectedNilPrevote: false,
|
||||
},
|
||||
{
|
||||
name: "rejected block is not prevoted",
|
||||
accept: false,
|
||||
expectedNilPrevote: true,
|
||||
},
|
||||
} {
|
||||
t.Run(testCase.name, func(t *testing.T) {
|
||||
m := abcimocks.NewApplication(t)
|
||||
status := abci.ResponseProcessProposal_REJECT
|
||||
if testCase.accept {
|
||||
status = abci.ResponseProcessProposal_ACCEPT
|
||||
}
|
||||
m.On("ProcessProposal", mock.Anything).Return(abci.ResponseProcessProposal{Status: status})
|
||||
m.On("PrepareProposal", mock.Anything).Return(abci.ResponsePrepareProposal{}).Maybe()
|
||||
cs1, _ := randStateWithApp(4, m)
|
||||
height, round := cs1.Height, cs1.Round
|
||||
|
||||
proposalCh := subscribe(cs1.eventBus, types.EventQueryCompleteProposal)
|
||||
newRoundCh := subscribe(cs1.eventBus, types.EventQueryNewRound)
|
||||
pv1, err := cs1.privValidator.GetPubKey()
|
||||
require.NoError(t, err)
|
||||
addr := pv1.Address()
|
||||
voteCh := subscribeToVoter(cs1, addr)
|
||||
|
||||
startTestRound(cs1, cs1.Height, round)
|
||||
ensureNewRound(newRoundCh, height, round)
|
||||
|
||||
ensureNewProposal(proposalCh, height, round)
|
||||
rs := cs1.GetRoundState()
|
||||
var prevoteHash tmbytes.HexBytes
|
||||
if !testCase.expectedNilPrevote {
|
||||
prevoteHash = rs.ProposalBlock.Hash()
|
||||
}
|
||||
ensurePrevoteMatch(t, voteCh, height, round, prevoteHash)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// 4 vals, 3 Nil Precommits at P0
|
||||
// What we want:
|
||||
// P0 waits for timeoutPrecommit before starting next round
|
||||
|
||||
@@ -80,6 +80,15 @@ type RoundState struct {
|
||||
LockedBlock *types.Block `json:"locked_block"`
|
||||
LockedBlockParts *types.PartSet `json:"locked_block_parts"`
|
||||
|
||||
// The variables below starting with "Valid..." derive their name from
|
||||
// the algorithm presented in this paper:
|
||||
// [The latest gossip on BFT consensus](https://arxiv.org/abs/1807.04938).
|
||||
// Therefore, "Valid...":
|
||||
// * means that the block or round that the variable refers to has
|
||||
// received 2/3+ non-`nil` prevotes (a.k.a. a *polka*)
|
||||
// * has nothing to do with whether the Application returned "Accept" in its
|
||||
// response to `ProcessProposal`, or "Reject"
|
||||
|
||||
// Last known round with POL for non-nil valid block.
|
||||
ValidRound int32 `json:"valid_round"`
|
||||
ValidBlock *types.Block `json:"valid_block"` // Last known block of POL mentioned above.
|
||||
@@ -186,8 +195,8 @@ func (rs *RoundState) StringIndented(indent string) string {
|
||||
%s ProposalBlock: %v %v
|
||||
%s LockedRound: %v
|
||||
%s LockedBlock: %v %v
|
||||
%s ValidRound: %v
|
||||
%s ValidBlock: %v %v
|
||||
%s ValidRound: %v
|
||||
%s ValidBlock: %v %v
|
||||
%s Votes: %v
|
||||
%s LastCommit: %v
|
||||
%s LastValidators:%v
|
||||
|
||||
@@ -406,7 +406,6 @@ 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 := sf.MakeBlock(state, i, lastCommit)
|
||||
|
||||
block.Header.Time = defaultEvidenceTime.Add(time.Duration(i) * time.Minute)
|
||||
block.Header.Version = tmversion.Consensus{Block: version.BlockProtocol, App: 1}
|
||||
const parts = 1
|
||||
|
||||
92
internal/test/block.go
Normal file
92
internal/test/block.go
Normal file
@@ -0,0 +1,92 @@
|
||||
package factory
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/tendermint/tendermint/crypto"
|
||||
"github.com/tendermint/tendermint/crypto/tmhash"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
"github.com/tendermint/tendermint/version"
|
||||
)
|
||||
|
||||
const (
|
||||
DefaultTestChainID = "test-chain"
|
||||
)
|
||||
|
||||
var (
|
||||
DefaultTestTime = time.Date(2020, 1, 1, 0, 0, 0, 0, time.UTC)
|
||||
)
|
||||
|
||||
func RandomAddress() []byte {
|
||||
return crypto.CRandBytes(crypto.AddressSize)
|
||||
}
|
||||
|
||||
func RandomHash() []byte {
|
||||
return crypto.CRandBytes(tmhash.Size)
|
||||
}
|
||||
|
||||
func MakeBlockID() types.BlockID {
|
||||
return MakeBlockIDWithHash(RandomHash())
|
||||
}
|
||||
|
||||
func MakeBlockIDWithHash(hash []byte) types.BlockID {
|
||||
return types.BlockID{
|
||||
Hash: hash,
|
||||
PartSetHeader: types.PartSetHeader{
|
||||
Total: 100,
|
||||
Hash: RandomHash(),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// MakeHeader fills the rest of the contents of the header such that it passes
|
||||
// validate basic
|
||||
func MakeHeader(t *testing.T, h *types.Header) *types.Header {
|
||||
t.Helper()
|
||||
if h.Version.Block == 0 {
|
||||
h.Version.Block = version.BlockProtocol
|
||||
}
|
||||
if h.Height == 0 {
|
||||
h.Height = 1
|
||||
}
|
||||
if h.LastBlockID.IsZero() {
|
||||
h.LastBlockID = MakeBlockID()
|
||||
}
|
||||
if h.ChainID == "" {
|
||||
h.ChainID = DefaultTestChainID
|
||||
}
|
||||
if len(h.LastCommitHash) == 0 {
|
||||
h.LastCommitHash = RandomHash()
|
||||
}
|
||||
if len(h.DataHash) == 0 {
|
||||
h.DataHash = RandomHash()
|
||||
}
|
||||
if len(h.ValidatorsHash) == 0 {
|
||||
h.ValidatorsHash = RandomHash()
|
||||
}
|
||||
if len(h.NextValidatorsHash) == 0 {
|
||||
h.NextValidatorsHash = RandomHash()
|
||||
}
|
||||
if len(h.ConsensusHash) == 0 {
|
||||
h.ConsensusHash = RandomHash()
|
||||
}
|
||||
if len(h.AppHash) == 0 {
|
||||
h.AppHash = RandomHash()
|
||||
}
|
||||
if len(h.LastResultsHash) == 0 {
|
||||
h.LastResultsHash = RandomHash()
|
||||
}
|
||||
if len(h.EvidenceHash) == 0 {
|
||||
h.EvidenceHash = RandomHash()
|
||||
}
|
||||
if len(h.ProposerAddress) == 0 {
|
||||
h.ProposerAddress = RandomAddress()
|
||||
}
|
||||
|
||||
require.NoError(t, h.ValidateBasic())
|
||||
|
||||
return h
|
||||
}
|
||||
6
internal/test/doc.go
Normal file
6
internal/test/doc.go
Normal file
@@ -0,0 +1,6 @@
|
||||
/*
|
||||
Package factory provides generation code for common structs in Tendermint.
|
||||
It is used primarily for the testing of internal components such as statesync,
|
||||
consensus, blocksync etc..
|
||||
*/
|
||||
package factory
|
||||
11
internal/test/factory_test.go
Normal file
11
internal/test/factory_test.go
Normal file
@@ -0,0 +1,11 @@
|
||||
package factory
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
func TestMakeHeader(t *testing.T) {
|
||||
MakeHeader(t, &types.Header{})
|
||||
}
|
||||
34
internal/test/genesis.go
Normal file
34
internal/test/genesis.go
Normal file
@@ -0,0 +1,34 @@
|
||||
package factory
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
cfg "github.com/tendermint/tendermint/config"
|
||||
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
func GenesisDoc(
|
||||
config *cfg.Config,
|
||||
time time.Time,
|
||||
validators []*types.Validator,
|
||||
consensusParams *tmproto.ConsensusParams,
|
||||
) *types.GenesisDoc {
|
||||
|
||||
genesisValidators := make([]types.GenesisValidator, len(validators))
|
||||
|
||||
for i := range validators {
|
||||
genesisValidators[i] = types.GenesisValidator{
|
||||
Power: validators[i].VotingPower,
|
||||
PubKey: validators[i].PubKey,
|
||||
}
|
||||
}
|
||||
|
||||
return &types.GenesisDoc{
|
||||
GenesisTime: time,
|
||||
InitialHeight: 1,
|
||||
ChainID: config.ChainID(),
|
||||
Validators: genesisValidators,
|
||||
ConsensusParams: consensusParams,
|
||||
}
|
||||
}
|
||||
11
internal/test/tx.go
Normal file
11
internal/test/tx.go
Normal file
@@ -0,0 +1,11 @@
|
||||
package factory
|
||||
|
||||
import "github.com/tendermint/tendermint/types"
|
||||
|
||||
func MakeNTxs(height, n int64) []types.Tx {
|
||||
txs := make([]types.Tx, n)
|
||||
for i := range txs {
|
||||
txs[i] = types.Tx([]byte{byte(height), byte(i / 256), byte(i % 256)})
|
||||
}
|
||||
return txs
|
||||
}
|
||||
41
internal/test/validator.go
Normal file
41
internal/test/validator.go
Normal file
@@ -0,0 +1,41 @@
|
||||
package factory
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sort"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
func Validator(ctx context.Context, votingPower int64) (*types.Validator, types.PrivValidator, error) {
|
||||
privVal := types.NewMockPV()
|
||||
pubKey, err := privVal.GetPubKey()
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
val := types.NewValidator(pubKey, votingPower)
|
||||
return val, privVal, nil
|
||||
}
|
||||
|
||||
func ValidatorSet(ctx context.Context, t *testing.T, numValidators int, votingPower int64) (*types.ValidatorSet, []types.PrivValidator) {
|
||||
var (
|
||||
valz = make([]*types.Validator, numValidators)
|
||||
privValidators = make([]types.PrivValidator, numValidators)
|
||||
)
|
||||
t.Helper()
|
||||
|
||||
for i := 0; i < numValidators; i++ {
|
||||
val, privValidator, err := Validator(ctx, votingPower)
|
||||
require.NoError(t, err)
|
||||
valz[i] = val
|
||||
privValidators[i] = privValidator
|
||||
}
|
||||
|
||||
sort.Sort(types.PrivValidatorsByAddress(privValidators))
|
||||
|
||||
return types.NewValidatorSet(valz), privValidators
|
||||
}
|
||||
44
internal/test/vote.go
Normal file
44
internal/test/vote.go
Normal file
@@ -0,0 +1,44 @@
|
||||
package factory
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
func MakeVote(
|
||||
ctx context.Context,
|
||||
val types.PrivValidator,
|
||||
chainID string,
|
||||
valIndex int32,
|
||||
height int64,
|
||||
round int32,
|
||||
step int,
|
||||
blockID types.BlockID,
|
||||
time time.Time,
|
||||
) (*types.Vote, error) {
|
||||
pubKey, err := val.GetPubKey()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
v := &types.Vote{
|
||||
ValidatorAddress: pubKey.Address(),
|
||||
ValidatorIndex: valIndex,
|
||||
Height: height,
|
||||
Round: round,
|
||||
Type: tmproto.SignedMsgType(step),
|
||||
BlockID: blockID,
|
||||
Timestamp: time,
|
||||
}
|
||||
|
||||
vpb := v.ToProto()
|
||||
if err := val.SignVote(chainID, vpb); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
v.Signature = vpb.Signature
|
||||
return v, nil
|
||||
}
|
||||
@@ -37,6 +37,7 @@ message Request {
|
||||
RequestLoadSnapshotChunk load_snapshot_chunk = 14;
|
||||
RequestApplySnapshotChunk apply_snapshot_chunk = 15;
|
||||
RequestPrepareProposal prepare_proposal = 16;
|
||||
RequestProcessProposal process_proposal = 17;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -76,10 +77,10 @@ message RequestQuery {
|
||||
}
|
||||
|
||||
message RequestBeginBlock {
|
||||
bytes hash = 1;
|
||||
tendermint.types.Header header = 2 [(gogoproto.nullable) = false];
|
||||
LastCommitInfo last_commit_info = 3 [(gogoproto.nullable) = false];
|
||||
repeated Evidence byzantine_validators = 4 [(gogoproto.nullable) = false];
|
||||
bytes hash = 1;
|
||||
tendermint.types.Header header = 2 [(gogoproto.nullable) = false];
|
||||
CommitInfo last_commit_info = 3 [(gogoproto.nullable) = false];
|
||||
repeated Evidence byzantine_validators = 4 [(gogoproto.nullable) = false];
|
||||
}
|
||||
|
||||
enum CheckTxType {
|
||||
@@ -137,6 +138,19 @@ message RequestPrepareProposal {
|
||||
int64 max_tx_bytes = 6;
|
||||
}
|
||||
|
||||
message RequestProcessProposal {
|
||||
repeated bytes txs = 1;
|
||||
CommitInfo proposed_last_commit = 2 [(gogoproto.nullable) = false];
|
||||
repeated Evidence misbehavior = 3 [(gogoproto.nullable) = false];
|
||||
// hash is the merkle root hash of the fields of the proposed block.
|
||||
bytes hash = 4;
|
||||
int64 height = 5;
|
||||
google.protobuf.Timestamp time = 6 [(gogoproto.nullable) = false, (gogoproto.stdtime) = true];
|
||||
bytes next_validators_hash = 7;
|
||||
// address of the public key of the original proposer of the block.
|
||||
bytes proposer_address = 8;
|
||||
}
|
||||
|
||||
//----------------------------------------
|
||||
// Response types
|
||||
|
||||
@@ -159,7 +173,7 @@ message Response {
|
||||
ResponseLoadSnapshotChunk load_snapshot_chunk = 15;
|
||||
ResponseApplySnapshotChunk apply_snapshot_chunk = 16;
|
||||
ResponsePrepareProposal prepare_proposal = 17;
|
||||
|
||||
ResponseProcessProposal process_proposal = 18;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -249,7 +263,7 @@ message ResponseDeliverTx {
|
||||
}
|
||||
|
||||
message ResponseEndBlock {
|
||||
repeated ValidatorUpdate validator_updates = 1 [(gogoproto.nullable) = false];
|
||||
repeated ValidatorUpdate validator_updates = 1 [(gogoproto.nullable) = false];
|
||||
ConsensusParams consensus_param_updates = 2;
|
||||
repeated Event events = 3
|
||||
[(gogoproto.nullable) = false, (gogoproto.jsontag) = "events,omitempty"];
|
||||
@@ -298,7 +312,17 @@ message ResponseApplySnapshotChunk {
|
||||
}
|
||||
|
||||
message ResponsePrepareProposal {
|
||||
repeated TxRecord tx_records = 1;
|
||||
repeated TxRecord tx_records = 1;
|
||||
}
|
||||
|
||||
message ResponseProcessProposal {
|
||||
ProposalStatus status = 1;
|
||||
|
||||
enum ProposalStatus {
|
||||
UNKNOWN = 0;
|
||||
ACCEPT = 1;
|
||||
REJECT = 2;
|
||||
}
|
||||
}
|
||||
|
||||
//----------------------------------------
|
||||
@@ -321,7 +345,7 @@ message BlockParams {
|
||||
int64 max_gas = 2;
|
||||
}
|
||||
|
||||
message LastCommitInfo {
|
||||
message CommitInfo {
|
||||
int32 round = 1;
|
||||
repeated VoteInfo votes = 2 [(gogoproto.nullable) = false];
|
||||
}
|
||||
@@ -456,4 +480,5 @@ service ABCIApplication {
|
||||
rpc ApplySnapshotChunk(RequestApplySnapshotChunk)
|
||||
returns (ResponseApplySnapshotChunk);
|
||||
rpc PrepareProposal(RequestPrepareProposal) returns (ResponsePrepareProposal);
|
||||
rpc ProcessProposal(RequestProcessProposal) returns (ResponseProcessProposal);
|
||||
}
|
||||
|
||||
@@ -104,7 +104,7 @@ func (m *NewRoundStep) GetLastCommitRound() int32 {
|
||||
}
|
||||
|
||||
// NewValidBlock is sent when a validator observes a valid block B in some round r,
|
||||
//i.e., there is a Proposal for block B and 2/3+ prevotes for the block B in the round r.
|
||||
// i.e., there is a Proposal for block B and 2/3+ prevotes for the block B in the round r.
|
||||
// In case the block is also committed, then IsCommit flag is set to true.
|
||||
type NewValidBlock struct {
|
||||
Height int64 `protobuf:"varint,1,opt,name=height,proto3" json:"height,omitempty"`
|
||||
|
||||
@@ -18,7 +18,7 @@ message NewRoundStep {
|
||||
}
|
||||
|
||||
// NewValidBlock is sent when a validator observes a valid block B in some round r,
|
||||
//i.e., there is a Proposal for block B and 2/3+ prevotes for the block B in the round r.
|
||||
// i.e., there is a Proposal for block B and 2/3+ prevotes for the block B in the round r.
|
||||
// In case the block is also committed, then IsCommit flag is set to true.
|
||||
message NewValidBlock {
|
||||
int64 height = 1;
|
||||
|
||||
@@ -17,20 +17,20 @@ message Evidence {
|
||||
|
||||
// DuplicateVoteEvidence contains evidence of a validator signed two conflicting votes.
|
||||
message DuplicateVoteEvidence {
|
||||
tendermint.types.Vote vote_a = 1;
|
||||
tendermint.types.Vote vote_b = 2;
|
||||
int64 total_voting_power = 3;
|
||||
int64 validator_power = 4;
|
||||
google.protobuf.Timestamp timestamp = 5 [(gogoproto.nullable) = false, (gogoproto.stdtime) = true];
|
||||
tendermint.types.Vote vote_a = 1;
|
||||
tendermint.types.Vote vote_b = 2;
|
||||
int64 total_voting_power = 3;
|
||||
int64 validator_power = 4;
|
||||
google.protobuf.Timestamp timestamp = 5 [(gogoproto.nullable) = false, (gogoproto.stdtime) = true];
|
||||
}
|
||||
|
||||
// LightClientAttackEvidence contains evidence of a set of validators attempting to mislead a light client.
|
||||
message LightClientAttackEvidence {
|
||||
tendermint.types.LightBlock conflicting_block = 1;
|
||||
int64 common_height = 2;
|
||||
tendermint.types.LightBlock conflicting_block = 1;
|
||||
int64 common_height = 2;
|
||||
repeated tendermint.types.Validator byzantine_validators = 3;
|
||||
int64 total_voting_power = 4;
|
||||
google.protobuf.Timestamp timestamp = 5 [(gogoproto.nullable) = false, (gogoproto.stdtime) = true];
|
||||
int64 total_voting_power = 4;
|
||||
google.protobuf.Timestamp timestamp = 5 [(gogoproto.nullable) = false, (gogoproto.stdtime) = true];
|
||||
}
|
||||
|
||||
message EvidenceList {
|
||||
|
||||
@@ -106,10 +106,10 @@ message Vote {
|
||||
|
||||
// Commit contains the evidence that a block was committed by a set of validators.
|
||||
message Commit {
|
||||
int64 height = 1;
|
||||
int32 round = 2;
|
||||
BlockID block_id = 3 [(gogoproto.nullable) = false, (gogoproto.customname) = "BlockID"];
|
||||
repeated CommitSig signatures = 4 [(gogoproto.nullable) = false];
|
||||
int64 height = 1;
|
||||
int32 round = 2;
|
||||
BlockID block_id = 3 [(gogoproto.nullable) = false, (gogoproto.customname) = "BlockID"];
|
||||
repeated CommitSig signatures = 4 [(gogoproto.nullable) = false];
|
||||
}
|
||||
|
||||
// CommitSig is a part of the Vote included in a Commit.
|
||||
|
||||
@@ -15,8 +15,8 @@ type AppConnConsensus interface {
|
||||
Error() error
|
||||
|
||||
InitChainSync(types.RequestInitChain) (*types.ResponseInitChain, error)
|
||||
|
||||
PrepareProposalSync(types.RequestPrepareProposal) (*types.ResponsePrepareProposal, error)
|
||||
ProcessProposalSync(types.RequestProcessProposal) (*types.ResponseProcessProposal, error)
|
||||
BeginBlockSync(types.RequestBeginBlock) (*types.ResponseBeginBlock, error)
|
||||
DeliverTxAsync(types.RequestDeliverTx) *abcicli.ReqRes
|
||||
EndBlockSync(types.RequestEndBlock) (*types.ResponseEndBlock, error)
|
||||
@@ -85,6 +85,10 @@ func (app *appConnConsensus) PrepareProposalSync(
|
||||
return app.appConn.PrepareProposalSync(req)
|
||||
}
|
||||
|
||||
func (app *appConnConsensus) ProcessProposalSync(req types.RequestProcessProposal) (*types.ResponseProcessProposal, error) {
|
||||
return app.appConn.ProcessProposalSync(req)
|
||||
}
|
||||
|
||||
func (app *appConnConsensus) BeginBlockSync(req types.RequestBeginBlock) (*types.ResponseBeginBlock, error) {
|
||||
return app.appConn.BeginBlockSync(req)
|
||||
}
|
||||
|
||||
@@ -159,6 +159,29 @@ func (_m *AppConnConsensus) PrepareProposalSync(_a0 types.RequestPrepareProposal
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// ProcessProposalSync provides a mock function with given fields: _a0
|
||||
func (_m *AppConnConsensus) ProcessProposalSync(_a0 types.RequestProcessProposal) (*types.ResponseProcessProposal, error) {
|
||||
ret := _m.Called(_a0)
|
||||
|
||||
var r0 *types.ResponseProcessProposal
|
||||
if rf, ok := ret.Get(0).(func(types.RequestProcessProposal) *types.ResponseProcessProposal); ok {
|
||||
r0 = rf(_a0)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*types.ResponseProcessProposal)
|
||||
}
|
||||
}
|
||||
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(1).(func(types.RequestProcessProposal) error); ok {
|
||||
r1 = rf(_a0)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// SetResponseCallback provides a mock function with given fields: _a0
|
||||
func (_m *AppConnConsensus) SetResponseCallback(_a0 abcicli.Callback) {
|
||||
_m.Called(_a0)
|
||||
|
||||
15
scripts/mockery_generate.sh
Executable file
15
scripts/mockery_generate.sh
Executable file
@@ -0,0 +1,15 @@
|
||||
#!/bin/sh
|
||||
#
|
||||
# Invoke Mockery v2 to update generated mocks for the given type.
|
||||
#
|
||||
# This script runs a locally-installed "mockery" if available, otherwise it
|
||||
# runs the published Docker container. This legerdemain is so that the CI build
|
||||
# and a local build can work off the same script.
|
||||
#
|
||||
if ! which mockery ; then
|
||||
mockery() {
|
||||
docker run --rm -v "$PWD":/w --workdir=/w vektra/mockery:v2.12.3
|
||||
}
|
||||
fi
|
||||
|
||||
mockery --disable-version-string --case underscore --name "$@"
|
||||
@@ -112,7 +112,7 @@ func (blockExec *BlockExecutor) CreateProposalBlock(
|
||||
txs := blockExec.mempool.ReapMaxBytesMaxGas(maxDataBytes, maxGas)
|
||||
block := state.MakeBlock(height, txs, commit, evidence, proposerAddr)
|
||||
|
||||
localLastCommit := getBeginBlockValidatorInfo(block, blockExec.store, state.InitialHeight)
|
||||
localLastCommit := buildLastCommitInfo(block, blockExec.store, state.InitialHeight)
|
||||
rpp, err := blockExec.proxyApp.PrepareProposalSync(
|
||||
abci.RequestPrepareProposal{
|
||||
Hash: block.Hash(),
|
||||
@@ -132,7 +132,7 @@ func (blockExec *BlockExecutor) CreateProposalBlock(
|
||||
// Either way, we cannot recover in a meaningful way, unless we skip proposing
|
||||
// this block, repair what caused the error and try again. Hence, we return an
|
||||
// error for now (the production code calling this function is expected to panic).
|
||||
return nil, err
|
||||
panic(err)
|
||||
}
|
||||
txrSet := types.NewTxRecordSet(rpp.TxRecords)
|
||||
|
||||
@@ -149,6 +149,30 @@ func (blockExec *BlockExecutor) CreateProposalBlock(
|
||||
return state.MakeBlock(height, itxs, commit, evidence, proposerAddr), nil
|
||||
}
|
||||
|
||||
func (blockExec *BlockExecutor) ProcessProposal(
|
||||
block *types.Block,
|
||||
state State,
|
||||
) (bool, error) {
|
||||
resp, err := blockExec.proxyApp.ProcessProposalSync(abci.RequestProcessProposal{
|
||||
Hash: block.Header.Hash(),
|
||||
Height: block.Header.Height,
|
||||
Time: block.Header.Time,
|
||||
Txs: block.Data.Txs.ToSliceOfBytes(),
|
||||
ProposedLastCommit: buildLastCommitInfo(block, blockExec.store, state.InitialHeight),
|
||||
Misbehavior: block.Evidence.Evidence.ToABCI(),
|
||||
ProposerAddress: block.ProposerAddress,
|
||||
NextValidatorsHash: block.NextValidatorsHash,
|
||||
})
|
||||
if err != nil {
|
||||
return false, ErrInvalidBlock(err)
|
||||
}
|
||||
if resp.IsStatusUnknown() {
|
||||
panic(fmt.Sprintf("ProcessProposal responded with status %s", resp.Status.String()))
|
||||
}
|
||||
|
||||
return resp.IsAccepted(), nil
|
||||
}
|
||||
|
||||
// ValidateBlock validates the given block against the given state.
|
||||
// If the block is invalid, it returns an error.
|
||||
// Validation does not mutate state, but does require historical information from the stateDB,
|
||||
@@ -329,12 +353,7 @@ func execBlockOnProxyApp(
|
||||
}
|
||||
proxyAppConn.SetResponseCallback(proxyCb)
|
||||
|
||||
commitInfo := getBeginBlockValidatorInfo(block, store, initialHeight)
|
||||
|
||||
byzVals := make([]abci.Evidence, 0)
|
||||
for _, evidence := range block.Evidence.Evidence {
|
||||
byzVals = append(byzVals, evidence.ABCI()...)
|
||||
}
|
||||
commitInfo := buildLastCommitInfo(block, store, initialHeight)
|
||||
|
||||
// Begin block
|
||||
var err error
|
||||
@@ -347,7 +366,7 @@ func execBlockOnProxyApp(
|
||||
Hash: block.Hash(),
|
||||
Header: *pbh,
|
||||
LastCommitInfo: commitInfo,
|
||||
ByzantineValidators: byzVals,
|
||||
ByzantineValidators: block.Evidence.Evidence.ToABCI(),
|
||||
})
|
||||
if err != nil {
|
||||
logger.Error("error in proxyAppConn.BeginBlock", "err", err)
|
||||
@@ -373,47 +392,48 @@ func execBlockOnProxyApp(
|
||||
return abciResponses, nil
|
||||
}
|
||||
|
||||
func getBeginBlockValidatorInfo(block *types.Block, store Store,
|
||||
initialHeight int64) abci.LastCommitInfo {
|
||||
voteInfos := make([]abci.VoteInfo, block.LastCommit.Size())
|
||||
// Initial block -> LastCommitInfo.Votes are empty.
|
||||
// Remember that the first LastCommit is intentionally empty, so it makes
|
||||
// sense for LastCommitInfo.Votes to also be empty.
|
||||
if block.Height > initialHeight {
|
||||
lastValSet, err := store.LoadValidators(block.Height - 1)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
func buildLastCommitInfo(block *types.Block, store Store, initialHeight int64) abci.CommitInfo {
|
||||
if block.Height == initialHeight {
|
||||
// there is no last commit for the initial height.
|
||||
// return an empty value.
|
||||
return abci.CommitInfo{}
|
||||
}
|
||||
|
||||
// Sanity check that commit size matches validator set size - only applies
|
||||
// after first block.
|
||||
var (
|
||||
commitSize = block.LastCommit.Size()
|
||||
valSetLen = len(lastValSet.Validators)
|
||||
)
|
||||
if commitSize != valSetLen {
|
||||
panic(fmt.Sprintf(
|
||||
"commit size (%d) doesn't match valset length (%d) at height %d\n\n%v\n\n%v",
|
||||
commitSize, valSetLen, block.Height, block.LastCommit.Signatures, lastValSet.Validators,
|
||||
))
|
||||
}
|
||||
lastValSet, err := store.LoadValidators(block.Height - 1)
|
||||
if err != nil {
|
||||
panic(fmt.Errorf("failed to load validator set at height %d: %w", block.Height-1, err))
|
||||
}
|
||||
|
||||
for i, val := range lastValSet.Validators {
|
||||
commitSig := block.LastCommit.Signatures[i]
|
||||
voteInfos[i] = abci.VoteInfo{
|
||||
Validator: types.TM2PB.Validator(val),
|
||||
SignedLastBlock: !commitSig.Absent(),
|
||||
}
|
||||
var (
|
||||
commitSize = block.LastCommit.Size()
|
||||
valSetLen = len(lastValSet.Validators)
|
||||
)
|
||||
|
||||
// ensure that the size of the validator set in the last commit matches
|
||||
// the size of the validator set in the state store.
|
||||
if commitSize != valSetLen {
|
||||
panic(fmt.Sprintf(
|
||||
"commit size (%d) doesn't match validator set length (%d) at height %d\n\n%v\n\n%v",
|
||||
commitSize, valSetLen, block.Height, block.LastCommit.Signatures, lastValSet.Validators,
|
||||
))
|
||||
}
|
||||
|
||||
votes := make([]abci.VoteInfo, block.LastCommit.Size())
|
||||
for i, val := range lastValSet.Validators {
|
||||
commitSig := block.LastCommit.Signatures[i]
|
||||
votes[i] = abci.VoteInfo{
|
||||
Validator: types.TM2PB.Validator(val),
|
||||
SignedLastBlock: commitSig.BlockIDFlag != types.BlockIDFlagAbsent,
|
||||
}
|
||||
}
|
||||
|
||||
return abci.LastCommitInfo{
|
||||
return abci.CommitInfo{
|
||||
Round: block.LastCommit.Round,
|
||||
Votes: voteInfos,
|
||||
Votes: votes,
|
||||
}
|
||||
}
|
||||
|
||||
func extendedCommitInfo(c abci.LastCommitInfo, votes []*types.Vote) abci.ExtendedCommitInfo {
|
||||
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{
|
||||
|
||||
@@ -245,6 +245,85 @@ func TestBeginBlockByzantineValidators(t *testing.T) {
|
||||
assert.Equal(t, abciEv, app.ByzantineValidators)
|
||||
}
|
||||
|
||||
func TestProcessProposal(t *testing.T) {
|
||||
const height = 2
|
||||
txs := factory.MakeNTxs(height, 10)
|
||||
|
||||
logger := log.NewNopLogger()
|
||||
app := abcimocks.NewBaseMock()
|
||||
app.On("ProcessProposal", mock.Anything).Return(abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT})
|
||||
|
||||
cc := proxy.NewLocalClientCreator(app)
|
||||
proxyApp := proxy.NewAppConns(cc)
|
||||
err := proxyApp.Start()
|
||||
require.NoError(t, err)
|
||||
defer proxyApp.Stop() //nolint:errcheck // ignore for tests
|
||||
|
||||
state, stateDB, privVals := makeState(1, height)
|
||||
stateStore := sm.NewStore(stateDB)
|
||||
|
||||
eventBus := types.NewEventBus()
|
||||
err = eventBus.Start()
|
||||
require.NoError(t, err)
|
||||
|
||||
blockExec := sm.NewBlockExecutor(
|
||||
stateStore,
|
||||
logger,
|
||||
proxyApp.Consensus(),
|
||||
new(mpmocks.Mempool),
|
||||
sm.EmptyEvidencePool{},
|
||||
)
|
||||
|
||||
block0 := sf.MakeBlock(state, height-1, new(types.Commit))
|
||||
lastCommitSig := []types.CommitSig{}
|
||||
partSet, err := block0.MakePartSet(types.BlockPartSizeBytes)
|
||||
require.NoError(t, err)
|
||||
blockID := types.BlockID{Hash: block0.Hash(), PartSetHeader: partSet.Header()}
|
||||
voteInfos := []abci.VoteInfo{}
|
||||
for _, privVal := range privVals {
|
||||
vote, err := types.MakeVote(height, blockID, state.Validators, privVal, block0.Header.ChainID, time.Now())
|
||||
require.NoError(t, err)
|
||||
pk, err := privVal.GetPubKey()
|
||||
require.NoError(t, err)
|
||||
addr := pk.Address()
|
||||
voteInfos = append(voteInfos,
|
||||
abci.VoteInfo{
|
||||
SignedLastBlock: true,
|
||||
Validator: abci.Validator{
|
||||
Address: addr,
|
||||
Power: 1000,
|
||||
},
|
||||
})
|
||||
lastCommitSig = append(lastCommitSig, vote.CommitSig())
|
||||
}
|
||||
|
||||
block1 := sf.MakeBlock(state, height, &types.Commit{
|
||||
Height: height - 1,
|
||||
Signatures: lastCommitSig,
|
||||
})
|
||||
block1.Txs = txs
|
||||
|
||||
expectedRpp := abci.RequestProcessProposal{
|
||||
Txs: block1.Txs.ToSliceOfBytes(),
|
||||
Hash: block1.Hash(),
|
||||
Height: block1.Header.Height,
|
||||
Time: block1.Header.Time,
|
||||
Misbehavior: block1.Evidence.Evidence.ToABCI(),
|
||||
ProposedLastCommit: abci.CommitInfo{
|
||||
Round: 0,
|
||||
Votes: voteInfos,
|
||||
},
|
||||
NextValidatorsHash: block1.NextValidatorsHash,
|
||||
ProposerAddress: block1.ProposerAddress,
|
||||
}
|
||||
|
||||
acceptBlock, err := blockExec.ProcessProposal(block1, state)
|
||||
require.NoError(t, err)
|
||||
require.True(t, acceptBlock)
|
||||
app.AssertExpectations(t)
|
||||
app.AssertCalled(t, "ProcessProposal", expectedRpp)
|
||||
}
|
||||
|
||||
func TestValidateValidatorUpdates(t *testing.T) {
|
||||
pubkey1 := ed25519.GenPrivKey().PubKey()
|
||||
pubkey2 := ed25519.GenPrivKey().PubKey()
|
||||
|
||||
@@ -170,7 +170,6 @@ func makeHeaderPartsResponsesValPowerChange(
|
||||
) (types.Header, types.BlockID, *tmstate.ABCIResponses) {
|
||||
|
||||
block := sf.MakeBlock(state, state.LastBlockHeight+1, new(types.Commit))
|
||||
|
||||
abciResponses := &tmstate.ABCIResponses{
|
||||
BeginBlock: &abci.ResponseBeginBlock{},
|
||||
EndBlock: &abci.ResponseEndBlock{ValidatorUpdates: nil},
|
||||
@@ -265,3 +264,12 @@ func (app *testApp) Commit() abci.ResponseCommit {
|
||||
func (app *testApp) Query(reqQuery abci.RequestQuery) (resQuery abci.ResponseQuery) {
|
||||
return
|
||||
}
|
||||
|
||||
func (app *testApp) ProcessProposal(req abci.RequestProcessProposal) abci.ResponseProcessProposal {
|
||||
for _, tx := range req.Txs {
|
||||
if len(tx) == 0 {
|
||||
return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_REJECT}
|
||||
}
|
||||
}
|
||||
return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT}
|
||||
}
|
||||
|
||||
@@ -275,6 +275,18 @@ func (app *Application) PrepareProposal(
|
||||
return abci.ResponsePrepareProposal{TxRecords: trs}
|
||||
}
|
||||
|
||||
// ProcessProposal implements part of the Application interface.
|
||||
// It accepts any proposal that does not contain a malformed transaction.
|
||||
func (app *Application) ProcessProposal(req abci.RequestProcessProposal) abci.ResponseProcessProposal {
|
||||
for _, tx := range req.Txs {
|
||||
_, _, err := parseTx(tx)
|
||||
if err != nil {
|
||||
return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_REJECT}
|
||||
}
|
||||
}
|
||||
return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT}
|
||||
}
|
||||
|
||||
func (app *Application) Rollback() error {
|
||||
return app.state.Rollback()
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user