Synopsis This sections describes how the app-side mempool can be used and replaced.
Since v0.47 the application has its own mempool to allow much more granular block building than previous versions. This change was enabled by ABCI 1.0. Notably it introduces the PrepareProposal and ProcessProposal steps of ABCI++.
Pre-requisite Readings

Prepare Proposal

PrepareProposal handles construction of the block, meaning that when a proposer is preparing to propose a block, it requests the application to evaluate a RequestPrepareProposal, which contains a series of transactions from CometBFT’s mempool. At this point, the application has complete control over the proposal. It can modify, delete, and inject transactions from it’s own app-side mempool into the proposal or even ignore all the transactions altogether. What the application does with the transactions provided to it by RequestPrepareProposal have no effect on CometBFT’s mempool. Note, that the application defines the semantics of the PrepareProposal and it MAY be non-deterministic and is only executed by the current block proposer. Now, reading mempool twice in the previous sentence is confusing, lets break it down. CometBFT has a mempool that handles gossiping transactions to other nodes in the network. How these transactions are ordered is determined by CometBFT’s mempool, typically FIFO. However, since the application is able to fully inspect all transactions, it can provide greater control over transaction ordering. Allowing the application to handle ordering enables the application to define how it would like the block constructed. The Cosmos SDK defines the DefaultProposalHandler type, which provides applications with PrepareProposal and ProcessProposal handlers. If you decide to implement your own PrepareProposal handler, you must be sure to ensure that the transactions selected DO NOT exceed the maximum block gas (if set) and the maximum bytes provided by req.MaxBytes.
package baseapp

import (
    
	"bytes"
    "fmt"
    "cosmossdk.io/math"
    "github.com/cockroachdb/errors"
	abci "github.com/cometbft/cometbft/abci/types"
	cmtcrypto "github.com/cometbft/cometbft/crypto"
	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"

	cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types"
	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 (
	/ Validator 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.
	Validator interface {
    CmtConsPublicKey() (cmtprotocrypto.PublicKey, error)

BondedTokens()

math.Int
}

	/ 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 {
    GetValidatorByConsAddr(sdk.Context, cryptotypes.Address) (Validator, error)

TotalBondedTokens(ctx sdk.Context)

math.Int
}
)

/ 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 ProcessProposal. 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 && 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 sumVP math.Int
    for _, vote := range extCommit.Votes {
    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 := cmtcrypto.Address(vote.Validator.Address)

validator, err := valStore.GetValidatorByConsAddr(ctx, valConsAddr)
    if err != nil {
    return fmt.Errorf("failed to get validator %X: %w", valConsAddr, err)
}
    if validator == nil {
    return fmt.Errorf("validator %X not found", valConsAddr)
}

cmtPubKeyProto, err := validator.CmtConsPublicKey()
    if err != nil {
    return fmt.Errorf("failed to get validator %X public key: %w", valConsAddr, err)
}

cmtPubKey, err := cryptoenc.PubKeyFromProto(cmtPubKeyProto)
    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 = sumVP.Add(validator.BondedTokens())
}

	/ Ensure we have at least 2/3 voting power that submitted valid vote
	/ extensions.
    totalVP := valStore.TotalBondedTokens(ctx)
    percentSubmitted := math.LegacyNewDecFromInt(sumVP).Quo(math.LegacyNewDecFromInt(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)
}

	/ 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 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) {
		/ 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
}, nil
}

var (
			selectedTxs  [][]byte
			totalTxBytes int64
		)
    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 {
    txSize := int64(len(bz))
    if totalTxBytes += txSize; totalTxBytes <= req.MaxTxBytes {
    selectedTxs = append(selectedTxs, bz)
}

else {
					/ We've reached capacity per req.MaxTxBytes so we cannot select any
					/ more transactions.
					break
}
	
}

iterator = iterator.Next()
}

return &abci.ResponsePrepareProposal{
    Txs: 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) {
    for _, txBytes := range req.Txs {
			_, err := h.txVerifier.ProcessProposalVerifyTx(txBytes)
    if err != nil {
    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
}
}
This default implementation can be overridden by the application developer in favor of a custom implementation in app.go:
prepareOpt := func(app *baseapp.BaseApp) {
    abciPropHandler := baseapp.NewDefaultProposalHandler(mempool, app)

app.SetPrepareProposal(abciPropHandler.PrepareProposalHandler())
}

baseAppOptions = append(baseAppOptions, prepareOpt)

Process Proposal

ProcessProposal handles the validation of a proposal from PrepareProposal, which also includes a block header. Meaning, that after a block has been proposed the other validators have the right to vote on a block. The validator in the default implementation of PrepareProposal runs basic validity checks on each transaction. Note, ProcessProposal MAY NOT be non-deterministic, i.e. it must be deterministic. This means if ProcessProposal panics or fails and we reject, all honest validator processes will prevote nil and the CometBFT round will proceed again until a valid proposal is proposed. Here is the implementation of the default implementation:
package baseapp

import (
    
	"bytes"
    "fmt"
    "cosmossdk.io/math"
    "github.com/cockroachdb/errors"
	abci "github.com/cometbft/cometbft/abci/types"
	cmtcrypto "github.com/cometbft/cometbft/crypto"
	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"

	cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types"
	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 (
	/ Validator 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.
	Validator interface {
    CmtConsPublicKey() (cmtprotocrypto.PublicKey, error)

BondedTokens()

math.Int
}

	/ 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 {
    GetValidatorByConsAddr(sdk.Context, cryptotypes.Address) (Validator, error)

TotalBondedTokens(ctx sdk.Context)

math.Int
}
)

/ 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 ProcessProposal. 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 && 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 sumVP math.Int
    for _, vote := range extCommit.Votes {
    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 := cmtcrypto.Address(vote.Validator.Address)

validator, err := valStore.GetValidatorByConsAddr(ctx, valConsAddr)
    if err != nil {
    return fmt.Errorf("failed to get validator %X: %w", valConsAddr, err)
}
    if validator == nil {
    return fmt.Errorf("validator %X not found", valConsAddr)
}

cmtPubKeyProto, err := validator.CmtConsPublicKey()
    if err != nil {
    return fmt.Errorf("failed to get validator %X public key: %w", valConsAddr, err)
}

cmtPubKey, err := cryptoenc.PubKeyFromProto(cmtPubKeyProto)
    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 = sumVP.Add(validator.BondedTokens())
}

	/ Ensure we have at least 2/3 voting power that submitted valid vote
	/ extensions.
    totalVP := valStore.TotalBondedTokens(ctx)
    percentSubmitted := math.LegacyNewDecFromInt(sumVP).Quo(math.LegacyNewDecFromInt(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)
}

	/ 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 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) {
		/ 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
}, nil
}

var (
			selectedTxs  [][]byte
			totalTxBytes int64
		)
    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 {
    txSize := int64(len(bz))
    if totalTxBytes += txSize; totalTxBytes <= req.MaxTxBytes {
    selectedTxs = append(selectedTxs, bz)
}

else {
					/ We've reached capacity per req.MaxTxBytes so we cannot select any
					/ more transactions.
					break
}
	
}

iterator = iterator.Next()
}

return &abci.ResponsePrepareProposal{
    Txs: 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) {
    for _, txBytes := range req.Txs {
			_, err := h.txVerifier.ProcessProposalVerifyTx(txBytes)
    if err != nil {
    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
}
}
Like PrepareProposal this implementation is the default and can be modified by the application developer in app.go. If you decide to implement your own ProcessProposal handler, you must be sure to ensure that the transactions provided in the proposal DO NOT exceed the maximum block gas (if set).
processOpt := func(app *baseapp.BaseApp) {
    abciPropHandler := baseapp.NewDefaultProposalHandler(mempool, app)

app.SetProcessProposal(abciPropHandler.ProcessProposalHandler())
}

baseAppOptions = append(baseAppOptions, processOpt)

Mempool

Now that we have walked through the PrepareProposal & ProcessProposal, we can move on to walking through the mempool. There are countless designs that an application developer can write for a mempool, the SDK opted to provide only simple mempool implementations. Namely, the SDK provides the following mempools: The default SDK is a No-op Mempool, but it can be replaced by the application developer in app.go:
nonceMempool := mempool.NewSenderNonceMempool()
    mempoolOpt   := baseapp.SetMempool(nonceMempool)

baseAppOptions = append(baseAppOptions, mempoolOpt)

No-op Mempool

A no-op mempool is a mempool where transactions are completely discarded and ignored when BaseApp interacts with the mempool. When this mempool is used, it assumed that an application will rely on CometBFT’s transaction ordering defined in RequestPrepareProposal, which is FIFO-ordered by default.
Note: If a NoOp mempool is used, PrepareProposal and ProcessProposal both should be aware of this as PrepareProposal could include transactions that could fail verification in ProcessProposal.

Sender Nonce Mempool

The nonce mempool is a mempool that keeps transactions from an sorted by nonce in order to avoid the issues with nonces. It works by storing the transaction in a list sorted by the transaction nonce. When the proposer asks for transactions to be included in a block it randomly selects a sender and gets the first transaction in the list. It repeats this until the mempool is empty or the block is full. It is configurable with the following parameters:

MaxTxs

It is an integer value that sets the mempool in one of three modes, bounded, unbounded, or disabled.
  • negative: Disabled, mempool does not insert new transaction and return early.
  • zero: Unbounded mempool has no transaction limit and will never fail with ErrMempoolTxMaxCapacity.
  • positive: Bounded, it fails with ErrMempoolTxMaxCapacity when maxTx value is the same as CountTx()

Seed

Set the seed for the random number generator used to select transactions from the mempool.

Priority Nonce Mempool

The priority nonce mempool is a mempool implementation that stores txs in a partially ordered set by 2 dimensions:
  • priority
  • sender-nonce (sequence number)
Internally it uses one priority ordered skip list and one skip list per sender ordered by sender-nonce (sequence number). When there are multiple txs from the same sender, they are not always comparable by priority to other sender txs and must be partially ordered by both sender-nonce and priority. It is configurable with the following parameters:

MaxTxs

It is an integer value that sets the mempool in one of three modes, bounded, unbounded, or disabled.
  • negative: Disabled, mempool does not insert new transaction and return early.
  • zero: Unbounded mempool has no transaction limit and will never fail with ErrMempoolTxMaxCapacity.
  • positive: Bounded, it fails with ErrMempoolTxMaxCapacity when maxTx value is the same as CountTx()

Callback

The priority nonce mempool provides mempool options allowing the application sets callback(s).
  • OnRead: Set a callback to be called when a transaction is read from the mempool.
  • TxReplacement: Sets a callback to be called when duplicated transaction nonce detected during mempool insert. Application can define a transaction replacement rule based on tx priority or certain transaction fields.
More information on the SDK mempool implementation can be found in the godocs.