From 6d12b1df8c2cbb41af335fdc743be59eaf4f051f Mon Sep 17 00:00:00 2001 From: Facundo Medica <14063057+facundomedica@users.noreply.github.com> Date: Thu, 19 Dec 2024 13:40:02 +0100 Subject: [PATCH] feat(server/v2/cometbft): Implement necessary handlers and helpers for vote extensions (#22830) --- runtime/v2/builder.go | 28 ++- server/v2/cometbft/abci.go | 18 +- server/v2/cometbft/handlers/defaults.go | 10 +- server/v2/cometbft/handlers/handlers.go | 8 +- server/v2/cometbft/options.go | 2 +- server/v2/cometbft/utils.go | 223 +++++++++++++++++++++++- simapp/v2/simdv2/cmd/config.go | 4 + x/auth/tx/gogotx.go | 7 +- 8 files changed, 271 insertions(+), 29 deletions(-) diff --git a/runtime/v2/builder.go b/runtime/v2/builder.go index 631ac550c0c..de9535dca25 100644 --- a/runtime/v2/builder.go +++ b/runtime/v2/builder.go @@ -30,6 +30,7 @@ type AppBuilder[T transaction.Tx] struct { branch func(state store.ReaderMap) store.WriterMap txValidator func(ctx context.Context, tx T) error postTxExec func(ctx context.Context, tx T, success bool) error + preblocker func(ctx context.Context, txs []T, mmPreblocker func() error) error } // RegisterModules registers the provided modules with the module manager. @@ -95,11 +96,22 @@ func (a *AppBuilder[T]) Build(opts ...AppBuilderOption[T]) (*App[T], error) { endBlocker, valUpdate := a.app.moduleManager.EndBlock() + preblockerFn := func(ctx context.Context, txs []T) error { + if a.preblocker != nil { + return a.preblocker(ctx, txs, func() error { + return a.app.moduleManager.PreBlocker()(ctx, txs) + }) + } + + // if there is no preblocker set, call the module manager's preblocker directly + return a.app.moduleManager.PreBlocker()(ctx, txs) + } + stf, err := stf.New[T]( a.app.logger.With("module", "stf"), a.app.msgRouterBuilder, a.app.queryRouterBuilder, - a.app.moduleManager.PreBlocker(), + preblockerFn, a.app.moduleManager.BeginBlock(), endBlocker, a.txValidator, @@ -219,3 +231,17 @@ func AppBuilderWithPostTxExec[T transaction.Tx]( a.postTxExec = postTxExec } } + +// AppBuilderWithPreblocker sets logic that will be executed before each block. +// mmPreblocker can be used to call module manager's preblocker, so that it can be +// called before or after depending on the app's logic. +// This is especially useful when implementing vote extensions. +func AppBuilderWithPreblocker[T transaction.Tx]( + preblocker func( + ctx context.Context, txs []T, mmPreblocker func() error, + ) error, +) AppBuilderOption[T] { + return func(a *AppBuilder[T]) { + a.preblocker = preblocker + } +} diff --git a/server/v2/cometbft/abci.go b/server/v2/cometbft/abci.go index 31e197737fd..3fbf5173e7e 100644 --- a/server/v2/cometbft/abci.go +++ b/server/v2/cometbft/abci.go @@ -68,7 +68,7 @@ type consensus[T transaction.Tx] struct { prepareProposalHandler handlers.PrepareHandler[T] processProposalHandler handlers.ProcessHandler[T] - verifyVoteExt handlers.VerifyVoteExtensionhandler + verifyVoteExt handlers.VerifyVoteExtensionHandler extendVote handlers.ExtendVoteHandler checkTxHandler handlers.CheckTxHandler[T] @@ -142,7 +142,7 @@ func (c *consensus[T]) Info(ctx context.Context, _ *abciproto.InfoRequest) (*abc // if height is 0, we dont know the consensus params var appVersion uint64 = 0 if version > 0 { - cp, err := c.GetConsensusParams(ctx) + cp, err := GetConsensusParams(ctx, c.app) // if the consensus params are not found, we set the app version to 0 // in the case that the start version is > 0 if cp == nil || errors.Is(err, collections.ErrNotFound) { @@ -387,7 +387,7 @@ func (c *consensus[T]) PrepareProposal( LastCommit: toCoreExtendedCommitInfo(req.LocalLastCommit), }) - txs, err := c.prepareProposalHandler(ciCtx, c.app, c.appCodecs.TxCodec, req) + txs, err := c.prepareProposalHandler(ciCtx, c.app, c.appCodecs.TxCodec, req, c.chainID) if err != nil { return nil, err } @@ -433,7 +433,7 @@ func (c *consensus[T]) ProcessProposal( LastCommit: toCoreCommitInfo(req.ProposedLastCommit), }) - err := c.processProposalHandler(ciCtx, c.app, c.appCodecs.TxCodec, req) + err := c.processProposalHandler(ciCtx, c.app, c.appCodecs.TxCodec, req, c.chainID) if err != nil { c.logger.Error("failed to process proposal", "height", req.Height, "time", req.Time, "hash", fmt.Sprintf("%X", req.Hash), "err", err) return &abciproto.ProcessProposalResponse{ @@ -533,7 +533,7 @@ func (c *consensus[T]) FinalizeBlock( c.lastCommittedHeight.Store(req.Height) - cp, err := c.GetConsensusParams(ctx) // we get the consensus params from the latest state because we committed state above + cp, err := GetConsensusParams(ctx, c.app) // we get the consensus params from the latest state because we committed state above if err != nil { return nil, err } @@ -600,7 +600,7 @@ func (c *consensus[T]) Commit(ctx context.Context, _ *abciproto.CommitRequest) ( c.snapshotManager.SnapshotIfApplicable(lastCommittedHeight) - cp, err := c.GetConsensusParams(ctx) + cp, err := GetConsensusParams(ctx, c.app) if err != nil { return nil, err } @@ -619,7 +619,7 @@ func (c *consensus[T]) VerifyVoteExtension( ) (*abciproto.VerifyVoteExtensionResponse, error) { // If vote extensions are not enabled, as a safety precaution, we return an // error. - cp, err := c.GetConsensusParams(ctx) + cp, err := GetConsensusParams(ctx, c.app) if err != nil { return nil, err } @@ -658,7 +658,7 @@ func (c *consensus[T]) VerifyVoteExtension( func (c *consensus[T]) ExtendVote(ctx context.Context, req *abciproto.ExtendVoteRequest) (*abciproto.ExtendVoteResponse, error) { // If vote extensions are not enabled, as a safety precaution, we return an // error. - cp, err := c.GetConsensusParams(ctx) + cp, err := GetConsensusParams(ctx, c.app) if err != nil { return nil, err } @@ -702,6 +702,8 @@ func decodeTxs[T transaction.Tx](logger log.Logger, rawTxs [][]byte, codec trans if err != nil { // do not return an error here, as we want to deliver the block even if some txs are invalid logger.Debug("failed to decode tx", "err", err) + txs[i] = RawTx(rawTx).(T) // allows getting the raw bytes down the line + continue } txs[i] = tx } diff --git a/server/v2/cometbft/handlers/defaults.go b/server/v2/cometbft/handlers/defaults.go index d8f43bb2fd2..08403abd28a 100644 --- a/server/v2/cometbft/handlers/defaults.go +++ b/server/v2/cometbft/handlers/defaults.go @@ -32,7 +32,7 @@ func NewDefaultProposalHandler[T transaction.Tx](mp mempool.Mempool[T]) *Default } func (h *DefaultProposalHandler[T]) PrepareHandler() PrepareHandler[T] { - return func(ctx context.Context, app AppManager[T], codec transaction.Codec[T], req *abci.PrepareProposalRequest) ([]T, error) { + return func(ctx context.Context, app AppManager[T], codec transaction.Codec[T], req *abci.PrepareProposalRequest, chainID string) ([]T, error) { var maxBlockGas uint64 res, err := app.Query(ctx, 0, &consensustypes.QueryParamsRequest{}) @@ -98,7 +98,7 @@ func (h *DefaultProposalHandler[T]) PrepareHandler() PrepareHandler[T] { } func (h *DefaultProposalHandler[T]) ProcessHandler() ProcessHandler[T] { - return func(ctx context.Context, app AppManager[T], codec transaction.Codec[T], req *abci.ProcessProposalRequest) error { + return func(ctx context.Context, app AppManager[T], codec transaction.Codec[T], req *abci.ProcessProposalRequest, chainID string) error { // If the mempool is nil we simply return ACCEPT, // because PrepareProposal may have included txs that could fail verification. _, isNoOp := h.mempool.(mempool.NoOpMempool[T]) @@ -174,7 +174,7 @@ func decodeTxs[T transaction.Tx](codec transaction.Codec[T], txsBz [][]byte) []T // NoOpPrepareProposal defines a no-op PrepareProposal handler. It will always // return the transactions sent by the client's request. func NoOpPrepareProposal[T transaction.Tx]() PrepareHandler[T] { - return func(ctx context.Context, app AppManager[T], codec transaction.Codec[T], req *abci.PrepareProposalRequest) ([]T, error) { + return func(ctx context.Context, app AppManager[T], codec transaction.Codec[T], req *abci.PrepareProposalRequest, chainID string) ([]T, error) { return decodeTxs(codec, req.Txs), nil } } @@ -182,7 +182,7 @@ func NoOpPrepareProposal[T transaction.Tx]() PrepareHandler[T] { // NoOpProcessProposal defines a no-op ProcessProposal Handler. It will always // return ACCEPT. func NoOpProcessProposal[T transaction.Tx]() ProcessHandler[T] { - return func(context.Context, AppManager[T], transaction.Codec[T], *abci.ProcessProposalRequest) error { + return func(context.Context, AppManager[T], transaction.Codec[T], *abci.ProcessProposalRequest, string) error { return nil } } @@ -197,7 +197,7 @@ func NoOpExtendVote() ExtendVoteHandler { // NoOpVerifyVoteExtensionHandler defines a no-op VerifyVoteExtension handler. It // will always return an ACCEPT status with no error. -func NoOpVerifyVoteExtensionHandler() VerifyVoteExtensionhandler { +func NoOpVerifyVoteExtensionHandler() VerifyVoteExtensionHandler { return func(context.Context, store.ReaderMap, *abci.VerifyVoteExtensionRequest) (*abci.VerifyVoteExtensionResponse, error) { return &abci.VerifyVoteExtensionResponse{Status: abci.VERIFY_VOTE_EXTENSION_STATUS_ACCEPT}, nil } diff --git a/server/v2/cometbft/handlers/handlers.go b/server/v2/cometbft/handlers/handlers.go index 015594f469f..b6cc213dc30 100644 --- a/server/v2/cometbft/handlers/handlers.go +++ b/server/v2/cometbft/handlers/handlers.go @@ -13,16 +13,16 @@ import ( type ( // PrepareHandler passes in the list of Txs that are being proposed. The app can then do stateful operations // over the list of proposed transactions. It can return a modified list of txs to include in the proposal. - PrepareHandler[T transaction.Tx] func(context.Context, AppManager[T], transaction.Codec[T], *abci.PrepareProposalRequest) ([]T, error) + PrepareHandler[T transaction.Tx] func(ctx context.Context, app AppManager[T], cdc transaction.Codec[T], req *abci.PrepareProposalRequest, chainID string) ([]T, error) // ProcessHandler is a function that takes a list of transactions and returns a boolean and an error. // If the verification of a transaction fails, the boolean is false and the error is non-nil. - ProcessHandler[T transaction.Tx] func(context.Context, AppManager[T], transaction.Codec[T], *abci.ProcessProposalRequest) error + ProcessHandler[T transaction.Tx] func(ctx context.Context, app AppManager[T], cdc transaction.Codec[T], req *abci.ProcessProposalRequest, chainID string) error - // VerifyVoteExtensionhandler is a function type that handles the verification of a vote extension request. + // VerifyVoteExtensionHandler is a function type that handles the verification of a vote extension request. // It takes a context, a store reader map, and a request to verify a vote extension. // It returns a response to verify the vote extension and an error if any. - VerifyVoteExtensionhandler func(context.Context, store.ReaderMap, *abci.VerifyVoteExtensionRequest) (*abci.VerifyVoteExtensionResponse, error) + VerifyVoteExtensionHandler func(context.Context, store.ReaderMap, *abci.VerifyVoteExtensionRequest) (*abci.VerifyVoteExtensionResponse, error) // ExtendVoteHandler is a function type that handles the extension of a vote. // It takes a context, a store reader map, and a request to extend a vote. diff --git a/server/v2/cometbft/options.go b/server/v2/cometbft/options.go index b5936148b5a..5016495461e 100644 --- a/server/v2/cometbft/options.go +++ b/server/v2/cometbft/options.go @@ -20,7 +20,7 @@ type ServerOptions[T transaction.Tx] struct { PrepareProposalHandler handlers.PrepareHandler[T] ProcessProposalHandler handlers.ProcessHandler[T] CheckTxHandler handlers.CheckTxHandler[T] - VerifyVoteExtensionHandler handlers.VerifyVoteExtensionhandler + VerifyVoteExtensionHandler handlers.VerifyVoteExtensionHandler ExtendVoteHandler handlers.ExtendVoteHandler KeygenF keyGenF diff --git a/server/v2/cometbft/utils.go b/server/v2/cometbft/utils.go index 81d47b7f8cb..700a84b247a 100644 --- a/server/v2/cometbft/utils.go +++ b/server/v2/cometbft/utils.go @@ -1,15 +1,21 @@ package cometbft import ( + "bytes" "context" + "crypto/sha256" "errors" "fmt" "math" + "slices" "strings" "time" abci "github.com/cometbft/cometbft/api/cometbft/abci/v1" cmtproto "github.com/cometbft/cometbft/api/cometbft/types/v1" + cryptoenc "github.com/cometbft/cometbft/crypto/encoding" + protoio "github.com/cosmos/gogoproto/io" + "github.com/cosmos/gogoproto/proto" gogoproto "github.com/cosmos/gogoproto/proto" gogoany "github.com/cosmos/gogoproto/types/any" "google.golang.org/grpc/codes" @@ -21,8 +27,11 @@ import ( "cosmossdk.io/core/server" "cosmossdk.io/core/transaction" errorsmod "cosmossdk.io/errors" // we aren't using errors/v2 as it doesn't support grpc status codes + "cosmossdk.io/server/v2/cometbft/handlers" "cosmossdk.io/x/consensus/types" + cryptocodec "github.com/cosmos/cosmos-sdk/crypto/codec" + cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" ) @@ -344,13 +353,8 @@ func (c *consensus[T]) validateFinalizeBlockHeight(req *abci.FinalizeBlockReques // GetConsensusParams makes a query to the consensus module in order to get the latest consensus // parameters from committed state -func (c *consensus[T]) GetConsensusParams(ctx context.Context) (*cmtproto.ConsensusParams, error) { - latestVersion, err := c.store.GetLatestVersion() - if err != nil { - return nil, err - } - - res, err := c.app.Query(ctx, latestVersion, &types.QueryParamsRequest{}) +func GetConsensusParams[T transaction.Tx](ctx context.Context, app handlers.AppManager[T]) (*cmtproto.ConsensusParams, error) { + res, err := app.Query(ctx, 0, &types.QueryParamsRequest{}) if err != nil { return nil, err } @@ -442,3 +446,208 @@ func uint64ToInt64(u uint64) int64 { } return int64(u) } + +// RawTx allows access to the raw bytes of a transaction even if it failed +// to decode. +func RawTx(tx []byte) transaction.Tx { + return InjectedTx(tx) +} + +type InjectedTx []byte + +var _ transaction.Tx = InjectedTx{} + +func (tx InjectedTx) Bytes() []byte { + return tx +} + +func (tx InjectedTx) Hash() [32]byte { + return sha256.Sum256(tx) +} + +func (tx InjectedTx) GetGasLimit() (uint64, error) { + return 0, nil +} + +func (tx InjectedTx) GetMessages() ([]transaction.Msg, error) { + return nil, nil +} + +func (tx InjectedTx) GetSenders() ([]transaction.Identity, error) { + return [][]byte{[]byte("cometbft")}, nil +} + +// 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. +// If commitInfo is nil, this function can be used to check a set of vote extensions +// without comparing them to a commit. +func ValidateVoteExtensions[T transaction.Tx]( + ctx context.Context, + app handlers.AppManager[T], + chainID string, + validatorStore func(context.Context, []byte) (cryptotypes.PubKey, error), + extCommit abci.ExtendedCommitInfo, + currentHeight int64, + commitInfo *abci.CommitInfo, +) error { + cp, err := GetConsensusParams(ctx, app) + if err != nil { + return err + } + + if commitInfo != nil { + // Check that both extCommit + commit are ordered in accordance with vp/address. + if err := validateExtendedCommitAgainstLastCommit(extCommit, *commitInfo); err != nil { + return err + } + } + + // Start checking vote extensions only **after** the vote extensions enable + // height, because when `currentHeight == VoteExtensionsEnableHeight` + // PrepareProposal doesn't get any vote extensions in its request. + extsEnabled := cp.Feature != nil && cp.Feature.VoteExtensionsEnableHeight != nil && currentHeight > cp.Feature.VoteExtensionsEnableHeight.Value && cp.Feature.VoteExtensionsEnableHeight.Value != 0 + if !extsEnabled { + extsEnabled = cp.Abci != nil && currentHeight > cp.Abci.VoteExtensionsEnableHeight && cp.Abci.VoteExtensionsEnableHeight != 0 + } + if !extsEnabled { + return nil + } + + 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 the 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 := validatorStore(ctx, valConsAddr) + if err != nil { + return fmt.Errorf("failed to get validator %X public key: %w", valConsAddr, err) + } + + cmtpk, err := cryptocodec.ToCmtProtoPublicKey(pubKeyProto) + if err != nil { + return fmt.Errorf("failed to convert validator %X public key: %w", valConsAddr, err) + } + + cmtPubKey, err := cryptoenc.PubKeyFromProto(cmtpk) + 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 + } + + // This check is probably unnecessary, but better safe than sorry. + if totalVP <= 0 { + return fmt.Errorf("total voting power must be positive, got: %d", totalVP) + } + + // If the sum of the voting power has not reached (2/3 + 1) we need to error. + if requiredVP := ((totalVP * 2) / 3) + 1; sumVP < requiredVP { + return fmt.Errorf( + "insufficient cumulative voting power received to verify vote extensions; got: %d, expected: >=%d", + sumVP, requiredVP, + ) + } + return nil +} + +// validateExtendedCommitAgainstLastCommit validates an ExtendedCommitInfo against a LastCommit. Specifically, +// it checks that the ExtendedCommit + LastCommit (for the same height), are consistent with each other + that +// they are ordered correctly (by voting power) in accordance with +// [comet](https://github.com/cometbft/cometbft/blob/4ce0277b35f31985bbf2c25d3806a184a4510010/types/validator_set.go#L784). +func validateExtendedCommitAgainstLastCommit(ec abci.ExtendedCommitInfo, lc abci.CommitInfo) error { + // check that the rounds are the same + if ec.Round != lc.Round { + return fmt.Errorf("extended commit round %d does not match last commit round %d", ec.Round, lc.Round) + } + + // check that the # of votes are the same + if len(ec.Votes) != len(lc.Votes) { + return fmt.Errorf("extended commit votes length %d does not match last commit votes length %d", len(ec.Votes), len(lc.Votes)) + } + + // check sort order of extended commit votes + if !slices.IsSortedFunc(ec.Votes, func(vote1, vote2 abci.ExtendedVoteInfo) int { + if vote1.Validator.Power == vote2.Validator.Power { + return bytes.Compare(vote1.Validator.Address, vote2.Validator.Address) // addresses sorted in ascending order (used to break vp conflicts) + } + return -int(vote1.Validator.Power - vote2.Validator.Power) // vp sorted in descending order + }) { + return errors.New("extended commit votes are not sorted by voting power") + } + + addressCache := make(map[string]struct{}, len(ec.Votes)) + // check consistency between LastCommit and ExtendedCommit + for i, vote := range ec.Votes { + // cache addresses to check for duplicates + if _, ok := addressCache[string(vote.Validator.Address)]; ok { + return fmt.Errorf("extended commit vote address %X is duplicated", vote.Validator.Address) + } + addressCache[string(vote.Validator.Address)] = struct{}{} + + if !bytes.Equal(vote.Validator.Address, lc.Votes[i].Validator.Address) { + return fmt.Errorf("extended commit vote address %X does not match last commit vote address %X", vote.Validator.Address, lc.Votes[i].Validator.Address) + } + if vote.Validator.Power != lc.Votes[i].Validator.Power { + return fmt.Errorf("extended commit vote power %d does not match last commit vote power %d", vote.Validator.Power, lc.Votes[i].Validator.Power) + } + } + + return nil +} diff --git a/simapp/v2/simdv2/cmd/config.go b/simapp/v2/simdv2/cmd/config.go index 96325705b38..ff3f7058a98 100644 --- a/simapp/v2/simdv2/cmd/config.go +++ b/simapp/v2/simdv2/cmd/config.go @@ -92,6 +92,10 @@ func initCometConfig() cometbft.CfgOption { func initCometOptions[T transaction.Tx]() cometbft.ServerOptions[T] { serverOptions := cometbft.DefaultServerOptions[T]() + // Implement custom handlers (e.g. for Vote Extensions) + // serverOptions.PrepareProposalHandler = CustomPrepareProposal[T]() + // serverOptions.ProcessProposalHandler = CustomProcessProposalHandler[T]() + // serverOptions.ExtendVoteHandler = CustomExtendVoteHandler[T]() // overwrite app mempool, using max-txs option // serverOptions.Mempool = func(cfg map[string]any) mempool.Mempool[T] { diff --git a/x/auth/tx/gogotx.go b/x/auth/tx/gogotx.go index aa424f1c991..196b6a6827c 100644 --- a/x/auth/tx/gogotx.go +++ b/x/auth/tx/gogotx.go @@ -32,7 +32,7 @@ func newWrapperFromDecodedTx( addrCodec address.Codec, cdc codec.BinaryCodec, decodedTx *decode.DecodedTx, ) (*gogoTxWrapper, error) { var ( - fees = make(sdk.Coins, len(decodedTx.Tx.AuthInfo.Fee.Amount)) + fees = sdk.Coins{} // decodedTx.Tx.AuthInfo.Fee.Amount might be nil err error ) for i, fee := range decodedTx.Tx.AuthInfo.Fee.Amount { @@ -43,10 +43,11 @@ func newWrapperFromDecodedTx( if err = sdk.ValidateDenom(fee.Denom); err != nil { return nil, fmt.Errorf("invalid fee coin denom at index %d: %w", i, err) } - fees[i] = sdk.Coin{ + + fees = fees.Add(sdk.Coin{ Denom: fee.Denom, Amount: amtInt, - } + }) } if !fees.IsSorted() { return nil, fmt.Errorf("invalid not sorted tx fees: %s", fees.String())