From 74cb71903fb001cb7c6880ad3bc1312814a8a2c8 Mon Sep 17 00:00:00 2001 From: Aleksandr Bezobchuk Date: Sun, 24 Sep 2023 15:24:30 -0400 Subject: [PATCH 1/7] fix(baseapp): select txs correctly with no-op mempool (#17769) (cherry picked from commit 63d046dd59f06855416e3570304fbaf9ce6aedfd) # Conflicts: # CHANGELOG.md # baseapp/abci_utils.go # baseapp/abci_utils_test.go # baseapp/baseapp.go --- CHANGELOG.md | 6 + baseapp/abci_utils.go | 395 +++++++++++++++++++++++++++++++++++++ baseapp/abci_utils_test.go | 379 +++++++++++++++++++++++++++++++++++ baseapp/baseapp.go | 9 + 4 files changed, 789 insertions(+) create mode 100644 baseapp/abci_utils.go create mode 100644 baseapp/abci_utils_test.go diff --git a/CHANGELOG.md b/CHANGELOG.md index 38df5e2fea63..db11f4a87cd6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -43,6 +43,12 @@ Ref: https://keepachangelog.com/en/1.0.0/ ### Bug Fixes +<<<<<<< HEAD +======= +* (baseapp) [#17769](https://github.com/cosmos/cosmos-sdk/pull/17769) Ensure we respect block size constraints in the `DefaultProposalHandler`'s `PrepareProposal` handler when a nil or no-op mempool is used. We provide a `TxSelector` type to assist in making transaction selection generalized. We also fix a comparison bug in tx selection when `req.maxTxBytes` is reached. +* (types) [#16583](https://github.com/cosmos/cosmos-sdk/pull/16583), [#17372](https://github.com/cosmos/cosmos-sdk/pull/17372), [#17421](https://github.com/cosmos/cosmos-sdk/pull/17421), [#17713](https://github.com/cosmos/cosmos-sdk/pull/17713) Introduce `PreBlock`, which executes in `FinalizeBlock` before `BeginBlock`. It allows the application to modify consensus parameters and have access to VE state. Note, `PreFinalizeBlockHook` is replaced by`PreBlocker`. +* (baseapp) [#17518](https://github.com/cosmos/cosmos-sdk/pull/17518) Utilizing voting power from vote extensions (CometBFT) instead of the current bonded tokens (x/staking) to determine if a set of vote extensions are valid. +>>>>>>> 63d046dd5 (fix(baseapp): select txs correctly with no-op mempool (#17769)) * (config) [#17649](https://github.com/cosmos/cosmos-sdk/pull/17649) Fix `mempool.max-txs` configuration is invalid in `app.config`. * (mempool) [#17668](https://github.com/cosmos/cosmos-sdk/pull/17668) Fix `PriorityNonceIterator.Next()` nil pointer ref for min priority at the end of iteration. diff --git a/baseapp/abci_utils.go b/baseapp/abci_utils.go new file mode 100644 index 000000000000..9a8da22c191c --- /dev/null +++ b/baseapp/abci_utils.go @@ -0,0 +1,395 @@ +package baseapp + +import ( + "bytes" + "context" + "fmt" + + "github.com/cockroachdb/errors" + abci "github.com/cometbft/cometbft/abci/types" + cryptoenc "github.com/cometbft/cometbft/crypto/encoding" + cmtprotocrypto "github.com/cometbft/cometbft/proto/tendermint/crypto" + cmtproto "github.com/cometbft/cometbft/proto/tendermint/types" + protoio "github.com/cosmos/gogoproto/io" + "github.com/cosmos/gogoproto/proto" + + "cosmossdk.io/math" + + sdk "github.com/cosmos/cosmos-sdk/types" + "github.com/cosmos/cosmos-sdk/types/mempool" +) + +// VoteExtensionThreshold defines the total voting power % that must be +// submitted in order for all vote extensions to be considered valid for a +// given height. +var VoteExtensionThreshold = math.LegacyNewDecWithPrec(667, 3) + +type ( + // ValidatorStore defines the interface contract require for verifying vote + // extension signatures. Typically, this will be implemented by the x/staking + // module, which has knowledge of the CometBFT public key. + ValidatorStore interface { + GetPubKeyByConsAddr(context.Context, sdk.ConsAddress) (cmtprotocrypto.PublicKey, error) + } + + // GasTx defines the contract that a transaction with a gas limit must implement. + GasTx interface { + GetGas() uint64 + } +) + +// ValidateVoteExtensions defines a helper function for verifying vote extension +// signatures that may be passed or manually injected into a block proposal from +// a proposer in PrepareProposal. It returns an error if any signature is invalid +// or if unexpected vote extensions and/or signatures are found or less than 2/3 +// power is received. +func ValidateVoteExtensions( + ctx sdk.Context, + valStore ValidatorStore, + currentHeight int64, + chainID string, + extCommit abci.ExtendedCommitInfo, +) error { + cp := ctx.ConsensusParams() + extsEnabled := cp.Abci != nil && currentHeight >= cp.Abci.VoteExtensionsEnableHeight && cp.Abci.VoteExtensionsEnableHeight != 0 + marshalDelimitedFn := func(msg proto.Message) ([]byte, error) { + var buf bytes.Buffer + if err := protoio.NewDelimitedWriter(&buf).WriteMsg(msg); err != nil { + return nil, err + } + + return buf.Bytes(), nil + } + + var ( + // Total voting power of all vote extensions. + totalVP int64 + // Total voting power of all validators that submitted valid vote extensions. + sumVP int64 + ) + + for _, vote := range extCommit.Votes { + totalVP += vote.Validator.Power + + // Only check + include power if the vote is a commit vote. There must be super-majority, otherwise the + // previous block (the block vote is for) could not have been committed. + if vote.BlockIdFlag != cmtproto.BlockIDFlagCommit { + continue + } + + if !extsEnabled { + if len(vote.VoteExtension) > 0 { + return fmt.Errorf("vote extensions disabled; received non-empty vote extension at height %d", currentHeight) + } + if len(vote.ExtensionSignature) > 0 { + return fmt.Errorf("vote extensions disabled; received non-empty vote extension signature at height %d", currentHeight) + } + + continue + } + + if len(vote.ExtensionSignature) == 0 { + return fmt.Errorf("vote extensions enabled; received empty vote extension signature at height %d", currentHeight) + } + + valConsAddr := sdk.ConsAddress(vote.Validator.Address) + pubKeyProto, err := valStore.GetPubKeyByConsAddr(ctx, valConsAddr) + if err != nil { + return fmt.Errorf("failed to get validator %X public key: %w", valConsAddr, err) + } + + cmtPubKey, err := cryptoenc.PubKeyFromProto(pubKeyProto) + if err != nil { + return fmt.Errorf("failed to convert validator %X public key: %w", valConsAddr, err) + } + + cve := cmtproto.CanonicalVoteExtension{ + Extension: vote.VoteExtension, + Height: currentHeight - 1, // the vote extension was signed in the previous height + Round: int64(extCommit.Round), + ChainId: chainID, + } + + extSignBytes, err := marshalDelimitedFn(&cve) + if err != nil { + return fmt.Errorf("failed to encode CanonicalVoteExtension: %w", err) + } + + if !cmtPubKey.VerifySignature(extSignBytes, vote.ExtensionSignature) { + return fmt.Errorf("failed to verify validator %X vote extension signature", valConsAddr) + } + + sumVP += vote.Validator.Power + } + + if totalVP > 0 { + percentSubmitted := math.LegacyNewDecFromInt(math.NewInt(sumVP)).Quo(math.LegacyNewDecFromInt(math.NewInt(totalVP))) + if percentSubmitted.LT(VoteExtensionThreshold) { + return fmt.Errorf("insufficient cumulative voting power received to verify vote extensions; got: %s, expected: >=%s", percentSubmitted, VoteExtensionThreshold) + } + } + + return nil +} + +type ( + // ProposalTxVerifier defines the interface that is implemented by BaseApp, + // that any custom ABCI PrepareProposal and ProcessProposal handler can use + // to verify a transaction. + ProposalTxVerifier interface { + PrepareProposalVerifyTx(tx sdk.Tx) ([]byte, error) + ProcessProposalVerifyTx(txBz []byte) (sdk.Tx, error) + TxDecode(txBz []byte) (sdk.Tx, error) + TxEncode(tx sdk.Tx) ([]byte, error) + } + + // DefaultProposalHandler defines the default ABCI PrepareProposal and + // ProcessProposal handlers. + DefaultProposalHandler struct { + mempool mempool.Mempool + txVerifier ProposalTxVerifier + txSelector TxSelector + } +) + +func NewDefaultProposalHandler(mp mempool.Mempool, txVerifier ProposalTxVerifier) *DefaultProposalHandler { + return &DefaultProposalHandler{ + mempool: mp, + txVerifier: txVerifier, + txSelector: NewDefaultTxSelector(), + } +} + +// SetTxSelector sets the TxSelector function on the DefaultProposalHandler. +func (h *DefaultProposalHandler) SetTxSelector(ts TxSelector) { + h.txSelector = ts +} + +// PrepareProposalHandler returns the default implementation for processing an +// ABCI proposal. The application's mempool is enumerated and all valid +// transactions are added to the proposal. Transactions are valid if they: +// +// 1) Successfully encode to bytes. +// 2) Are valid (i.e. pass runTx, AnteHandler only). +// +// Enumeration is halted once RequestPrepareProposal.MaxBytes of transactions is +// reached or the mempool is exhausted. +// +// Note: +// +// - Step (2) is identical to the validation step performed in +// DefaultProcessProposal. It is very important that the same validation logic +// is used in both steps, and applications must ensure that this is the case in +// non-default handlers. +// +// - If no mempool is set or if the mempool is a no-op mempool, the transactions +// requested from CometBFT will simply be returned, which, by default, are in +// FIFO order. +func (h *DefaultProposalHandler) PrepareProposalHandler() sdk.PrepareProposalHandler { + return func(ctx sdk.Context, req *abci.RequestPrepareProposal) (*abci.ResponsePrepareProposal, error) { + var maxBlockGas uint64 + if b := ctx.ConsensusParams().Block; b != nil { + maxBlockGas = uint64(b.MaxGas) + } + + defer h.txSelector.Clear() + + // If the mempool is nil or NoOp we simply return the transactions + // requested from CometBFT, which, by default, should be in FIFO order. + // + // Note, we still need to ensure the transactions returned respect req.MaxTxBytes. + _, isNoOp := h.mempool.(mempool.NoOpMempool) + if h.mempool == nil || isNoOp { + for _, txBz := range req.Txs { + tx, err := h.txVerifier.TxDecode(txBz) + if err != nil { + return nil, err + } + + stop := h.txSelector.SelectTxForProposal(uint64(req.MaxTxBytes), maxBlockGas, tx, txBz) + if stop { + break + } + } + + return &abci.ResponsePrepareProposal{Txs: h.txSelector.SelectedTxs()}, nil + } + + iterator := h.mempool.Select(ctx, req.Txs) + for iterator != nil { + memTx := iterator.Tx() + + // NOTE: Since transaction verification was already executed in CheckTx, + // which calls mempool.Insert, in theory everything in the pool should be + // valid. But some mempool implementations may insert invalid txs, so we + // check again. + txBz, err := h.txVerifier.PrepareProposalVerifyTx(memTx) + if err != nil { + err := h.mempool.Remove(memTx) + if err != nil && !errors.Is(err, mempool.ErrTxNotFound) { + return nil, err + } + } else { + stop := h.txSelector.SelectTxForProposal(uint64(req.MaxTxBytes), maxBlockGas, memTx, txBz) + if stop { + break + } + } + + iterator = iterator.Next() + } + + return &abci.ResponsePrepareProposal{Txs: h.txSelector.SelectedTxs()}, nil + } +} + +// ProcessProposalHandler returns the default implementation for processing an +// ABCI proposal. Every transaction in the proposal must pass 2 conditions: +// +// 1. The transaction bytes must decode to a valid transaction. +// 2. The transaction must be valid (i.e. pass runTx, AnteHandler only) +// +// If any transaction fails to pass either condition, the proposal is rejected. +// Note that step (2) is identical to the validation step performed in +// DefaultPrepareProposal. It is very important that the same validation logic +// is used in both steps, and applications must ensure that this is the case in +// non-default handlers. +func (h *DefaultProposalHandler) ProcessProposalHandler() sdk.ProcessProposalHandler { + // If the mempool is nil or NoOp we simply return ACCEPT, + // because PrepareProposal may have included txs that could fail verification. + _, isNoOp := h.mempool.(mempool.NoOpMempool) + if h.mempool == nil || isNoOp { + return NoOpProcessProposal() + } + + return func(ctx sdk.Context, req *abci.RequestProcessProposal) (*abci.ResponseProcessProposal, error) { + var totalTxGas uint64 + + var maxBlockGas int64 + if b := ctx.ConsensusParams().Block; b != nil { + maxBlockGas = b.MaxGas + } + + for _, txBytes := range req.Txs { + tx, err := h.txVerifier.ProcessProposalVerifyTx(txBytes) + if err != nil { + return &abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_REJECT}, nil + } + + if maxBlockGas > 0 { + gasTx, ok := tx.(GasTx) + if ok { + totalTxGas += gasTx.GetGas() + } + + if totalTxGas > uint64(maxBlockGas) { + return &abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_REJECT}, nil + } + } + } + + return &abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT}, nil + } +} + +// NoOpPrepareProposal defines a no-op PrepareProposal handler. It will always +// return the transactions sent by the client's request. +func NoOpPrepareProposal() sdk.PrepareProposalHandler { + return func(_ sdk.Context, req *abci.RequestPrepareProposal) (*abci.ResponsePrepareProposal, error) { + return &abci.ResponsePrepareProposal{Txs: req.Txs}, nil + } +} + +// NoOpProcessProposal defines a no-op ProcessProposal Handler. It will always +// return ACCEPT. +func NoOpProcessProposal() sdk.ProcessProposalHandler { + return func(_ sdk.Context, _ *abci.RequestProcessProposal) (*abci.ResponseProcessProposal, error) { + return &abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT}, nil + } +} + +// NoOpExtendVote defines a no-op ExtendVote handler. It will always return an +// empty byte slice as the vote extension. +func NoOpExtendVote() sdk.ExtendVoteHandler { + return func(_ sdk.Context, _ *abci.RequestExtendVote) (*abci.ResponseExtendVote, error) { + return &abci.ResponseExtendVote{VoteExtension: []byte{}}, nil + } +} + +// NoOpVerifyVoteExtensionHandler defines a no-op VerifyVoteExtension handler. It +// will always return an ACCEPT status with no error. +func NoOpVerifyVoteExtensionHandler() sdk.VerifyVoteExtensionHandler { + return func(_ sdk.Context, _ *abci.RequestVerifyVoteExtension) (*abci.ResponseVerifyVoteExtension, error) { + return &abci.ResponseVerifyVoteExtension{Status: abci.ResponseVerifyVoteExtension_ACCEPT}, nil + } +} + +// TxSelector defines a helper type that assists in selecting transactions during +// mempool transaction selection in PrepareProposal. It keeps track of the total +// number of bytes and total gas of the selected transactions. It also keeps +// track of the selected transactions themselves. +type TxSelector interface { + // SelectedTxs should return a copy of the selected transactions. + SelectedTxs() [][]byte + + // Clear should clear the TxSelector, nulling out all relevant fields. + Clear() + + // SelectTxForProposal should attempt to select a transaction for inclusion in + // a proposal based on inclusion criteria defined by the TxSelector. It must + // return if the caller should halt the transaction selection loop + // (typically over a mempool) or otherwise. + SelectTxForProposal(maxTxBytes, maxBlockGas uint64, memTx sdk.Tx, txBz []byte) bool +} + +type defaultTxSelector struct { + totalTxBytes uint64 + totalTxGas uint64 + selectedTxs [][]byte +} + +func NewDefaultTxSelector() TxSelector { + return &defaultTxSelector{} +} + +func (ts *defaultTxSelector) SelectedTxs() [][]byte { + txs := make([][]byte, len(ts.selectedTxs)) + copy(txs, ts.selectedTxs) + return txs +} + +func (ts *defaultTxSelector) Clear() { + ts.totalTxBytes = 0 + ts.totalTxGas = 0 + ts.selectedTxs = nil +} + +func (ts *defaultTxSelector) SelectTxForProposal(maxTxBytes, maxBlockGas uint64, memTx sdk.Tx, txBz []byte) bool { + txSize := uint64(len(txBz)) + + var txGasLimit uint64 + if memTx != nil { + if gasTx, ok := memTx.(GasTx); ok { + txGasLimit = gasTx.GetGas() + } + } + + // only add the transaction to the proposal if we have enough capacity + if (txSize + ts.totalTxBytes) <= maxTxBytes { + // If there is a max block gas limit, add the tx only if the limit has + // not been met. + if maxBlockGas > 0 { + if (txGasLimit + ts.totalTxGas) <= maxBlockGas { + ts.totalTxGas += txGasLimit + ts.totalTxBytes += txSize + ts.selectedTxs = append(ts.selectedTxs, txBz) + } + } else { + ts.totalTxBytes += txSize + ts.selectedTxs = append(ts.selectedTxs, txBz) + } + } + + // check if we've reached capacity; if so, we cannot select any more transactions + return ts.totalTxBytes >= maxTxBytes || (maxBlockGas > 0 && (ts.totalTxGas >= maxBlockGas)) +} diff --git a/baseapp/abci_utils_test.go b/baseapp/abci_utils_test.go new file mode 100644 index 000000000000..8919ee81ba8c --- /dev/null +++ b/baseapp/abci_utils_test.go @@ -0,0 +1,379 @@ +package baseapp_test + +import ( + "bytes" + "testing" + + abci "github.com/cometbft/cometbft/abci/types" + "github.com/cometbft/cometbft/crypto/secp256k1" + cmtprotocrypto "github.com/cometbft/cometbft/proto/tendermint/crypto" + cmtproto "github.com/cometbft/cometbft/proto/tendermint/types" + dbm "github.com/cosmos/cosmos-db" + protoio "github.com/cosmos/gogoproto/io" + "github.com/cosmos/gogoproto/proto" + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/suite" + + "cosmossdk.io/log" + + "github.com/cosmos/cosmos-sdk/baseapp" + baseapptestutil "github.com/cosmos/cosmos-sdk/baseapp/testutil" + "github.com/cosmos/cosmos-sdk/baseapp/testutil/mock" + codectestutil "github.com/cosmos/cosmos-sdk/codec/testutil" + "github.com/cosmos/cosmos-sdk/testutil/testdata" + sdk "github.com/cosmos/cosmos-sdk/types" + "github.com/cosmos/cosmos-sdk/types/mempool" + authtx "github.com/cosmos/cosmos-sdk/x/auth/tx" +) + +const ( + chainID = "chain-id" +) + +type testValidator struct { + consAddr sdk.ConsAddress + tmPk cmtprotocrypto.PublicKey + privKey secp256k1.PrivKey +} + +func newTestValidator() testValidator { + privkey := secp256k1.GenPrivKey() + pubkey := privkey.PubKey() + tmPk := cmtprotocrypto.PublicKey{ + Sum: &cmtprotocrypto.PublicKey_Secp256K1{ + Secp256K1: pubkey.Bytes(), + }, + } + + return testValidator{ + consAddr: sdk.ConsAddress(pubkey.Address()), + tmPk: tmPk, + privKey: privkey, + } +} + +func (t testValidator) toValidator(power int64) abci.Validator { + return abci.Validator{ + Address: t.consAddr.Bytes(), + Power: power, + } +} + +type ABCIUtilsTestSuite struct { + suite.Suite + + valStore *mock.MockValidatorStore + vals [3]testValidator + ctx sdk.Context +} + +func NewABCIUtilsTestSuite(t *testing.T) *ABCIUtilsTestSuite { + t.Helper() + // create 3 validators + s := &ABCIUtilsTestSuite{ + vals: [3]testValidator{ + newTestValidator(), + newTestValidator(), + newTestValidator(), + }, + } + + // create mock + ctrl := gomock.NewController(t) + valStore := mock.NewMockValidatorStore(ctrl) + s.valStore = valStore + + // set up mock + s.valStore.EXPECT().GetPubKeyByConsAddr(gomock.Any(), s.vals[0].consAddr.Bytes()).Return(s.vals[0].tmPk, nil).AnyTimes() + s.valStore.EXPECT().GetPubKeyByConsAddr(gomock.Any(), s.vals[1].consAddr.Bytes()).Return(s.vals[1].tmPk, nil).AnyTimes() + s.valStore.EXPECT().GetPubKeyByConsAddr(gomock.Any(), s.vals[2].consAddr.Bytes()).Return(s.vals[2].tmPk, nil).AnyTimes() + + // create context + s.ctx = sdk.Context{}.WithConsensusParams(cmtproto.ConsensusParams{ + Abci: &cmtproto.ABCIParams{ + VoteExtensionsEnableHeight: 2, + }, + }) + return s +} + +func TestABCIUtilsTestSuite(t *testing.T) { + suite.Run(t, NewABCIUtilsTestSuite(t)) +} + +// check ValidateVoteExtensions works when all nodes have CommitBlockID votes +func (s *ABCIUtilsTestSuite) TestValidateVoteExtensionsHappyPath() { + ext := []byte("vote-extension") + cve := cmtproto.CanonicalVoteExtension{ + Extension: ext, + Height: 2, + Round: int64(0), + ChainId: chainID, + } + + bz, err := marshalDelimitedFn(&cve) + s.Require().NoError(err) + + extSig0, err := s.vals[0].privKey.Sign(bz) + s.Require().NoError(err) + + extSig1, err := s.vals[1].privKey.Sign(bz) + s.Require().NoError(err) + + extSig2, err := s.vals[2].privKey.Sign(bz) + s.Require().NoError(err) + + llc := abci.ExtendedCommitInfo{ + Round: 0, + Votes: []abci.ExtendedVoteInfo{ + { + Validator: s.vals[0].toValidator(333), + VoteExtension: ext, + ExtensionSignature: extSig0, + BlockIdFlag: cmtproto.BlockIDFlagCommit, + }, + { + Validator: s.vals[1].toValidator(333), + VoteExtension: ext, + ExtensionSignature: extSig1, + BlockIdFlag: cmtproto.BlockIDFlagCommit, + }, + { + Validator: s.vals[2].toValidator(334), + VoteExtension: ext, + ExtensionSignature: extSig2, + BlockIdFlag: cmtproto.BlockIDFlagCommit, + }, + }, + } + // expect-pass (votes of height 2 are included in next block) + s.Require().NoError(baseapp.ValidateVoteExtensions(s.ctx, s.valStore, 3, chainID, llc)) +} + +// check ValidateVoteExtensions works when a single node has submitted a BlockID_Absent +func (s *ABCIUtilsTestSuite) TestValidateVoteExtensionsSingleVoteAbsent() { + ext := []byte("vote-extension") + cve := cmtproto.CanonicalVoteExtension{ + Extension: ext, + Height: 2, + Round: int64(0), + ChainId: chainID, + } + + bz, err := marshalDelimitedFn(&cve) + s.Require().NoError(err) + + extSig0, err := s.vals[0].privKey.Sign(bz) + s.Require().NoError(err) + + extSig2, err := s.vals[2].privKey.Sign(bz) + s.Require().NoError(err) + + llc := abci.ExtendedCommitInfo{ + Round: 0, + Votes: []abci.ExtendedVoteInfo{ + { + Validator: s.vals[0].toValidator(333), + VoteExtension: ext, + ExtensionSignature: extSig0, + BlockIdFlag: cmtproto.BlockIDFlagCommit, + }, + // validator of power <1/3 is missing, so commit-info shld still be valid + { + Validator: s.vals[1].toValidator(333), + BlockIdFlag: cmtproto.BlockIDFlagAbsent, + }, + { + Validator: s.vals[2].toValidator(334), + VoteExtension: ext, + ExtensionSignature: extSig2, + BlockIdFlag: cmtproto.BlockIDFlagCommit, + }, + }, + } + // expect-pass (votes of height 2 are included in next block) + s.Require().NoError(baseapp.ValidateVoteExtensions(s.ctx, s.valStore, 3, chainID, llc)) +} + +// check ValidateVoteExtensions works when a single node has submitted a BlockID_Nil +func (s *ABCIUtilsTestSuite) TestValidateVoteExtensionsSingleVoteNil() { + ext := []byte("vote-extension") + cve := cmtproto.CanonicalVoteExtension{ + Extension: ext, + Height: 2, + Round: int64(0), + ChainId: chainID, + } + + bz, err := marshalDelimitedFn(&cve) + s.Require().NoError(err) + + extSig0, err := s.vals[0].privKey.Sign(bz) + s.Require().NoError(err) + + extSig2, err := s.vals[2].privKey.Sign(bz) + s.Require().NoError(err) + + llc := abci.ExtendedCommitInfo{ + Round: 0, + Votes: []abci.ExtendedVoteInfo{ + { + Validator: s.vals[0].toValidator(333), + VoteExtension: ext, + ExtensionSignature: extSig0, + BlockIdFlag: cmtproto.BlockIDFlagCommit, + }, + // validator of power <1/3 is missing, so commit-info should still be valid + { + Validator: s.vals[1].toValidator(333), + BlockIdFlag: cmtproto.BlockIDFlagNil, + }, + { + Validator: s.vals[2].toValidator(334), + VoteExtension: ext, + ExtensionSignature: extSig2, + BlockIdFlag: cmtproto.BlockIDFlagCommit, + }, + }, + } + // expect-pass (votes of height 2 are included in next block) + s.Require().NoError(baseapp.ValidateVoteExtensions(s.ctx, s.valStore, 3, chainID, llc)) +} + +// check ValidateVoteExtensions works when two nodes have submitted a BlockID_Nil / BlockID_Absent +func (s *ABCIUtilsTestSuite) TestValidateVoteExtensionsTwoVotesNilAbsent() { + ext := []byte("vote-extension") + cve := cmtproto.CanonicalVoteExtension{ + Extension: ext, + Height: 2, + Round: int64(0), + ChainId: chainID, + } + + bz, err := marshalDelimitedFn(&cve) + s.Require().NoError(err) + + extSig0, err := s.vals[0].privKey.Sign(bz) + s.Require().NoError(err) + + llc := abci.ExtendedCommitInfo{ + Round: 0, + Votes: []abci.ExtendedVoteInfo{ + // validator of power >2/3 is missing, so commit-info should not be valid + { + Validator: s.vals[0].toValidator(333), + BlockIdFlag: cmtproto.BlockIDFlagCommit, + VoteExtension: ext, + ExtensionSignature: extSig0, + }, + { + Validator: s.vals[1].toValidator(333), + BlockIdFlag: cmtproto.BlockIDFlagNil, + }, + { + Validator: s.vals[2].toValidator(334), + VoteExtension: ext, + BlockIdFlag: cmtproto.BlockIDFlagAbsent, + }, + }, + } + + // expect-pass (votes of height 2 are included in next block) + s.Require().Error(baseapp.ValidateVoteExtensions(s.ctx, s.valStore, 3, chainID, llc)) +} + +func (s *ABCIUtilsTestSuite) TestDefaultProposalHandler_NoOpMempoolTxSelection() { + // create a codec for marshaling + cdc := codectestutil.CodecOptions{}.NewCodec() + baseapptestutil.RegisterInterfaces(cdc.InterfaceRegistry()) + + // create a baseapp along with a tx config for tx generation + txConfig := authtx.NewTxConfig(cdc, authtx.DefaultSignModes) + app := baseapp.NewBaseApp(s.T().Name(), log.NewNopLogger(), dbm.NewMemDB(), txConfig.TxDecoder()) + + // create a proposal handler + ph := baseapp.NewDefaultProposalHandler(mempool.NoOpMempool{}, app) + handler := ph.PrepareProposalHandler() + + // build a tx + _, _, addr := testdata.KeyTestPubAddr() + builder := txConfig.NewTxBuilder() + s.Require().NoError(builder.SetMsgs( + &baseapptestutil.MsgCounter{Counter: 0, FailOnHandler: false, Signer: addr.String()}, + )) + builder.SetGasLimit(100) + setTxSignature(s.T(), builder, 0) + + // encode the tx to be used in the proposal request + tx := builder.GetTx() + txBz, err := txConfig.TxEncoder()(tx) + s.Require().NoError(err) + s.Require().Len(txBz, 152) + + testCases := map[string]struct { + ctx sdk.Context + req *abci.RequestPrepareProposal + expectedTxs int + }{ + "small max tx bytes": { + ctx: s.ctx, + req: &abci.RequestPrepareProposal{ + Txs: [][]byte{txBz, txBz, txBz, txBz, txBz}, + MaxTxBytes: 10, + }, + expectedTxs: 0, + }, + "small max gas": { + ctx: s.ctx.WithConsensusParams(cmtproto.ConsensusParams{ + Block: &cmtproto.BlockParams{ + MaxGas: 10, + }, + }), + req: &abci.RequestPrepareProposal{ + Txs: [][]byte{txBz, txBz, txBz, txBz, txBz}, + MaxTxBytes: 456, + }, + expectedTxs: 0, + }, + "large max tx bytes": { + ctx: s.ctx, + req: &abci.RequestPrepareProposal{ + Txs: [][]byte{txBz, txBz, txBz, txBz, txBz}, + MaxTxBytes: 456, + }, + expectedTxs: 3, + }, + "max gas and tx bytes": { + ctx: s.ctx.WithConsensusParams(cmtproto.ConsensusParams{ + Block: &cmtproto.BlockParams{ + MaxGas: 200, + }, + }), + req: &abci.RequestPrepareProposal{ + Txs: [][]byte{txBz, txBz, txBz, txBz, txBz}, + MaxTxBytes: 456, + }, + expectedTxs: 2, + }, + } + + for name, tc := range testCases { + s.Run(name, func() { + // iterate multiple times to ensure the tx selector is cleared each time + for i := 0; i < 5; i++ { + resp, err := handler(tc.ctx, tc.req) + s.Require().NoError(err) + s.Require().Len(resp.Txs, tc.expectedTxs) + } + }) + } +} + +func marshalDelimitedFn(msg proto.Message) ([]byte, error) { + var buf bytes.Buffer + if err := protoio.NewDelimitedWriter(&buf).WriteMsg(msg); err != nil { + return nil, err + } + + return buf.Bytes(), nil +} diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index 14f2654c51a3..2b4da3c51197 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -906,6 +906,7 @@ func (app *BaseApp) ProcessProposalVerifyTx(txBz []byte) (sdk.Tx, error) { return tx, nil } +<<<<<<< HEAD type ( // ProposalTxVerifier defines the interface that is implemented by BaseApp, // that any custom ABCI PrepareProposal and ProcessProposal handler can use @@ -1069,6 +1070,14 @@ func NoOpProcessProposal() sdk.ProcessProposalHandler { return func(_ sdk.Context, _ abci.RequestProcessProposal) abci.ResponseProcessProposal { return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT} } +======= +func (app *BaseApp) TxDecode(txBytes []byte) (sdk.Tx, error) { + return app.txDecoder(txBytes) +} + +func (app *BaseApp) TxEncode(tx sdk.Tx) ([]byte, error) { + return app.txEncoder(tx) +>>>>>>> 63d046dd5 (fix(baseapp): select txs correctly with no-op mempool (#17769)) } // Close is called in start cmd to gracefully cleanup resources. From c8fd731155b8779b489f78f76d2680dbc0cfe237 Mon Sep 17 00:00:00 2001 From: Aleksandr Bezobchuk Date: Mon, 25 Sep 2023 08:39:03 -0700 Subject: [PATCH 2/7] updates --- CHANGELOG.md | 5 ----- 1 file changed, 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index db11f4a87cd6..d41a5a6c49b9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -43,12 +43,7 @@ Ref: https://keepachangelog.com/en/1.0.0/ ### Bug Fixes -<<<<<<< HEAD -======= * (baseapp) [#17769](https://github.com/cosmos/cosmos-sdk/pull/17769) Ensure we respect block size constraints in the `DefaultProposalHandler`'s `PrepareProposal` handler when a nil or no-op mempool is used. We provide a `TxSelector` type to assist in making transaction selection generalized. We also fix a comparison bug in tx selection when `req.maxTxBytes` is reached. -* (types) [#16583](https://github.com/cosmos/cosmos-sdk/pull/16583), [#17372](https://github.com/cosmos/cosmos-sdk/pull/17372), [#17421](https://github.com/cosmos/cosmos-sdk/pull/17421), [#17713](https://github.com/cosmos/cosmos-sdk/pull/17713) Introduce `PreBlock`, which executes in `FinalizeBlock` before `BeginBlock`. It allows the application to modify consensus parameters and have access to VE state. Note, `PreFinalizeBlockHook` is replaced by`PreBlocker`. -* (baseapp) [#17518](https://github.com/cosmos/cosmos-sdk/pull/17518) Utilizing voting power from vote extensions (CometBFT) instead of the current bonded tokens (x/staking) to determine if a set of vote extensions are valid. ->>>>>>> 63d046dd5 (fix(baseapp): select txs correctly with no-op mempool (#17769)) * (config) [#17649](https://github.com/cosmos/cosmos-sdk/pull/17649) Fix `mempool.max-txs` configuration is invalid in `app.config`. * (mempool) [#17668](https://github.com/cosmos/cosmos-sdk/pull/17668) Fix `PriorityNonceIterator.Next()` nil pointer ref for min priority at the end of iteration. From dd7ea86f0ab524e4e2602c6119a00fc9bf0f87b4 Mon Sep 17 00:00:00 2001 From: Aleksandr Bezobchuk Date: Mon, 25 Sep 2023 08:55:02 -0700 Subject: [PATCH 3/7] updates --- baseapp/abci_utils.go | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/baseapp/abci_utils.go b/baseapp/abci_utils.go index 9a8da22c191c..43e6dc1e5de3 100644 --- a/baseapp/abci_utils.go +++ b/baseapp/abci_utils.go @@ -139,8 +139,6 @@ type ( ProposalTxVerifier interface { PrepareProposalVerifyTx(tx sdk.Tx) ([]byte, error) ProcessProposalVerifyTx(txBz []byte) (sdk.Tx, error) - TxDecode(txBz []byte) (sdk.Tx, error) - TxEncode(tx sdk.Tx) ([]byte, error) } // DefaultProposalHandler defines the default ABCI PrepareProposal and @@ -201,18 +199,15 @@ func (h *DefaultProposalHandler) PrepareProposalHandler() sdk.PrepareProposalHan _, isNoOp := h.mempool.(mempool.NoOpMempool) if h.mempool == nil || isNoOp { for _, txBz := range req.Txs { - tx, err := h.txVerifier.TxDecode(txBz) - if err != nil { - return nil, err - } - - stop := h.txSelector.SelectTxForProposal(uint64(req.MaxTxBytes), maxBlockGas, tx, txBz) + // XXX: We pass nil as the memTx because we have no way of decoding the + // txBz. We'd need to break (update) the ProposalTxVerifier interface. + stop := txSelector.SelectTxForProposal(nil, txBz) if stop { break } } - return &abci.ResponsePrepareProposal{Txs: h.txSelector.SelectedTxs()}, nil + return &abci.ResponsePrepareProposal{Txs: txSelector.SelectedTxs()}, nil } iterator := h.mempool.Select(ctx, req.Txs) From b07fdb4ddbb55d410891d930f9e118a1e7fe181d Mon Sep 17 00:00:00 2001 From: Aleksandr Bezobchuk Date: Mon, 25 Sep 2023 09:05:40 -0700 Subject: [PATCH 4/7] updates --- baseapp/abci_utils.go | 161 ++-------------------- baseapp/baseapp.go | 313 +++++++++++++++++++----------------------- 2 files changed, 153 insertions(+), 321 deletions(-) diff --git a/baseapp/abci_utils.go b/baseapp/abci_utils.go index 43e6dc1e5de3..87507d836ab1 100644 --- a/baseapp/abci_utils.go +++ b/baseapp/abci_utils.go @@ -1,138 +1,19 @@ package baseapp import ( - "bytes" - "context" - "fmt" - "github.com/cockroachdb/errors" abci "github.com/cometbft/cometbft/abci/types" - cryptoenc "github.com/cometbft/cometbft/crypto/encoding" - cmtprotocrypto "github.com/cometbft/cometbft/proto/tendermint/crypto" - cmtproto "github.com/cometbft/cometbft/proto/tendermint/types" - protoio "github.com/cosmos/gogoproto/io" - "github.com/cosmos/gogoproto/proto" - - "cosmossdk.io/math" sdk "github.com/cosmos/cosmos-sdk/types" "github.com/cosmos/cosmos-sdk/types/mempool" ) -// VoteExtensionThreshold defines the total voting power % that must be -// submitted in order for all vote extensions to be considered valid for a -// given height. -var VoteExtensionThreshold = math.LegacyNewDecWithPrec(667, 3) - type ( - // ValidatorStore defines the interface contract require for verifying vote - // extension signatures. Typically, this will be implemented by the x/staking - // module, which has knowledge of the CometBFT public key. - ValidatorStore interface { - GetPubKeyByConsAddr(context.Context, sdk.ConsAddress) (cmtprotocrypto.PublicKey, error) - } - // GasTx defines the contract that a transaction with a gas limit must implement. GasTx interface { GetGas() uint64 } -) - -// ValidateVoteExtensions defines a helper function for verifying vote extension -// signatures that may be passed or manually injected into a block proposal from -// a proposer in PrepareProposal. It returns an error if any signature is invalid -// or if unexpected vote extensions and/or signatures are found or less than 2/3 -// power is received. -func ValidateVoteExtensions( - ctx sdk.Context, - valStore ValidatorStore, - currentHeight int64, - chainID string, - extCommit abci.ExtendedCommitInfo, -) error { - cp := ctx.ConsensusParams() - extsEnabled := cp.Abci != nil && currentHeight >= cp.Abci.VoteExtensionsEnableHeight && cp.Abci.VoteExtensionsEnableHeight != 0 - marshalDelimitedFn := func(msg proto.Message) ([]byte, error) { - var buf bytes.Buffer - if err := protoio.NewDelimitedWriter(&buf).WriteMsg(msg); err != nil { - return nil, err - } - - return buf.Bytes(), nil - } - - var ( - // Total voting power of all vote extensions. - totalVP int64 - // Total voting power of all validators that submitted valid vote extensions. - sumVP int64 - ) - - for _, vote := range extCommit.Votes { - totalVP += vote.Validator.Power - - // Only check + include power if the vote is a commit vote. There must be super-majority, otherwise the - // previous block (the block vote is for) could not have been committed. - if vote.BlockIdFlag != cmtproto.BlockIDFlagCommit { - continue - } - - if !extsEnabled { - if len(vote.VoteExtension) > 0 { - return fmt.Errorf("vote extensions disabled; received non-empty vote extension at height %d", currentHeight) - } - if len(vote.ExtensionSignature) > 0 { - return fmt.Errorf("vote extensions disabled; received non-empty vote extension signature at height %d", currentHeight) - } - - continue - } - - if len(vote.ExtensionSignature) == 0 { - return fmt.Errorf("vote extensions enabled; received empty vote extension signature at height %d", currentHeight) - } - - valConsAddr := sdk.ConsAddress(vote.Validator.Address) - pubKeyProto, err := valStore.GetPubKeyByConsAddr(ctx, valConsAddr) - if err != nil { - return fmt.Errorf("failed to get validator %X public key: %w", valConsAddr, err) - } - cmtPubKey, err := cryptoenc.PubKeyFromProto(pubKeyProto) - if err != nil { - return fmt.Errorf("failed to convert validator %X public key: %w", valConsAddr, err) - } - - cve := cmtproto.CanonicalVoteExtension{ - Extension: vote.VoteExtension, - Height: currentHeight - 1, // the vote extension was signed in the previous height - Round: int64(extCommit.Round), - ChainId: chainID, - } - - extSignBytes, err := marshalDelimitedFn(&cve) - if err != nil { - return fmt.Errorf("failed to encode CanonicalVoteExtension: %w", err) - } - - if !cmtPubKey.VerifySignature(extSignBytes, vote.ExtensionSignature) { - return fmt.Errorf("failed to verify validator %X vote extension signature", valConsAddr) - } - - sumVP += vote.Validator.Power - } - - if totalVP > 0 { - percentSubmitted := math.LegacyNewDecFromInt(math.NewInt(sumVP)).Quo(math.LegacyNewDecFromInt(math.NewInt(totalVP))) - if percentSubmitted.LT(VoteExtensionThreshold) { - return fmt.Errorf("insufficient cumulative voting power received to verify vote extensions; got: %s, expected: >=%s", percentSubmitted, VoteExtensionThreshold) - } - } - - return nil -} - -type ( // ProposalTxVerifier defines the interface that is implemented by BaseApp, // that any custom ABCI PrepareProposal and ProcessProposal handler can use // to verify a transaction. @@ -184,7 +65,7 @@ func (h *DefaultProposalHandler) SetTxSelector(ts TxSelector) { // requested from CometBFT will simply be returned, which, by default, are in // FIFO order. func (h *DefaultProposalHandler) PrepareProposalHandler() sdk.PrepareProposalHandler { - return func(ctx sdk.Context, req *abci.RequestPrepareProposal) (*abci.ResponsePrepareProposal, error) { + return func(ctx sdk.Context, req abci.RequestPrepareProposal) abci.ResponsePrepareProposal { var maxBlockGas uint64 if b := ctx.ConsensusParams().Block; b != nil { maxBlockGas = uint64(b.MaxGas) @@ -201,13 +82,13 @@ func (h *DefaultProposalHandler) PrepareProposalHandler() sdk.PrepareProposalHan for _, txBz := range req.Txs { // XXX: We pass nil as the memTx because we have no way of decoding the // txBz. We'd need to break (update) the ProposalTxVerifier interface. - stop := txSelector.SelectTxForProposal(nil, txBz) + stop := h.txSelector.SelectTxForProposal(uint64(req.MaxTxBytes), maxBlockGas, nil, txBz) if stop { break } } - return &abci.ResponsePrepareProposal{Txs: txSelector.SelectedTxs()}, nil + return abci.ResponsePrepareProposal{Txs: h.txSelector.SelectedTxs()} } iterator := h.mempool.Select(ctx, req.Txs) @@ -222,7 +103,7 @@ func (h *DefaultProposalHandler) PrepareProposalHandler() sdk.PrepareProposalHan if err != nil { err := h.mempool.Remove(memTx) if err != nil && !errors.Is(err, mempool.ErrTxNotFound) { - return nil, err + panic(err) } } else { stop := h.txSelector.SelectTxForProposal(uint64(req.MaxTxBytes), maxBlockGas, memTx, txBz) @@ -234,7 +115,7 @@ func (h *DefaultProposalHandler) PrepareProposalHandler() sdk.PrepareProposalHan iterator = iterator.Next() } - return &abci.ResponsePrepareProposal{Txs: h.txSelector.SelectedTxs()}, nil + return abci.ResponsePrepareProposal{Txs: h.txSelector.SelectedTxs()} } } @@ -257,7 +138,7 @@ func (h *DefaultProposalHandler) ProcessProposalHandler() sdk.ProcessProposalHan return NoOpProcessProposal() } - return func(ctx sdk.Context, req *abci.RequestProcessProposal) (*abci.ResponseProcessProposal, error) { + return func(ctx sdk.Context, req abci.RequestProcessProposal) abci.ResponseProcessProposal { var totalTxGas uint64 var maxBlockGas int64 @@ -268,7 +149,7 @@ func (h *DefaultProposalHandler) ProcessProposalHandler() sdk.ProcessProposalHan for _, txBytes := range req.Txs { tx, err := h.txVerifier.ProcessProposalVerifyTx(txBytes) if err != nil { - return &abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_REJECT}, nil + return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_REJECT} } if maxBlockGas > 0 { @@ -278,44 +159,28 @@ func (h *DefaultProposalHandler) ProcessProposalHandler() sdk.ProcessProposalHan } if totalTxGas > uint64(maxBlockGas) { - return &abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_REJECT}, nil + return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_REJECT} } } } - return &abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT}, nil + return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT} } } // NoOpPrepareProposal defines a no-op PrepareProposal handler. It will always // return the transactions sent by the client's request. func NoOpPrepareProposal() sdk.PrepareProposalHandler { - return func(_ sdk.Context, req *abci.RequestPrepareProposal) (*abci.ResponsePrepareProposal, error) { - return &abci.ResponsePrepareProposal{Txs: req.Txs}, nil + return func(_ sdk.Context, req abci.RequestPrepareProposal) abci.ResponsePrepareProposal { + return abci.ResponsePrepareProposal{Txs: req.Txs} } } // NoOpProcessProposal defines a no-op ProcessProposal Handler. It will always // return ACCEPT. func NoOpProcessProposal() sdk.ProcessProposalHandler { - return func(_ sdk.Context, _ *abci.RequestProcessProposal) (*abci.ResponseProcessProposal, error) { - return &abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT}, nil - } -} - -// NoOpExtendVote defines a no-op ExtendVote handler. It will always return an -// empty byte slice as the vote extension. -func NoOpExtendVote() sdk.ExtendVoteHandler { - return func(_ sdk.Context, _ *abci.RequestExtendVote) (*abci.ResponseExtendVote, error) { - return &abci.ResponseExtendVote{VoteExtension: []byte{}}, nil - } -} - -// NoOpVerifyVoteExtensionHandler defines a no-op VerifyVoteExtension handler. It -// will always return an ACCEPT status with no error. -func NoOpVerifyVoteExtensionHandler() sdk.VerifyVoteExtensionHandler { - return func(_ sdk.Context, _ *abci.RequestVerifyVoteExtension) (*abci.ResponseVerifyVoteExtension, error) { - return &abci.ResponseVerifyVoteExtension{Status: abci.ResponseVerifyVoteExtension_ACCEPT}, nil + return func(_ sdk.Context, _ abci.RequestProcessProposal) abci.ResponseProcessProposal { + return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT} } } diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index 2b4da3c51197..7aec0ecc0628 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -906,179 +906,146 @@ func (app *BaseApp) ProcessProposalVerifyTx(txBz []byte) (sdk.Tx, error) { return tx, nil } -<<<<<<< HEAD -type ( - // ProposalTxVerifier defines the interface that is implemented by BaseApp, - // that any custom ABCI PrepareProposal and ProcessProposal handler can use - // to verify a transaction. - ProposalTxVerifier interface { - PrepareProposalVerifyTx(tx sdk.Tx) ([]byte, error) - ProcessProposalVerifyTx(txBz []byte) (sdk.Tx, error) - } - - // DefaultProposalHandler defines the default ABCI PrepareProposal and - // ProcessProposal handlers. - DefaultProposalHandler struct { - mempool mempool.Mempool - txVerifier ProposalTxVerifier - } -) - -func NewDefaultProposalHandler(mp mempool.Mempool, txVerifier ProposalTxVerifier) DefaultProposalHandler { - return DefaultProposalHandler{ - mempool: mp, - txVerifier: txVerifier, - } -} - -// PrepareProposalHandler returns the default implementation for processing an -// ABCI proposal. The application's mempool is enumerated and all valid -// transactions are added to the proposal. Transactions are valid if they: -// -// 1) Successfully encode to bytes. -// 2) Are valid (i.e. pass runTx, AnteHandler only). -// -// Enumeration is halted once RequestPrepareProposal.MaxBytes of transactions is -// reached or the mempool is exhausted. -// -// Note: -// -// - Step (2) is identical to the validation step performed in -// DefaultProcessProposal. It is very important that the same validation logic -// is used in both steps, and applications must ensure that this is the case in -// non-default handlers. -// -// - If no mempool is set or if the mempool is a no-op mempool, the transactions -// requested from Tendermint will simply be returned, which, by default, are in -// FIFO order. -func (h DefaultProposalHandler) PrepareProposalHandler() sdk.PrepareProposalHandler { - return func(ctx sdk.Context, req abci.RequestPrepareProposal) abci.ResponsePrepareProposal { - // If the mempool is nil or NoOp we simply return the transactions - // requested from CometBFT, which, by default, should be in FIFO order. - _, isNoOp := h.mempool.(mempool.NoOpMempool) - if h.mempool == nil || isNoOp { - return abci.ResponsePrepareProposal{Txs: req.Txs} - } - - var maxBlockGas int64 - if b := ctx.ConsensusParams().Block; b != nil { - maxBlockGas = b.MaxGas - } - - var ( - selectedTxs [][]byte - totalTxBytes int64 - totalTxGas uint64 - ) - - iterator := h.mempool.Select(ctx, req.Txs) - - for iterator != nil { - memTx := iterator.Tx() - - // NOTE: Since transaction verification was already executed in CheckTx, - // which calls mempool.Insert, in theory everything in the pool should be - // valid. But some mempool implementations may insert invalid txs, so we - // check again. - bz, err := h.txVerifier.PrepareProposalVerifyTx(memTx) - if err != nil { - err := h.mempool.Remove(memTx) - if err != nil && !errors.Is(err, mempool.ErrTxNotFound) { - panic(err) - } - } else { - var txGasLimit uint64 - txSize := int64(len(bz)) - - gasTx, ok := memTx.(interface{ GetGas() uint64 }) - if ok { - txGasLimit = gasTx.GetGas() - } - - // only add the transaction to the proposal if we have enough capacity - if (txSize + totalTxBytes) < req.MaxTxBytes { - // If there is a max block gas limit, add the tx only if the limit has - // not been met. - if maxBlockGas > 0 { - if (txGasLimit + totalTxGas) <= uint64(maxBlockGas) { - totalTxGas += txGasLimit - totalTxBytes += txSize - selectedTxs = append(selectedTxs, bz) - } - } else { - totalTxBytes += txSize - selectedTxs = append(selectedTxs, bz) - } - } - - // Check if we've reached capacity. If so, we cannot select any more - // transactions. - if totalTxBytes >= req.MaxTxBytes || (maxBlockGas > 0 && (totalTxGas >= uint64(maxBlockGas))) { - break - } - } - - iterator = iterator.Next() - } - - return abci.ResponsePrepareProposal{Txs: selectedTxs} - } -} - -// ProcessProposalHandler returns the default implementation for processing an -// ABCI proposal. Every transaction in the proposal must pass 2 conditions: -// -// 1. The transaction bytes must decode to a valid transaction. -// 2. The transaction must be valid (i.e. pass runTx, AnteHandler only) -// -// If any transaction fails to pass either condition, the proposal is rejected. -// Note that step (2) is identical to the validation step performed in -// DefaultPrepareProposal. It is very important that the same validation logic -// is used in both steps, and applications must ensure that this is the case in -// non-default handlers. -func (h DefaultProposalHandler) ProcessProposalHandler() sdk.ProcessProposalHandler { - // If the mempool is nil or NoOp we simply return ACCEPT, - // because PrepareProposal may have included txs that could fail verification. - _, isNoOp := h.mempool.(mempool.NoOpMempool) - if h.mempool == nil || isNoOp { - return NoOpProcessProposal() - } - - return func(ctx sdk.Context, req abci.RequestProcessProposal) abci.ResponseProcessProposal { - for _, txBytes := range req.Txs { - _, err := h.txVerifier.ProcessProposalVerifyTx(txBytes) - if err != nil { - return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_REJECT} - } - } - - return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT} - } -} - -// NoOpPrepareProposal defines a no-op PrepareProposal handler. It will always -// return the transactions sent by the client's request. -func NoOpPrepareProposal() sdk.PrepareProposalHandler { - return func(_ sdk.Context, req abci.RequestPrepareProposal) abci.ResponsePrepareProposal { - return abci.ResponsePrepareProposal{Txs: req.Txs} - } -} - -// NoOpProcessProposal defines a no-op ProcessProposal Handler. It will always -// return ACCEPT. -func NoOpProcessProposal() sdk.ProcessProposalHandler { - return func(_ sdk.Context, _ abci.RequestProcessProposal) abci.ResponseProcessProposal { - return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT} - } -======= -func (app *BaseApp) TxDecode(txBytes []byte) (sdk.Tx, error) { - return app.txDecoder(txBytes) -} - -func (app *BaseApp) TxEncode(tx sdk.Tx) ([]byte, error) { - return app.txEncoder(tx) ->>>>>>> 63d046dd5 (fix(baseapp): select txs correctly with no-op mempool (#17769)) -} +// // PrepareProposalHandler returns the default implementation for processing an +// // ABCI proposal. The application's mempool is enumerated and all valid +// // transactions are added to the proposal. Transactions are valid if they: +// // +// // 1) Successfully encode to bytes. +// // 2) Are valid (i.e. pass runTx, AnteHandler only). +// // +// // Enumeration is halted once RequestPrepareProposal.MaxBytes of transactions is +// // reached or the mempool is exhausted. +// // +// // Note: +// // +// // - Step (2) is identical to the validation step performed in +// // DefaultProcessProposal. It is very important that the same validation logic +// // is used in both steps, and applications must ensure that this is the case in +// // non-default handlers. +// // +// // - If no mempool is set or if the mempool is a no-op mempool, the transactions +// // requested from Tendermint will simply be returned, which, by default, are in +// // FIFO order. +// func (h DefaultProposalHandler) PrepareProposalHandler() sdk.PrepareProposalHandler { +// return func(ctx sdk.Context, req abci.RequestPrepareProposal) abci.ResponsePrepareProposal { +// // If the mempool is nil or NoOp we simply return the transactions +// // requested from CometBFT, which, by default, should be in FIFO order. +// _, isNoOp := h.mempool.(mempool.NoOpMempool) +// if h.mempool == nil || isNoOp { +// return abci.ResponsePrepareProposal{Txs: req.Txs} +// } + +// var maxBlockGas int64 +// if b := ctx.ConsensusParams().Block; b != nil { +// maxBlockGas = b.MaxGas +// } + +// var ( +// selectedTxs [][]byte +// totalTxBytes int64 +// totalTxGas uint64 +// ) + +// iterator := h.mempool.Select(ctx, req.Txs) + +// for iterator != nil { +// memTx := iterator.Tx() + +// // NOTE: Since transaction verification was already executed in CheckTx, +// // which calls mempool.Insert, in theory everything in the pool should be +// // valid. But some mempool implementations may insert invalid txs, so we +// // check again. +// bz, err := h.txVerifier.PrepareProposalVerifyTx(memTx) +// if err != nil { +// err := h.mempool.Remove(memTx) +// if err != nil && !errors.Is(err, mempool.ErrTxNotFound) { +// panic(err) +// } +// } else { +// var txGasLimit uint64 +// txSize := int64(len(bz)) + +// gasTx, ok := memTx.(interface{ GetGas() uint64 }) +// if ok { +// txGasLimit = gasTx.GetGas() +// } + +// // only add the transaction to the proposal if we have enough capacity +// if (txSize + totalTxBytes) < req.MaxTxBytes { +// // If there is a max block gas limit, add the tx only if the limit has +// // not been met. +// if maxBlockGas > 0 { +// if (txGasLimit + totalTxGas) <= uint64(maxBlockGas) { +// totalTxGas += txGasLimit +// totalTxBytes += txSize +// selectedTxs = append(selectedTxs, bz) +// } +// } else { +// totalTxBytes += txSize +// selectedTxs = append(selectedTxs, bz) +// } +// } + +// // Check if we've reached capacity. If so, we cannot select any more +// // transactions. +// if totalTxBytes >= req.MaxTxBytes || (maxBlockGas > 0 && (totalTxGas >= uint64(maxBlockGas))) { +// break +// } +// } + +// iterator = iterator.Next() +// } + +// return abci.ResponsePrepareProposal{Txs: selectedTxs} +// } +// } + +// // ProcessProposalHandler returns the default implementation for processing an +// // ABCI proposal. Every transaction in the proposal must pass 2 conditions: +// // +// // 1. The transaction bytes must decode to a valid transaction. +// // 2. The transaction must be valid (i.e. pass runTx, AnteHandler only) +// // +// // If any transaction fails to pass either condition, the proposal is rejected. +// // Note that step (2) is identical to the validation step performed in +// // DefaultPrepareProposal. It is very important that the same validation logic +// // is used in both steps, and applications must ensure that this is the case in +// // non-default handlers. +// func (h DefaultProposalHandler) ProcessProposalHandler() sdk.ProcessProposalHandler { +// // If the mempool is nil or NoOp we simply return ACCEPT, +// // because PrepareProposal may have included txs that could fail verification. +// _, isNoOp := h.mempool.(mempool.NoOpMempool) +// if h.mempool == nil || isNoOp { +// return NoOpProcessProposal() +// } + +// return func(ctx sdk.Context, req abci.RequestProcessProposal) abci.ResponseProcessProposal { +// for _, txBytes := range req.Txs { +// _, err := h.txVerifier.ProcessProposalVerifyTx(txBytes) +// if err != nil { +// return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_REJECT} +// } +// } + +// return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT} +// } +// } + +// // NoOpPrepareProposal defines a no-op PrepareProposal handler. It will always +// // return the transactions sent by the client's request. +// func NoOpPrepareProposal() sdk.PrepareProposalHandler { +// return func(_ sdk.Context, req abci.RequestPrepareProposal) abci.ResponsePrepareProposal { +// return abci.ResponsePrepareProposal{Txs: req.Txs} +// } +// } + +// // NoOpProcessProposal defines a no-op ProcessProposal Handler. It will always +// // return ACCEPT. +// func NoOpProcessProposal() sdk.ProcessProposalHandler { +// return func(_ sdk.Context, _ abci.RequestProcessProposal) abci.ResponseProcessProposal { +// return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT} +// } +// } // Close is called in start cmd to gracefully cleanup resources. func (app *BaseApp) Close() error { From 867dcd4e324dafa47de0c9027213251abf13368f Mon Sep 17 00:00:00 2001 From: Aleksandr Bezobchuk Date: Mon, 25 Sep 2023 09:06:33 -0700 Subject: [PATCH 5/7] updates --- baseapp/baseapp.go | 141 --------------------------------------------- 1 file changed, 141 deletions(-) diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index 7aec0ecc0628..7c82ff493dee 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -906,147 +906,6 @@ func (app *BaseApp) ProcessProposalVerifyTx(txBz []byte) (sdk.Tx, error) { return tx, nil } -// // PrepareProposalHandler returns the default implementation for processing an -// // ABCI proposal. The application's mempool is enumerated and all valid -// // transactions are added to the proposal. Transactions are valid if they: -// // -// // 1) Successfully encode to bytes. -// // 2) Are valid (i.e. pass runTx, AnteHandler only). -// // -// // Enumeration is halted once RequestPrepareProposal.MaxBytes of transactions is -// // reached or the mempool is exhausted. -// // -// // Note: -// // -// // - Step (2) is identical to the validation step performed in -// // DefaultProcessProposal. It is very important that the same validation logic -// // is used in both steps, and applications must ensure that this is the case in -// // non-default handlers. -// // -// // - If no mempool is set or if the mempool is a no-op mempool, the transactions -// // requested from Tendermint will simply be returned, which, by default, are in -// // FIFO order. -// func (h DefaultProposalHandler) PrepareProposalHandler() sdk.PrepareProposalHandler { -// return func(ctx sdk.Context, req abci.RequestPrepareProposal) abci.ResponsePrepareProposal { -// // If the mempool is nil or NoOp we simply return the transactions -// // requested from CometBFT, which, by default, should be in FIFO order. -// _, isNoOp := h.mempool.(mempool.NoOpMempool) -// if h.mempool == nil || isNoOp { -// return abci.ResponsePrepareProposal{Txs: req.Txs} -// } - -// var maxBlockGas int64 -// if b := ctx.ConsensusParams().Block; b != nil { -// maxBlockGas = b.MaxGas -// } - -// var ( -// selectedTxs [][]byte -// totalTxBytes int64 -// totalTxGas uint64 -// ) - -// iterator := h.mempool.Select(ctx, req.Txs) - -// for iterator != nil { -// memTx := iterator.Tx() - -// // NOTE: Since transaction verification was already executed in CheckTx, -// // which calls mempool.Insert, in theory everything in the pool should be -// // valid. But some mempool implementations may insert invalid txs, so we -// // check again. -// bz, err := h.txVerifier.PrepareProposalVerifyTx(memTx) -// if err != nil { -// err := h.mempool.Remove(memTx) -// if err != nil && !errors.Is(err, mempool.ErrTxNotFound) { -// panic(err) -// } -// } else { -// var txGasLimit uint64 -// txSize := int64(len(bz)) - -// gasTx, ok := memTx.(interface{ GetGas() uint64 }) -// if ok { -// txGasLimit = gasTx.GetGas() -// } - -// // only add the transaction to the proposal if we have enough capacity -// if (txSize + totalTxBytes) < req.MaxTxBytes { -// // If there is a max block gas limit, add the tx only if the limit has -// // not been met. -// if maxBlockGas > 0 { -// if (txGasLimit + totalTxGas) <= uint64(maxBlockGas) { -// totalTxGas += txGasLimit -// totalTxBytes += txSize -// selectedTxs = append(selectedTxs, bz) -// } -// } else { -// totalTxBytes += txSize -// selectedTxs = append(selectedTxs, bz) -// } -// } - -// // Check if we've reached capacity. If so, we cannot select any more -// // transactions. -// if totalTxBytes >= req.MaxTxBytes || (maxBlockGas > 0 && (totalTxGas >= uint64(maxBlockGas))) { -// break -// } -// } - -// iterator = iterator.Next() -// } - -// return abci.ResponsePrepareProposal{Txs: selectedTxs} -// } -// } - -// // ProcessProposalHandler returns the default implementation for processing an -// // ABCI proposal. Every transaction in the proposal must pass 2 conditions: -// // -// // 1. The transaction bytes must decode to a valid transaction. -// // 2. The transaction must be valid (i.e. pass runTx, AnteHandler only) -// // -// // If any transaction fails to pass either condition, the proposal is rejected. -// // Note that step (2) is identical to the validation step performed in -// // DefaultPrepareProposal. It is very important that the same validation logic -// // is used in both steps, and applications must ensure that this is the case in -// // non-default handlers. -// func (h DefaultProposalHandler) ProcessProposalHandler() sdk.ProcessProposalHandler { -// // If the mempool is nil or NoOp we simply return ACCEPT, -// // because PrepareProposal may have included txs that could fail verification. -// _, isNoOp := h.mempool.(mempool.NoOpMempool) -// if h.mempool == nil || isNoOp { -// return NoOpProcessProposal() -// } - -// return func(ctx sdk.Context, req abci.RequestProcessProposal) abci.ResponseProcessProposal { -// for _, txBytes := range req.Txs { -// _, err := h.txVerifier.ProcessProposalVerifyTx(txBytes) -// if err != nil { -// return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_REJECT} -// } -// } - -// return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT} -// } -// } - -// // NoOpPrepareProposal defines a no-op PrepareProposal handler. It will always -// // return the transactions sent by the client's request. -// func NoOpPrepareProposal() sdk.PrepareProposalHandler { -// return func(_ sdk.Context, req abci.RequestPrepareProposal) abci.ResponsePrepareProposal { -// return abci.ResponsePrepareProposal{Txs: req.Txs} -// } -// } - -// // NoOpProcessProposal defines a no-op ProcessProposal Handler. It will always -// // return ACCEPT. -// func NoOpProcessProposal() sdk.ProcessProposalHandler { -// return func(_ sdk.Context, _ abci.RequestProcessProposal) abci.ResponseProcessProposal { -// return abci.ResponseProcessProposal{Status: abci.ResponseProcessProposal_ACCEPT} -// } -// } - // Close is called in start cmd to gracefully cleanup resources. func (app *BaseApp) Close() error { return nil From c90bcfa3109343fa80485a50a306bffb32830385 Mon Sep 17 00:00:00 2001 From: Aleksandr Bezobchuk Date: Mon, 25 Sep 2023 09:13:53 -0700 Subject: [PATCH 6/7] updates --- baseapp/abci_utils_test.go | 379 ------------------------------------- baseapp/baseapp_test.go | 88 +++++++++ 2 files changed, 88 insertions(+), 379 deletions(-) delete mode 100644 baseapp/abci_utils_test.go diff --git a/baseapp/abci_utils_test.go b/baseapp/abci_utils_test.go deleted file mode 100644 index 8919ee81ba8c..000000000000 --- a/baseapp/abci_utils_test.go +++ /dev/null @@ -1,379 +0,0 @@ -package baseapp_test - -import ( - "bytes" - "testing" - - abci "github.com/cometbft/cometbft/abci/types" - "github.com/cometbft/cometbft/crypto/secp256k1" - cmtprotocrypto "github.com/cometbft/cometbft/proto/tendermint/crypto" - cmtproto "github.com/cometbft/cometbft/proto/tendermint/types" - dbm "github.com/cosmos/cosmos-db" - protoio "github.com/cosmos/gogoproto/io" - "github.com/cosmos/gogoproto/proto" - "github.com/golang/mock/gomock" - "github.com/stretchr/testify/suite" - - "cosmossdk.io/log" - - "github.com/cosmos/cosmos-sdk/baseapp" - baseapptestutil "github.com/cosmos/cosmos-sdk/baseapp/testutil" - "github.com/cosmos/cosmos-sdk/baseapp/testutil/mock" - codectestutil "github.com/cosmos/cosmos-sdk/codec/testutil" - "github.com/cosmos/cosmos-sdk/testutil/testdata" - sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/types/mempool" - authtx "github.com/cosmos/cosmos-sdk/x/auth/tx" -) - -const ( - chainID = "chain-id" -) - -type testValidator struct { - consAddr sdk.ConsAddress - tmPk cmtprotocrypto.PublicKey - privKey secp256k1.PrivKey -} - -func newTestValidator() testValidator { - privkey := secp256k1.GenPrivKey() - pubkey := privkey.PubKey() - tmPk := cmtprotocrypto.PublicKey{ - Sum: &cmtprotocrypto.PublicKey_Secp256K1{ - Secp256K1: pubkey.Bytes(), - }, - } - - return testValidator{ - consAddr: sdk.ConsAddress(pubkey.Address()), - tmPk: tmPk, - privKey: privkey, - } -} - -func (t testValidator) toValidator(power int64) abci.Validator { - return abci.Validator{ - Address: t.consAddr.Bytes(), - Power: power, - } -} - -type ABCIUtilsTestSuite struct { - suite.Suite - - valStore *mock.MockValidatorStore - vals [3]testValidator - ctx sdk.Context -} - -func NewABCIUtilsTestSuite(t *testing.T) *ABCIUtilsTestSuite { - t.Helper() - // create 3 validators - s := &ABCIUtilsTestSuite{ - vals: [3]testValidator{ - newTestValidator(), - newTestValidator(), - newTestValidator(), - }, - } - - // create mock - ctrl := gomock.NewController(t) - valStore := mock.NewMockValidatorStore(ctrl) - s.valStore = valStore - - // set up mock - s.valStore.EXPECT().GetPubKeyByConsAddr(gomock.Any(), s.vals[0].consAddr.Bytes()).Return(s.vals[0].tmPk, nil).AnyTimes() - s.valStore.EXPECT().GetPubKeyByConsAddr(gomock.Any(), s.vals[1].consAddr.Bytes()).Return(s.vals[1].tmPk, nil).AnyTimes() - s.valStore.EXPECT().GetPubKeyByConsAddr(gomock.Any(), s.vals[2].consAddr.Bytes()).Return(s.vals[2].tmPk, nil).AnyTimes() - - // create context - s.ctx = sdk.Context{}.WithConsensusParams(cmtproto.ConsensusParams{ - Abci: &cmtproto.ABCIParams{ - VoteExtensionsEnableHeight: 2, - }, - }) - return s -} - -func TestABCIUtilsTestSuite(t *testing.T) { - suite.Run(t, NewABCIUtilsTestSuite(t)) -} - -// check ValidateVoteExtensions works when all nodes have CommitBlockID votes -func (s *ABCIUtilsTestSuite) TestValidateVoteExtensionsHappyPath() { - ext := []byte("vote-extension") - cve := cmtproto.CanonicalVoteExtension{ - Extension: ext, - Height: 2, - Round: int64(0), - ChainId: chainID, - } - - bz, err := marshalDelimitedFn(&cve) - s.Require().NoError(err) - - extSig0, err := s.vals[0].privKey.Sign(bz) - s.Require().NoError(err) - - extSig1, err := s.vals[1].privKey.Sign(bz) - s.Require().NoError(err) - - extSig2, err := s.vals[2].privKey.Sign(bz) - s.Require().NoError(err) - - llc := abci.ExtendedCommitInfo{ - Round: 0, - Votes: []abci.ExtendedVoteInfo{ - { - Validator: s.vals[0].toValidator(333), - VoteExtension: ext, - ExtensionSignature: extSig0, - BlockIdFlag: cmtproto.BlockIDFlagCommit, - }, - { - Validator: s.vals[1].toValidator(333), - VoteExtension: ext, - ExtensionSignature: extSig1, - BlockIdFlag: cmtproto.BlockIDFlagCommit, - }, - { - Validator: s.vals[2].toValidator(334), - VoteExtension: ext, - ExtensionSignature: extSig2, - BlockIdFlag: cmtproto.BlockIDFlagCommit, - }, - }, - } - // expect-pass (votes of height 2 are included in next block) - s.Require().NoError(baseapp.ValidateVoteExtensions(s.ctx, s.valStore, 3, chainID, llc)) -} - -// check ValidateVoteExtensions works when a single node has submitted a BlockID_Absent -func (s *ABCIUtilsTestSuite) TestValidateVoteExtensionsSingleVoteAbsent() { - ext := []byte("vote-extension") - cve := cmtproto.CanonicalVoteExtension{ - Extension: ext, - Height: 2, - Round: int64(0), - ChainId: chainID, - } - - bz, err := marshalDelimitedFn(&cve) - s.Require().NoError(err) - - extSig0, err := s.vals[0].privKey.Sign(bz) - s.Require().NoError(err) - - extSig2, err := s.vals[2].privKey.Sign(bz) - s.Require().NoError(err) - - llc := abci.ExtendedCommitInfo{ - Round: 0, - Votes: []abci.ExtendedVoteInfo{ - { - Validator: s.vals[0].toValidator(333), - VoteExtension: ext, - ExtensionSignature: extSig0, - BlockIdFlag: cmtproto.BlockIDFlagCommit, - }, - // validator of power <1/3 is missing, so commit-info shld still be valid - { - Validator: s.vals[1].toValidator(333), - BlockIdFlag: cmtproto.BlockIDFlagAbsent, - }, - { - Validator: s.vals[2].toValidator(334), - VoteExtension: ext, - ExtensionSignature: extSig2, - BlockIdFlag: cmtproto.BlockIDFlagCommit, - }, - }, - } - // expect-pass (votes of height 2 are included in next block) - s.Require().NoError(baseapp.ValidateVoteExtensions(s.ctx, s.valStore, 3, chainID, llc)) -} - -// check ValidateVoteExtensions works when a single node has submitted a BlockID_Nil -func (s *ABCIUtilsTestSuite) TestValidateVoteExtensionsSingleVoteNil() { - ext := []byte("vote-extension") - cve := cmtproto.CanonicalVoteExtension{ - Extension: ext, - Height: 2, - Round: int64(0), - ChainId: chainID, - } - - bz, err := marshalDelimitedFn(&cve) - s.Require().NoError(err) - - extSig0, err := s.vals[0].privKey.Sign(bz) - s.Require().NoError(err) - - extSig2, err := s.vals[2].privKey.Sign(bz) - s.Require().NoError(err) - - llc := abci.ExtendedCommitInfo{ - Round: 0, - Votes: []abci.ExtendedVoteInfo{ - { - Validator: s.vals[0].toValidator(333), - VoteExtension: ext, - ExtensionSignature: extSig0, - BlockIdFlag: cmtproto.BlockIDFlagCommit, - }, - // validator of power <1/3 is missing, so commit-info should still be valid - { - Validator: s.vals[1].toValidator(333), - BlockIdFlag: cmtproto.BlockIDFlagNil, - }, - { - Validator: s.vals[2].toValidator(334), - VoteExtension: ext, - ExtensionSignature: extSig2, - BlockIdFlag: cmtproto.BlockIDFlagCommit, - }, - }, - } - // expect-pass (votes of height 2 are included in next block) - s.Require().NoError(baseapp.ValidateVoteExtensions(s.ctx, s.valStore, 3, chainID, llc)) -} - -// check ValidateVoteExtensions works when two nodes have submitted a BlockID_Nil / BlockID_Absent -func (s *ABCIUtilsTestSuite) TestValidateVoteExtensionsTwoVotesNilAbsent() { - ext := []byte("vote-extension") - cve := cmtproto.CanonicalVoteExtension{ - Extension: ext, - Height: 2, - Round: int64(0), - ChainId: chainID, - } - - bz, err := marshalDelimitedFn(&cve) - s.Require().NoError(err) - - extSig0, err := s.vals[0].privKey.Sign(bz) - s.Require().NoError(err) - - llc := abci.ExtendedCommitInfo{ - Round: 0, - Votes: []abci.ExtendedVoteInfo{ - // validator of power >2/3 is missing, so commit-info should not be valid - { - Validator: s.vals[0].toValidator(333), - BlockIdFlag: cmtproto.BlockIDFlagCommit, - VoteExtension: ext, - ExtensionSignature: extSig0, - }, - { - Validator: s.vals[1].toValidator(333), - BlockIdFlag: cmtproto.BlockIDFlagNil, - }, - { - Validator: s.vals[2].toValidator(334), - VoteExtension: ext, - BlockIdFlag: cmtproto.BlockIDFlagAbsent, - }, - }, - } - - // expect-pass (votes of height 2 are included in next block) - s.Require().Error(baseapp.ValidateVoteExtensions(s.ctx, s.valStore, 3, chainID, llc)) -} - -func (s *ABCIUtilsTestSuite) TestDefaultProposalHandler_NoOpMempoolTxSelection() { - // create a codec for marshaling - cdc := codectestutil.CodecOptions{}.NewCodec() - baseapptestutil.RegisterInterfaces(cdc.InterfaceRegistry()) - - // create a baseapp along with a tx config for tx generation - txConfig := authtx.NewTxConfig(cdc, authtx.DefaultSignModes) - app := baseapp.NewBaseApp(s.T().Name(), log.NewNopLogger(), dbm.NewMemDB(), txConfig.TxDecoder()) - - // create a proposal handler - ph := baseapp.NewDefaultProposalHandler(mempool.NoOpMempool{}, app) - handler := ph.PrepareProposalHandler() - - // build a tx - _, _, addr := testdata.KeyTestPubAddr() - builder := txConfig.NewTxBuilder() - s.Require().NoError(builder.SetMsgs( - &baseapptestutil.MsgCounter{Counter: 0, FailOnHandler: false, Signer: addr.String()}, - )) - builder.SetGasLimit(100) - setTxSignature(s.T(), builder, 0) - - // encode the tx to be used in the proposal request - tx := builder.GetTx() - txBz, err := txConfig.TxEncoder()(tx) - s.Require().NoError(err) - s.Require().Len(txBz, 152) - - testCases := map[string]struct { - ctx sdk.Context - req *abci.RequestPrepareProposal - expectedTxs int - }{ - "small max tx bytes": { - ctx: s.ctx, - req: &abci.RequestPrepareProposal{ - Txs: [][]byte{txBz, txBz, txBz, txBz, txBz}, - MaxTxBytes: 10, - }, - expectedTxs: 0, - }, - "small max gas": { - ctx: s.ctx.WithConsensusParams(cmtproto.ConsensusParams{ - Block: &cmtproto.BlockParams{ - MaxGas: 10, - }, - }), - req: &abci.RequestPrepareProposal{ - Txs: [][]byte{txBz, txBz, txBz, txBz, txBz}, - MaxTxBytes: 456, - }, - expectedTxs: 0, - }, - "large max tx bytes": { - ctx: s.ctx, - req: &abci.RequestPrepareProposal{ - Txs: [][]byte{txBz, txBz, txBz, txBz, txBz}, - MaxTxBytes: 456, - }, - expectedTxs: 3, - }, - "max gas and tx bytes": { - ctx: s.ctx.WithConsensusParams(cmtproto.ConsensusParams{ - Block: &cmtproto.BlockParams{ - MaxGas: 200, - }, - }), - req: &abci.RequestPrepareProposal{ - Txs: [][]byte{txBz, txBz, txBz, txBz, txBz}, - MaxTxBytes: 456, - }, - expectedTxs: 2, - }, - } - - for name, tc := range testCases { - s.Run(name, func() { - // iterate multiple times to ensure the tx selector is cleared each time - for i := 0; i < 5; i++ { - resp, err := handler(tc.ctx, tc.req) - s.Require().NoError(err) - s.Require().Len(resp.Txs, tc.expectedTxs) - } - }) - } -} - -func marshalDelimitedFn(msg proto.Message) ([]byte, error) { - var buf bytes.Buffer - if err := protoio.NewDelimitedWriter(&buf).WriteMsg(msg); err != nil { - return nil, err - } - - return buf.Bytes(), nil -} diff --git a/baseapp/baseapp_test.go b/baseapp/baseapp_test.go index a821ca1e4456..08f733c39e7e 100644 --- a/baseapp/baseapp_test.go +++ b/baseapp/baseapp_test.go @@ -25,6 +25,7 @@ import ( "github.com/cosmos/cosmos-sdk/testutil" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" + "github.com/cosmos/cosmos-sdk/types/mempool" authtx "github.com/cosmos/cosmos-sdk/x/auth/tx" ) @@ -654,3 +655,90 @@ func TestLoadVersionPruning(t *testing.T) { require.Nil(t, err) testLoadVersionHelper(t, app, int64(7), lastCommitID) } + +func TestDefaultProposalHandler_NoOpMempoolTxSelection(t *testing.T) { + // create a codec for marshaling + cdc := codec.NewProtoCodec(codectypes.NewInterfaceRegistry()) + baseapptestutil.RegisterInterfaces(cdc.InterfaceRegistry()) + + // create a baseapp along with a tx config for tx generation + txConfig := authtx.NewTxConfig(cdc, authtx.DefaultSignModes) + app := baseapp.NewBaseApp(t.Name(), log.NewNopLogger(), dbm.NewMemDB(), txConfig.TxDecoder()) + + // create a proposal handler + ph := baseapp.NewDefaultProposalHandler(mempool.NoOpMempool{}, app) + handler := ph.PrepareProposalHandler() + + // build a tx + builder := txConfig.NewTxBuilder() + require.NoError(t, builder.SetMsgs( + &baseapptestutil.MsgCounter{Counter: 0, FailOnHandler: false}, + )) + builder.SetGasLimit(100) + setTxSignature(t, builder, 0) + + // encode the tx to be used in the proposal request + tx := builder.GetTx() + txBz, err := txConfig.TxEncoder()(tx) + require.NoError(t, err) + require.Len(t, txBz, 152) + + ctx := app.NewContext(true, tmproto.Header{}) + + testCases := map[string]struct { + ctx sdk.Context + req abci.RequestPrepareProposal + expectedTxs int + }{ + "small max tx bytes": { + ctx: ctx, + req: abci.RequestPrepareProposal{ + Txs: [][]byte{txBz, txBz, txBz, txBz, txBz}, + MaxTxBytes: 10, + }, + expectedTxs: 0, + }, + "small max gas": { + ctx: ctx.WithConsensusParams(&tmproto.ConsensusParams{ + Block: &tmproto.BlockParams{ + MaxGas: 10, + }, + }), + req: abci.RequestPrepareProposal{ + Txs: [][]byte{txBz, txBz, txBz, txBz, txBz}, + MaxTxBytes: 456, + }, + expectedTxs: 0, + }, + "large max tx bytes": { + ctx: ctx, + req: abci.RequestPrepareProposal{ + Txs: [][]byte{txBz, txBz, txBz, txBz, txBz}, + MaxTxBytes: 456, + }, + expectedTxs: 3, + }, + "max gas and tx bytes": { + ctx: ctx.WithConsensusParams(&tmproto.ConsensusParams{ + Block: &tmproto.BlockParams{ + MaxGas: 200, + }, + }), + req: abci.RequestPrepareProposal{ + Txs: [][]byte{txBz, txBz, txBz, txBz, txBz}, + MaxTxBytes: 456, + }, + expectedTxs: 2, + }, + } + + for name, tc := range testCases { + t.Run(name, func(t *testing.T) { + // iterate multiple times to ensure the tx selector is cleared each time + for i := 0; i < 5; i++ { + resp := handler(tc.ctx, tc.req) + require.Len(t, resp.Txs, tc.expectedTxs) + } + }) + } +} From 5cae627b5639f0a9cf5513d109ca62cf42d248b8 Mon Sep 17 00:00:00 2001 From: Aleksandr Bezobchuk Date: Mon, 25 Sep 2023 09:32:23 -0700 Subject: [PATCH 7/7] updates --- baseapp/abci_utils.go | 1 + baseapp/baseapp_test.go | 15 ++++++++------- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/baseapp/abci_utils.go b/baseapp/abci_utils.go index 87507d836ab1..4d00ce4eb473 100644 --- a/baseapp/abci_utils.go +++ b/baseapp/abci_utils.go @@ -82,6 +82,7 @@ func (h *DefaultProposalHandler) PrepareProposalHandler() sdk.PrepareProposalHan for _, txBz := range req.Txs { // XXX: We pass nil as the memTx because we have no way of decoding the // txBz. We'd need to break (update) the ProposalTxVerifier interface. + // As a result, we CANNOT account for block max gas. stop := h.txSelector.SelectTxForProposal(uint64(req.MaxTxBytes), maxBlockGas, nil, txBz) if stop { break diff --git a/baseapp/baseapp_test.go b/baseapp/baseapp_test.go index 08f733c39e7e..84614a9a6f5f 100644 --- a/baseapp/baseapp_test.go +++ b/baseapp/baseapp_test.go @@ -681,9 +681,10 @@ func TestDefaultProposalHandler_NoOpMempoolTxSelection(t *testing.T) { tx := builder.GetTx() txBz, err := txConfig.TxEncoder()(tx) require.NoError(t, err) - require.Len(t, txBz, 152) + require.Len(t, txBz, 103) - ctx := app.NewContext(true, tmproto.Header{}) + ctx := sdk.NewContext(nil, tmproto.Header{}, false, nil). + WithConsensusParams(&tmproto.ConsensusParams{}) testCases := map[string]struct { ctx sdk.Context @@ -706,15 +707,15 @@ func TestDefaultProposalHandler_NoOpMempoolTxSelection(t *testing.T) { }), req: abci.RequestPrepareProposal{ Txs: [][]byte{txBz, txBz, txBz, txBz, txBz}, - MaxTxBytes: 456, + MaxTxBytes: 309, }, - expectedTxs: 0, + expectedTxs: 3, }, "large max tx bytes": { ctx: ctx, req: abci.RequestPrepareProposal{ Txs: [][]byte{txBz, txBz, txBz, txBz, txBz}, - MaxTxBytes: 456, + MaxTxBytes: 309, }, expectedTxs: 3, }, @@ -726,9 +727,9 @@ func TestDefaultProposalHandler_NoOpMempoolTxSelection(t *testing.T) { }), req: abci.RequestPrepareProposal{ Txs: [][]byte{txBz, txBz, txBz, txBz, txBz}, - MaxTxBytes: 456, + MaxTxBytes: 309, }, - expectedTxs: 2, + expectedTxs: 3, }, }