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
}
}