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 1/3] 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 631ac550c0c6..de9535dca257 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 31e197737fd8..3fbf5173e7e3 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 d8f43bb2fd25..08403abd28ae 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 015594f469f7..b6cc213dc30f 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 b5936148b5a5..5016495461e3 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 81d47b7f8cb8..700a84b247ac 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 96325705b380..ff3f7058a98e 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 aa424f1c991f..196b6a6827c7 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()) From 448b33d78d26beca1797acaa6028c14787756a79 Mon Sep 17 00:00:00 2001 From: Emmanuel Date: Thu, 19 Dec 2024 11:17:19 -0300 Subject: [PATCH 2/3] build(deps): upgrade ledger-cosmos-go package (#23007) Co-authored-by: Juan Leni --- client/v2/go.mod | 2 +- client/v2/go.sum | 4 ++-- go.mod | 2 +- go.sum | 4 ++-- server/v2/cometbft/go.mod | 2 +- server/v2/cometbft/go.sum | 4 ++-- simapp/go.mod | 2 +- simapp/go.sum | 4 ++-- simapp/v2/go.mod | 2 +- simapp/v2/go.sum | 4 ++-- tests/go.mod | 2 +- tests/go.sum | 4 ++-- tools/benchmark/go.mod | 2 +- tools/benchmark/go.sum | 4 ++-- x/accounts/defaults/base/go.mod | 2 +- x/accounts/defaults/base/go.sum | 4 ++-- x/accounts/defaults/lockup/go.mod | 2 +- x/accounts/defaults/lockup/go.sum | 4 ++-- x/accounts/defaults/multisig/go.mod | 2 +- x/accounts/defaults/multisig/go.sum | 4 ++-- x/accounts/go.mod | 2 +- x/accounts/go.sum | 4 ++-- x/authz/go.mod | 2 +- x/authz/go.sum | 4 ++-- x/bank/go.mod | 2 +- x/bank/go.sum | 4 ++-- x/circuit/go.mod | 2 +- x/circuit/go.sum | 4 ++-- x/consensus/go.mod | 2 +- x/consensus/go.sum | 4 ++-- x/distribution/go.mod | 2 +- x/distribution/go.sum | 4 ++-- x/epochs/go.mod | 2 +- x/epochs/go.sum | 4 ++-- x/evidence/go.mod | 2 +- x/evidence/go.sum | 4 ++-- x/feegrant/go.mod | 2 +- x/feegrant/go.sum | 4 ++-- x/gov/go.mod | 2 +- x/gov/go.sum | 4 ++-- x/group/go.mod | 2 +- x/group/go.sum | 4 ++-- x/mint/go.mod | 2 +- x/mint/go.sum | 4 ++-- x/nft/go.mod | 2 +- x/nft/go.sum | 4 ++-- x/params/go.mod | 2 +- x/params/go.sum | 4 ++-- x/protocolpool/go.mod | 2 +- x/protocolpool/go.sum | 4 ++-- x/slashing/go.mod | 2 +- x/slashing/go.sum | 4 ++-- x/staking/go.mod | 2 +- x/staking/go.sum | 4 ++-- x/upgrade/go.mod | 2 +- x/upgrade/go.sum | 4 ++-- 56 files changed, 84 insertions(+), 84 deletions(-) diff --git a/client/v2/go.mod b/client/v2/go.mod index 1fcab4c3f65c..88105c4e9d9c 100644 --- a/client/v2/go.mod +++ b/client/v2/go.mod @@ -67,7 +67,7 @@ require ( github.com/cosmos/gogoproto v1.7.0 github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/client/v2/go.sum b/client/v2/go.sum index c0b46caf1c52..6ee6e63398a3 100644 --- a/client/v2/go.sum +++ b/client/v2/go.sum @@ -138,8 +138,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/go.mod b/go.mod index 7e1aa23b4318..22955d0cf96a 100644 --- a/go.mod +++ b/go.mod @@ -26,7 +26,7 @@ require ( github.com/cosmos/go-bip39 v1.0.0 github.com/cosmos/gogogateway v1.2.0 github.com/cosmos/gogoproto v1.7.0 - github.com/cosmos/ledger-cosmos-go v0.13.3 + github.com/cosmos/ledger-cosmos-go v0.14.0 github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 github.com/golang/protobuf v1.5.4 github.com/google/go-cmp v0.6.0 diff --git a/go.sum b/go.sum index 310ff9ca9ff2..1b54d4a649f2 100644 --- a/go.sum +++ b/go.sum @@ -131,8 +131,8 @@ github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5Rtn github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= github.com/cosmos/keyring v1.2.0 h1:8C1lBP9xhImmIabyXW4c3vFjjLiBdGCmfLUfeZlV1Yo= github.com/cosmos/keyring v1.2.0/go.mod h1:fc+wB5KTk9wQ9sDx0kFXB3A0MaeGHM9AwRStKOQ5vOA= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/server/v2/cometbft/go.mod b/server/v2/cometbft/go.mod index a05511801b14..2cb6b2b15652 100644 --- a/server/v2/cometbft/go.mod +++ b/server/v2/cometbft/go.mod @@ -74,7 +74,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/server/v2/cometbft/go.sum b/server/v2/cometbft/go.sum index 067780b13990..514a30302f34 100644 --- a/server/v2/cometbft/go.sum +++ b/server/v2/cometbft/go.sum @@ -134,8 +134,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/simapp/go.mod b/simapp/go.mod index 85d739adcce8..17228aa22717 100644 --- a/simapp/go.mod +++ b/simapp/go.mod @@ -99,7 +99,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/creachadair/atomicfile v0.3.6 // indirect github.com/creachadair/tomledit v0.0.26 // indirect github.com/danieljoos/wincred v1.2.1 // indirect diff --git a/simapp/go.sum b/simapp/go.sum index 1c43632d1956..c4ed62ea88a5 100644 --- a/simapp/go.sum +++ b/simapp/go.sum @@ -346,8 +346,8 @@ github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5Rtn github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= github.com/cosmos/keyring v1.2.0 h1:8C1lBP9xhImmIabyXW4c3vFjjLiBdGCmfLUfeZlV1Yo= github.com/cosmos/keyring v1.2.0/go.mod h1:fc+wB5KTk9wQ9sDx0kFXB3A0MaeGHM9AwRStKOQ5vOA= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creachadair/atomicfile v0.3.6 h1:BcXSDRq7waImZhKflqnTJjf+02CLi0W2Exlb2lyZ2yE= github.com/creachadair/atomicfile v0.3.6/go.mod h1:iaBMVDkRBQTIGzbYGCTS+gXeZPidWAeVbthIxSbEphE= diff --git a/simapp/v2/go.mod b/simapp/v2/go.mod index bec27ffdbdc4..5acf7dfd0dfc 100644 --- a/simapp/v2/go.mod +++ b/simapp/v2/go.mod @@ -104,7 +104,7 @@ require ( github.com/cosmos/gogoproto v1.7.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/creachadair/atomicfile v0.3.6 // indirect github.com/creachadair/tomledit v0.0.26 // indirect github.com/danieljoos/wincred v1.2.1 // indirect diff --git a/simapp/v2/go.sum b/simapp/v2/go.sum index 52c6b7614104..39f9620f0efd 100644 --- a/simapp/v2/go.sum +++ b/simapp/v2/go.sum @@ -346,8 +346,8 @@ github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5Rtn github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= github.com/cosmos/keyring v1.2.0 h1:8C1lBP9xhImmIabyXW4c3vFjjLiBdGCmfLUfeZlV1Yo= github.com/cosmos/keyring v1.2.0/go.mod h1:fc+wB5KTk9wQ9sDx0kFXB3A0MaeGHM9AwRStKOQ5vOA= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creachadair/atomicfile v0.3.6 h1:BcXSDRq7waImZhKflqnTJjf+02CLi0W2Exlb2lyZ2yE= github.com/creachadair/atomicfile v0.3.6/go.mod h1:iaBMVDkRBQTIGzbYGCTS+gXeZPidWAeVbthIxSbEphE= diff --git a/tests/go.mod b/tests/go.mod index 099f659af30d..4d521a2bce26 100644 --- a/tests/go.mod +++ b/tests/go.mod @@ -108,7 +108,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/tests/go.sum b/tests/go.sum index 17145819d031..30da8c46413b 100644 --- a/tests/go.sum +++ b/tests/go.sum @@ -344,8 +344,8 @@ github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5Rtn github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= github.com/cosmos/keyring v1.2.0 h1:8C1lBP9xhImmIabyXW4c3vFjjLiBdGCmfLUfeZlV1Yo= github.com/cosmos/keyring v1.2.0/go.mod h1:fc+wB5KTk9wQ9sDx0kFXB3A0MaeGHM9AwRStKOQ5vOA= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/tools/benchmark/go.mod b/tools/benchmark/go.mod index 06e9db4d4f30..99c755c08e82 100644 --- a/tools/benchmark/go.mod +++ b/tools/benchmark/go.mod @@ -53,7 +53,7 @@ require ( github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/tools/benchmark/go.sum b/tools/benchmark/go.sum index 6b711dc0dd89..2af0c7024b34 100644 --- a/tools/benchmark/go.sum +++ b/tools/benchmark/go.sum @@ -112,8 +112,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/accounts/defaults/base/go.mod b/x/accounts/defaults/base/go.mod index 463a07366f1a..35c9d6decbed 100644 --- a/x/accounts/defaults/base/go.mod +++ b/x/accounts/defaults/base/go.mod @@ -56,7 +56,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/dgraph-io/badger/v4 v4.5.0 // indirect diff --git a/x/accounts/defaults/base/go.sum b/x/accounts/defaults/base/go.sum index 402e02f25912..2f0011d6a5cf 100644 --- a/x/accounts/defaults/base/go.sum +++ b/x/accounts/defaults/base/go.sum @@ -132,8 +132,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/accounts/defaults/lockup/go.mod b/x/accounts/defaults/lockup/go.mod index 06267f2e44d8..78e013084a65 100644 --- a/x/accounts/defaults/lockup/go.mod +++ b/x/accounts/defaults/lockup/go.mod @@ -57,7 +57,7 @@ require ( github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/accounts/defaults/lockup/go.sum b/x/accounts/defaults/lockup/go.sum index 6b711dc0dd89..2af0c7024b34 100644 --- a/x/accounts/defaults/lockup/go.sum +++ b/x/accounts/defaults/lockup/go.sum @@ -112,8 +112,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/accounts/defaults/multisig/go.mod b/x/accounts/defaults/multisig/go.mod index 43bcd4f7c846..c3130e05266f 100644 --- a/x/accounts/defaults/multisig/go.mod +++ b/x/accounts/defaults/multisig/go.mod @@ -55,7 +55,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/accounts/defaults/multisig/go.sum b/x/accounts/defaults/multisig/go.sum index 402e02f25912..2f0011d6a5cf 100644 --- a/x/accounts/defaults/multisig/go.sum +++ b/x/accounts/defaults/multisig/go.sum @@ -132,8 +132,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/accounts/go.mod b/x/accounts/go.mod index e1a197c35b7b..9844fbca3702 100644 --- a/x/accounts/go.mod +++ b/x/accounts/go.mod @@ -65,7 +65,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/dgraph-io/badger/v4 v4.5.0 // indirect diff --git a/x/accounts/go.sum b/x/accounts/go.sum index 402e02f25912..2f0011d6a5cf 100644 --- a/x/accounts/go.sum +++ b/x/accounts/go.sum @@ -132,8 +132,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/authz/go.mod b/x/authz/go.mod index 10ca5906e1f5..b59ec5d1554c 100644 --- a/x/authz/go.mod +++ b/x/authz/go.mod @@ -53,7 +53,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/authz/go.sum b/x/authz/go.sum index 402e02f25912..2f0011d6a5cf 100644 --- a/x/authz/go.sum +++ b/x/authz/go.sum @@ -132,8 +132,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/bank/go.mod b/x/bank/go.mod index c9acc6c2782e..68f3a756b98d 100644 --- a/x/bank/go.mod +++ b/x/bank/go.mod @@ -53,7 +53,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/bank/go.sum b/x/bank/go.sum index 402e02f25912..2f0011d6a5cf 100644 --- a/x/bank/go.sum +++ b/x/bank/go.sum @@ -132,8 +132,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/circuit/go.mod b/x/circuit/go.mod index 1bb24ff54de1..57812f55cfdb 100644 --- a/x/circuit/go.mod +++ b/x/circuit/go.mod @@ -57,7 +57,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/circuit/go.sum b/x/circuit/go.sum index 402e02f25912..2f0011d6a5cf 100644 --- a/x/circuit/go.sum +++ b/x/circuit/go.sum @@ -132,8 +132,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/consensus/go.mod b/x/consensus/go.mod index 459e097bc697..37a11e8833b8 100644 --- a/x/consensus/go.mod +++ b/x/consensus/go.mod @@ -57,7 +57,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/consensus/go.sum b/x/consensus/go.sum index 402e02f25912..2f0011d6a5cf 100644 --- a/x/consensus/go.sum +++ b/x/consensus/go.sum @@ -132,8 +132,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/distribution/go.mod b/x/distribution/go.mod index ca1ae51303d7..befaa65c7a68 100644 --- a/x/distribution/go.mod +++ b/x/distribution/go.mod @@ -60,7 +60,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/distribution/go.sum b/x/distribution/go.sum index 402e02f25912..2f0011d6a5cf 100644 --- a/x/distribution/go.sum +++ b/x/distribution/go.sum @@ -132,8 +132,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/epochs/go.mod b/x/epochs/go.mod index ea3da3d6634f..71debbefc663 100644 --- a/x/epochs/go.mod +++ b/x/epochs/go.mod @@ -53,7 +53,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/epochs/go.sum b/x/epochs/go.sum index 402e02f25912..2f0011d6a5cf 100644 --- a/x/epochs/go.sum +++ b/x/epochs/go.sum @@ -132,8 +132,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/evidence/go.mod b/x/evidence/go.mod index 26853a17d751..ca4b24a188fc 100644 --- a/x/evidence/go.mod +++ b/x/evidence/go.mod @@ -60,7 +60,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/evidence/go.sum b/x/evidence/go.sum index 402e02f25912..2f0011d6a5cf 100644 --- a/x/evidence/go.sum +++ b/x/evidence/go.sum @@ -132,8 +132,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/feegrant/go.mod b/x/feegrant/go.mod index db13105c37fb..2521cf7ccfc4 100644 --- a/x/feegrant/go.mod +++ b/x/feegrant/go.mod @@ -69,7 +69,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/feegrant/go.sum b/x/feegrant/go.sum index c0b46caf1c52..6ee6e63398a3 100644 --- a/x/feegrant/go.sum +++ b/x/feegrant/go.sum @@ -138,8 +138,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/gov/go.mod b/x/gov/go.mod index 689eb5dc8996..3611933fd117 100644 --- a/x/gov/go.mod +++ b/x/gov/go.mod @@ -61,7 +61,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/gov/go.sum b/x/gov/go.sum index c0b46caf1c52..6ee6e63398a3 100644 --- a/x/gov/go.sum +++ b/x/gov/go.sum @@ -138,8 +138,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/group/go.mod b/x/group/go.mod index 57c5e409cc9f..c05544b070b6 100644 --- a/x/group/go.mod +++ b/x/group/go.mod @@ -65,7 +65,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/group/go.sum b/x/group/go.sum index dbc5757ba67e..5a112ab33586 100644 --- a/x/group/go.sum +++ b/x/group/go.sum @@ -142,8 +142,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/mint/go.mod b/x/mint/go.mod index ecc59e6c893b..6c5a943407b8 100644 --- a/x/mint/go.mod +++ b/x/mint/go.mod @@ -53,7 +53,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/mint/go.sum b/x/mint/go.sum index 402e02f25912..2f0011d6a5cf 100644 --- a/x/mint/go.sum +++ b/x/mint/go.sum @@ -132,8 +132,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/nft/go.mod b/x/nft/go.mod index 00d55d068253..b643b19233a3 100644 --- a/x/nft/go.mod +++ b/x/nft/go.mod @@ -58,7 +58,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/nft/go.sum b/x/nft/go.sum index 402e02f25912..2f0011d6a5cf 100644 --- a/x/nft/go.sum +++ b/x/nft/go.sum @@ -132,8 +132,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/params/go.mod b/x/params/go.mod index b94b82095b72..7fc3a6507301 100644 --- a/x/params/go.mod +++ b/x/params/go.mod @@ -58,7 +58,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/params/go.sum b/x/params/go.sum index 27f23cb9999d..87fc1bdf453a 100644 --- a/x/params/go.sum +++ b/x/params/go.sum @@ -122,8 +122,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/protocolpool/go.mod b/x/protocolpool/go.mod index 38970dfe4a8a..3e8ca5b9b999 100644 --- a/x/protocolpool/go.mod +++ b/x/protocolpool/go.mod @@ -59,7 +59,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/protocolpool/go.sum b/x/protocolpool/go.sum index 402e02f25912..2f0011d6a5cf 100644 --- a/x/protocolpool/go.sum +++ b/x/protocolpool/go.sum @@ -132,8 +132,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/slashing/go.mod b/x/slashing/go.mod index e09244e7f155..637bb40b780f 100644 --- a/x/slashing/go.mod +++ b/x/slashing/go.mod @@ -61,7 +61,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/slashing/go.sum b/x/slashing/go.sum index 895af95e6f39..fffc9853cc47 100644 --- a/x/slashing/go.sum +++ b/x/slashing/go.sum @@ -134,8 +134,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/staking/go.mod b/x/staking/go.mod index 409c99359b52..e7fd868ee21e 100644 --- a/x/staking/go.mod +++ b/x/staking/go.mod @@ -53,7 +53,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/staking/go.sum b/x/staking/go.sum index 402e02f25912..2f0011d6a5cf 100644 --- a/x/staking/go.sum +++ b/x/staking/go.sum @@ -132,8 +132,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= diff --git a/x/upgrade/go.mod b/x/upgrade/go.mod index 0684c108537b..b7f0ef53e75e 100644 --- a/x/upgrade/go.mod +++ b/x/upgrade/go.mod @@ -76,7 +76,7 @@ require ( github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect github.com/cosmos/ics23/go v0.11.0 // indirect - github.com/cosmos/ledger-cosmos-go v0.13.3 // indirect + github.com/cosmos/ledger-cosmos-go v0.14.0 // indirect github.com/danieljoos/wincred v1.2.1 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect diff --git a/x/upgrade/go.sum b/x/upgrade/go.sum index c8644a290562..acd3e38dfe99 100644 --- a/x/upgrade/go.sum +++ b/x/upgrade/go.sum @@ -340,8 +340,8 @@ github.com/cosmos/iavl v1.3.4 h1:A0RUAms7TZ0L6EFrrBIPg4Dy7qD9vvD5lJKUxEXURLM= github.com/cosmos/iavl v1.3.4/go.mod h1:T6SfBcyhulVIY2G/ZtAtQm/QiJvsuhIos52V4dWYk88= github.com/cosmos/ics23/go v0.11.0 h1:jk5skjT0TqX5e5QJbEnwXIS2yI2vnmLOgpQPeM5RtnU= github.com/cosmos/ics23/go v0.11.0/go.mod h1:A8OjxPE67hHST4Icw94hOxxFEJMBG031xIGF/JHNIY0= -github.com/cosmos/ledger-cosmos-go v0.13.3 h1:7ehuBGuyIytsXbd4MP43mLeoN2LTOEnk5nvue4rK+yM= -github.com/cosmos/ledger-cosmos-go v0.13.3/go.mod h1:HENcEP+VtahZFw38HZ3+LS3Iv5XV6svsnkk9vdJtLr8= +github.com/cosmos/ledger-cosmos-go v0.14.0 h1:WfCHricT3rPbkPSVKRH+L4fQGKYHuGOK9Edpel8TYpE= +github.com/cosmos/ledger-cosmos-go v0.14.0/go.mod h1:E07xCWSBl3mTGofZ2QnL4cIUzMbbGVyik84QYKbX3RA= github.com/cpuguy83/go-md2man/v2 v2.0.4/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/danieljoos/wincred v1.2.1 h1:dl9cBrupW8+r5250DYkYxocLeZ1Y4vB1kxgtjxw8GQs= From a9c1149155e498a41f4b68a5694027a3f3f9f059 Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Thu, 19 Dec 2024 16:00:37 +0100 Subject: [PATCH 3/3] test: investigate test failure due to wal log (#22679) --- client/pruning/main.go | 4 ++-- client/snapshot/export.go | 8 +++----- client/snapshot/list.go | 5 +++++ client/snapshot/restore.go | 7 +++---- client/v2/go.mod | 2 +- client/v2/go.sum | 4 ++-- go.mod | 2 +- go.sum | 4 ++-- server/cmt_cmds.go | 4 +--- server/v2/api/telemetry/config.go | 2 +- server/v2/cometbft/go.sum | 4 ++-- server/v2/stf/stf_test.go | 8 +++++--- server/v2/store/commands.go | 3 +-- server/v2/store/snapshot.go | 10 +++++++--- server/v2/testdata/app.toml | 2 -- simapp/go.mod | 2 +- simapp/go.sum | 4 ++-- simapp/v2/go.mod | 2 +- simapp/v2/go.sum | 4 ++-- store/v2/db/pebbledb.go | 11 +++++++++++ systemtests/CHANGELOG.md | 4 ++++ systemtests/system.go | 12 +++++++----- systemtests/testnet_init.go | 1 + tests/go.mod | 2 +- tests/go.sum | 4 ++-- tests/systemtests/export_test.go | 22 ++++++++-------------- tests/systemtests/go.mod | 2 +- tests/systemtests/go.sum | 4 ++-- tests/systemtests/snapshots_test.go | 23 +++++++++++------------ tools/benchmark/go.sum | 4 ++-- tools/confix/data/v2-app.toml | 2 +- x/accounts/defaults/base/go.mod | 2 +- x/accounts/defaults/base/go.sum | 4 ++-- x/accounts/defaults/lockup/go.sum | 4 ++-- x/accounts/defaults/multisig/go.mod | 2 +- x/accounts/defaults/multisig/go.sum | 4 ++-- x/accounts/go.mod | 2 +- x/accounts/go.sum | 4 ++-- x/authz/go.mod | 2 +- x/authz/go.sum | 4 ++-- x/bank/go.mod | 2 +- x/bank/go.sum | 4 ++-- x/circuit/go.mod | 2 +- x/circuit/go.sum | 4 ++-- x/consensus/go.mod | 2 +- x/consensus/go.sum | 4 ++-- x/distribution/go.mod | 2 +- x/distribution/go.sum | 4 ++-- x/epochs/go.mod | 2 +- x/epochs/go.sum | 4 ++-- x/evidence/go.mod | 2 +- x/evidence/go.sum | 4 ++-- x/feegrant/go.mod | 2 +- x/feegrant/go.sum | 4 ++-- x/gov/go.mod | 2 +- x/gov/go.sum | 4 ++-- x/group/go.mod | 2 +- x/group/go.sum | 4 ++-- x/mint/go.mod | 2 +- x/mint/go.sum | 4 ++-- x/nft/go.mod | 2 +- x/nft/go.sum | 4 ++-- x/params/go.sum | 4 ++-- x/protocolpool/go.mod | 2 +- x/protocolpool/go.sum | 4 ++-- x/slashing/go.mod | 2 +- x/slashing/go.sum | 4 ++-- x/staking/go.mod | 2 +- x/staking/go.sum | 4 ++-- x/upgrade/go.mod | 2 +- x/upgrade/go.sum | 4 ++-- 71 files changed, 154 insertions(+), 140 deletions(-) diff --git a/client/pruning/main.go b/client/pruning/main.go index 0168308ea325..cc50bc612ca4 100644 --- a/client/pruning/main.go +++ b/client/pruning/main.go @@ -10,10 +10,10 @@ import ( "github.com/spf13/viper" corestore "cosmossdk.io/core/store" - "cosmossdk.io/log" pruningtypes "cosmossdk.io/store/pruning/types" "cosmossdk.io/store/rootmulti" + "github.com/cosmos/cosmos-sdk/client" "github.com/cosmos/cosmos-sdk/client/flags" "github.com/cosmos/cosmos-sdk/server" servertypes "github.com/cosmos/cosmos-sdk/server/types" @@ -78,7 +78,7 @@ Supported app-db-backend types include 'goleveldb', 'rocksdb', 'pebbledb'.`, // in our test, it's important to close db explicitly for pebbledb to write to disk. defer db.Close() - logger := log.NewLogger(cmd.OutOrStdout()) + logger := client.GetLoggerFromCmd(cmd) app := appCreator(logger, db, nil, vp) cms := app.CommitMultiStore() diff --git a/client/snapshot/export.go b/client/snapshot/export.go index 30f573676d33..42ec84b32d86 100644 --- a/client/snapshot/export.go +++ b/client/snapshot/export.go @@ -3,9 +3,8 @@ package snapshot import ( "github.com/spf13/cobra" - "cosmossdk.io/log" - "github.com/cosmos/cosmos-sdk/client" + "github.com/cosmos/cosmos-sdk/client/flags" "github.com/cosmos/cosmos-sdk/server" servertypes "github.com/cosmos/cosmos-sdk/server/types" ) @@ -17,20 +16,19 @@ func ExportSnapshotCmd[T servertypes.Application](appCreator servertypes.AppCrea Short: "Export app state to snapshot store", Args: cobra.NoArgs, RunE: func(cmd *cobra.Command, args []string) error { - cfg := client.GetConfigFromCmd(cmd) viper := client.GetViperFromCmd(cmd) + logger := client.GetLoggerFromCmd(cmd) height, err := cmd.Flags().GetInt64("height") if err != nil { return err } - home := cfg.RootDir + home := viper.GetString(flags.FlagHome) db, err := openDB(home, server.GetAppDBBackend(viper)) if err != nil { return err } - logger := log.NewLogger(cmd.OutOrStdout()) app := appCreator(logger, db, nil, viper) if height == 0 { diff --git a/client/snapshot/list.go b/client/snapshot/list.go index 43ff31cfd4ac..1d564c44f5cc 100644 --- a/client/snapshot/list.go +++ b/client/snapshot/list.go @@ -23,6 +23,11 @@ var ListSnapshotsCmd = &cobra.Command{ if err != nil { return fmt.Errorf("failed to list snapshots: %w", err) } + + if len(snapshots) == 0 { + cmd.Println("no snapshots found") + } + for _, snapshot := range snapshots { cmd.Println("height:", snapshot.Height, "format:", snapshot.Format, "chunks:", snapshot.Chunks) } diff --git a/client/snapshot/restore.go b/client/snapshot/restore.go index 3cb6e4154e25..db32801bf10e 100644 --- a/client/snapshot/restore.go +++ b/client/snapshot/restore.go @@ -8,9 +8,9 @@ import ( "github.com/spf13/cobra" corestore "cosmossdk.io/core/store" - "cosmossdk.io/log" "github.com/cosmos/cosmos-sdk/client" + "github.com/cosmos/cosmos-sdk/client/flags" "github.com/cosmos/cosmos-sdk/server" servertypes "github.com/cosmos/cosmos-sdk/server/types" ) @@ -23,8 +23,8 @@ func RestoreSnapshotCmd[T servertypes.Application](appCreator servertypes.AppCre Long: "Restore app state from local snapshot", Args: cobra.ExactArgs(2), RunE: func(cmd *cobra.Command, args []string) error { - cfg := client.GetConfigFromCmd(cmd) viper := client.GetViperFromCmd(cmd) + logger := client.GetLoggerFromCmd(cmd) height, err := strconv.ParseUint(args[0], 10, 64) if err != nil { @@ -35,12 +35,11 @@ func RestoreSnapshotCmd[T servertypes.Application](appCreator servertypes.AppCre return err } - home := cfg.RootDir + home := viper.GetString(flags.FlagHome) db, err := openDB(home, server.GetAppDBBackend(viper)) if err != nil { return err } - logger := log.NewLogger(cmd.OutOrStdout()) app := appCreator(logger, db, nil, viper) sm := app.SnapshotManager() diff --git a/client/v2/go.mod b/client/v2/go.mod index 88105c4e9d9c..07d6536ed8a1 100644 --- a/client/v2/go.mod +++ b/client/v2/go.mod @@ -61,7 +61,7 @@ require ( github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cometbft/cometbft/api v1.0.0 github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/go-bip39 v1.0.0 github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/gogoproto v1.7.0 diff --git a/client/v2/go.sum b/client/v2/go.sum index 6ee6e63398a3..353d76a21711 100644 --- a/client/v2/go.sum +++ b/client/v2/go.sum @@ -123,8 +123,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/go.mod b/go.mod index 22955d0cf96a..94c89a6064f1 100644 --- a/go.mod +++ b/go.mod @@ -21,7 +21,7 @@ require ( github.com/cometbft/cometbft v1.0.0 github.com/cometbft/cometbft/api v1.0.0 github.com/cosmos/btcutil v1.0.5 - github.com/cosmos/cosmos-db v1.1.0 + github.com/cosmos/cosmos-db v1.1.1 github.com/cosmos/cosmos-proto v1.0.0-beta.5 github.com/cosmos/go-bip39 v1.0.0 github.com/cosmos/gogogateway v1.2.0 diff --git a/go.sum b/go.sum index 1b54d4a649f2..84e939dc151c 100644 --- a/go.sum +++ b/go.sum @@ -114,8 +114,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/server/cmt_cmds.go b/server/cmt_cmds.go index d511cc105696..4c218ac478fc 100644 --- a/server/cmt_cmds.go +++ b/server/cmt_cmds.go @@ -17,8 +17,6 @@ import ( "github.com/spf13/cobra" "sigs.k8s.io/yaml" - "cosmossdk.io/log" - "github.com/cosmos/cosmos-sdk/client" "github.com/cosmos/cosmos-sdk/client/flags" "github.com/cosmos/cosmos-sdk/client/rpc" @@ -371,7 +369,7 @@ func BootstrapStateCmd[T types.Application](appCreator types.AppCreator[T]) *cob Example: "bootstrap-state --height 1000000", RunE: func(cmd *cobra.Command, args []string) error { serverCtx := GetServerContextFromCmd(cmd) - logger := log.NewLogger(cmd.OutOrStdout()) + logger := client.GetLoggerFromCmd(cmd) cfg := serverCtx.Config height, err := cmd.Flags().GetInt64("height") diff --git a/server/v2/api/telemetry/config.go b/server/v2/api/telemetry/config.go index bb26d942d7c7..277b030c8a33 100644 --- a/server/v2/api/telemetry/config.go +++ b/server/v2/api/telemetry/config.go @@ -3,7 +3,7 @@ package telemetry func DefaultConfig() *Config { return &Config{ Enable: true, - Address: "localhost:1327", + Address: "localhost:7180", ServiceName: "", EnableHostname: false, EnableHostnameLabel: false, diff --git a/server/v2/cometbft/go.sum b/server/v2/cometbft/go.sum index 514a30302f34..37fd6c96160b 100644 --- a/server/v2/cometbft/go.sum +++ b/server/v2/cometbft/go.sum @@ -119,8 +119,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/server/v2/stf/stf_test.go b/server/v2/stf/stf_test.go index 04ae8bf5859b..bb42ed3094ae 100644 --- a/server/v2/stf/stf_test.go +++ b/server/v2/stf/stf_test.go @@ -228,6 +228,8 @@ func TestSTF(t *testing.T) { if len(events) != 7 { t.Fatalf("Expected 7 TxEvents, got %d", len(events)) } + + const message = "message" for i, event := range events { if event.BlockStage != appdata.TxProcessingStage { t.Errorf("Expected BlockStage %d, got %d", appdata.TxProcessingStage, event.BlockStage) @@ -236,7 +238,7 @@ func TestSTF(t *testing.T) { t.Errorf("Expected TxIndex 1, got %d", event.TxIndex) } if event.EventIndex != int32(i%2+1) && - (event.Type == "message" && event.EventIndex != 3) { // special case for message event type as it happens in the msg handling flow + (event.Type == message && event.EventIndex != 3) { // special case for message event type as it happens in the msg handling flow t.Errorf("Expected EventIndex %d, got %d", i%2+1, event.EventIndex) } @@ -248,7 +250,7 @@ func TestSTF(t *testing.T) { t.Errorf("Expected 1 or 2 attributes, got %d", len(attrs)) } - if len(attrs) == 2 && event.Type != "message" { + if len(attrs) == 2 && event.Type != message { if attrs[1].Key != "index" || attrs[1].Value != "2" { t.Errorf("Expected attribute key 'index' and value '2', got key '%s' and value '%s'", attrs[1].Key, attrs[1].Value) } @@ -275,7 +277,7 @@ func TestSTF(t *testing.T) { t.Errorf("Expected msg attribute with value '&BoolValue{Value:true,XXX_unrecognized:[],}', got '%s'", attrs[0].Value) } case 4: - if event.Type != "message" { + if event.Type != message { t.Errorf("Expected event type 'message', got %s", event.Type) } diff --git a/server/v2/store/commands.go b/server/v2/store/commands.go index 2779ab5de20f..efb6b9b7b66b 100644 --- a/server/v2/store/commands.go +++ b/server/v2/store/commands.go @@ -38,8 +38,7 @@ Supported app-db-backend types include 'goleveldb', 'rocksdb', 'pebbledb'.`, if err := vp.BindPFlags(cmd.PersistentFlags()); err != nil { return err } - - logger := log.NewLogger(cmd.OutOrStdout()) + logger := serverv2.GetLoggerFromCmd(cmd) rootStore, opts, err := createRootStore(vp, logger) if err != nil { diff --git a/server/v2/store/snapshot.go b/server/v2/store/snapshot.go index bf9e5ddb3827..0fa2b67c708e 100644 --- a/server/v2/store/snapshot.go +++ b/server/v2/store/snapshot.go @@ -38,7 +38,7 @@ func (s *Server[T]) ExportSnapshotCmd() *cobra.Command { return err } - logger := log.NewLogger(cmd.OutOrStdout()) + logger := serverv2.GetLoggerFromCmd(cmd) rootStore, _, err := createRootStore(v, logger) if err != nil { return err @@ -83,6 +83,7 @@ func (s *Server[T]) RestoreSnapshotCmd() *cobra.Command { Args: cobra.ExactArgs(2), RunE: func(cmd *cobra.Command, args []string) error { v := serverv2.GetViperFromCmd(cmd) + logger := serverv2.GetLoggerFromCmd(cmd) height, err := strconv.ParseUint(args[0], 10, 64) if err != nil { @@ -93,8 +94,6 @@ func (s *Server[T]) RestoreSnapshotCmd() *cobra.Command { return err } - logger := log.NewLogger(cmd.OutOrStdout()) - rootStore, _, err := createRootStore(v, logger) if err != nil { return fmt.Errorf("failed to create root store: %w", err) @@ -128,6 +127,11 @@ func (s *Server[T]) ListSnapshotsCmd() *cobra.Command { if err != nil { return fmt.Errorf("failed to list snapshots: %w", err) } + + if len(snapshots) == 0 { + cmd.Println("no snapshots found") + } + for _, snapshot := range snapshots { cmd.Println("height:", snapshot.Height, "format:", snapshot.Format, "chunks:", snapshot.Chunks) } diff --git a/server/v2/testdata/app.toml b/server/v2/testdata/app.toml index be2f82566692..348b057cd5d7 100644 --- a/server/v2/testdata/app.toml +++ b/server/v2/testdata/app.toml @@ -25,8 +25,6 @@ minimum-gas-prices = '0stake' app-db-backend = 'goleveldb' [store.options] -# State storage database type. Currently we support: "pebble" and "rocksdb" -ss-type = 'pebble' # State commitment database type. Currently we support: "iavl" and "iavl-v2" sc-type = 'iavl' diff --git a/simapp/go.mod b/simapp/go.mod index 17228aa22717..6828ee5a5c05 100644 --- a/simapp/go.mod +++ b/simapp/go.mod @@ -93,7 +93,7 @@ require ( github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/cosmos-proto v1.0.0-beta.5 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect diff --git a/simapp/go.sum b/simapp/go.sum index c4ed62ea88a5..f2c848311372 100644 --- a/simapp/go.sum +++ b/simapp/go.sum @@ -329,8 +329,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/simapp/v2/go.mod b/simapp/v2/go.mod index 5acf7dfd0dfc..8e62fd71d471 100644 --- a/simapp/v2/go.mod +++ b/simapp/v2/go.mod @@ -97,7 +97,7 @@ require ( github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cometbft/cometbft/api v1.0.0 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/cosmos-proto v1.0.0-beta.5 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect diff --git a/simapp/v2/go.sum b/simapp/v2/go.sum index 39f9620f0efd..0cc5ffdcf230 100644 --- a/simapp/v2/go.sum +++ b/simapp/v2/go.sum @@ -329,8 +329,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/store/v2/db/pebbledb.go b/store/v2/db/pebbledb.go index e5265883c348..c4eb2c0122bf 100644 --- a/store/v2/db/pebbledb.go +++ b/store/v2/db/pebbledb.go @@ -30,6 +30,7 @@ func NewPebbleDB(name, dataDir string) (*PebbleDB, error) { func NewPebbleDBWithOpts(name, dataDir string, opts coreserver.DynamicConfig) (*PebbleDB, error) { do := &pebble.Options{ + Logger: &fatalLogger{}, // pebble info logs are messing up the logs (not a cosmossdk.io/log logger) MaxConcurrentCompactions: func() int { return 3 }, // default 1 } @@ -301,3 +302,13 @@ func (b *pebbleDBBatch) Close() error { func (b *pebbleDBBatch) GetByteSize() (int, error) { return b.batch.Len(), nil } + +type fatalLogger struct { + pebble.Logger +} + +func (*fatalLogger) Fatalf(format string, args ...interface{}) { + pebble.DefaultLogger.Fatalf(format, args...) +} + +func (*fatalLogger) Infof(format string, args ...interface{}) {} diff --git a/systemtests/CHANGELOG.md b/systemtests/CHANGELOG.md index e4ccd09d739c..736ef5c24881 100644 --- a/systemtests/CHANGELOG.md +++ b/systemtests/CHANGELOG.md @@ -36,6 +36,10 @@ Ref: https://github.com/commitizen/conventional-commit-types/blob/v3.0.0/index.j ## [Unreleased] +## [v1.0.0] - 2024-12-19 + +* [#22849](https://github.com/cosmos/cosmos-sdk/pull/22849) Avoid telemetry server conflicts on port 7180 + ## [v1.0.0-rc.4] - 2024-12-10 * [#22810](https://github.com/cosmos/cosmos-sdk/pull/22810) Avoid HTTP server conflicts on port 8080 diff --git a/systemtests/system.go b/systemtests/system.go index fa39beb4d509..ccbc9e05f5a4 100644 --- a/systemtests/system.go +++ b/systemtests/system.go @@ -39,11 +39,12 @@ var ( MaxGas = 10_000_000 // DefaultApiPort is the port for the node to interact with - DefaultApiPort = 1317 - DefaultRpcPort = 26657 - DefaultRestPort = 8080 - DefaultGrpcPort = 9090 - DefaultP2PPort = 16656 + DefaultApiPort = 1317 + DefaultRpcPort = 26657 + DefaultTelemetryPort = 7180 + DefaultRestPort = 8080 + DefaultGrpcPort = 9090 + DefaultP2PPort = 16656 ) type TestnetInitializer interface { @@ -737,6 +738,7 @@ func (s *SystemUnderTest) AddFullnode(t *testing.T, beforeStart ...func(nodeNumb EditToml(file, func(doc *tomledit.Document) { SetValue(doc, fmt.Sprintf("%s:%d", node.IP, DefaultApiPort+nodeNumber), "grpc-gateway", "address") SetValue(doc, fmt.Sprintf("%s:%d", node.IP, DefaultRestPort+nodeNumber), "rest", "address") + SetValue(doc, fmt.Sprintf("%s:%d", node.IP, DefaultTelemetryPort+nodeNumber), "telemetry", "address") }) } } diff --git a/systemtests/testnet_init.go b/systemtests/testnet_init.go index e5518cf1265e..1e7b41378220 100644 --- a/systemtests/testnet_init.go +++ b/systemtests/testnet_init.go @@ -165,6 +165,7 @@ func (s ModifyConfigYamlInitializer) Initialize() { copy(peers[nodeNumber:], nodeAddresses[nodeNumber+1:]) SetValue(doc, strings.Join(peers, ","), "p2p", "persistent_peers") SetValue(doc, s.commitTimeout.String(), "consensus", "timeout_commit") + SetValue(doc, "goleveldb", "db_backend") }) EditToml(filepath.Join(nodeDir, "app.toml"), func(doc *tomledit.Document) { UpdatePort(doc, DefaultApiPort+i, "api", "address") diff --git a/tests/go.mod b/tests/go.mod index 4d521a2bce26..b22264553d96 100644 --- a/tests/go.mod +++ b/tests/go.mod @@ -50,7 +50,7 @@ require ( cosmossdk.io/x/slashing v0.0.0-00010101000000-000000000000 cosmossdk.io/x/staking v0.0.0-20240226161501-23359a0b6d91 github.com/cometbft/cometbft/api v1.0.0 - github.com/cosmos/cosmos-db v1.1.0 + github.com/cosmos/cosmos-db v1.1.1 github.com/gogo/protobuf v1.3.2 // indirect github.com/google/go-cmp v0.6.0 github.com/google/gofuzz v1.2.0 diff --git a/tests/go.sum b/tests/go.sum index 30da8c46413b..b541c86155fb 100644 --- a/tests/go.sum +++ b/tests/go.sum @@ -327,8 +327,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/tests/systemtests/export_test.go b/tests/systemtests/export_test.go index bd50457cb234..279bb7f00a9a 100644 --- a/tests/systemtests/export_test.go +++ b/tests/systemtests/export_test.go @@ -6,7 +6,6 @@ import ( "fmt" "os" "path/filepath" - "strings" "testing" "github.com/stretchr/testify/assert" @@ -44,6 +43,7 @@ func TestChainExportImport(t *testing.T) { }) systest.Sut.StartChain(t) systest.Sut.AwaitNBlocks(t, 2) + systest.Sut.StopChain() } func TestExportCmd_WithHeight(t *testing.T) { @@ -62,22 +62,16 @@ func TestExportCmd_WithHeight(t *testing.T) { args []string expZeroHeight bool }{ - {"should export correct height", []string{"genesis", "export", "--home", systest.Sut.NodeDir(0), "--log_level=disabled"}, false}, - {"should export correct height with --height", []string{"genesis", "export", "--height=5", "--home", systest.Sut.NodeDir(0), "--log_level=disabled"}, false}, - {"should export height 0 with --for-zero-height", []string{"genesis", "export", "--for-zero-height=true", "--home", systest.Sut.NodeDir(0), "--log_level=disabled"}, true}, + {"should export correct height", []string{"genesis", "export", "--home", systest.Sut.NodeDir(0), disabledLog}, false}, + {"should export correct height with --height", []string{"genesis", "export", "--height=5", "--home", systest.Sut.NodeDir(0), disabledLog}, false}, + {"should export height 0 with --for-zero-height", []string{"genesis", "export", "--for-zero-height=true", "--home", systest.Sut.NodeDir(0), disabledLog}, true}, } for _, tc := range testCases { - res := cli.RunCommandWithArgs(tc.args...) - // PebbleDB logs are printed directly to stderr. - // Cosmos-DB and Store/v2 do not provide a way to override the logger. - // This isn't problematic in a real-world scenario, but it makes it hard to test the output. - // https://github.com/cockroachdb/pebble/blob/v1.1.2/internal/base/logger.go#L26-L40 - // We trim the output to get the JSON part only - if i := strings.Index(res, "{"); i > 0 { - res = res[i:] - } - + res := cli. + WithRunErrorsIgnored(). + WithRunSingleOutput(). // pebbledb prints logs to stderr, we cannot override the logger in store/v2 and cosmos-db. This isn't problematic in a real-world scenario, but it makes it hard to test the output + RunCommandWithArgs(tc.args...) height := gjson.Get(res, "initial_height").Int() if tc.expZeroHeight { require.Equal(t, height, int64(0)) diff --git a/tests/systemtests/go.mod b/tests/systemtests/go.mod index 150ebd18571d..47cac454ddf2 100644 --- a/tests/systemtests/go.mod +++ b/tests/systemtests/go.mod @@ -4,7 +4,7 @@ go 1.23 require ( cosmossdk.io/math v1.4.0 - cosmossdk.io/systemtests v1.0.0-rc.3 + cosmossdk.io/systemtests v1.0.0-rc.4.0.20241219115822-5bebfb028815 github.com/cosmos/cosmos-sdk v0.50.11 ) diff --git a/tests/systemtests/go.sum b/tests/systemtests/go.sum index 8f68598d649e..23707b5872a6 100644 --- a/tests/systemtests/go.sum +++ b/tests/systemtests/go.sum @@ -16,8 +16,8 @@ cosmossdk.io/math v1.4.0 h1:XbgExXFnXmF/CccPPEto40gOO7FpWu9yWNAZPN3nkNQ= cosmossdk.io/math v1.4.0/go.mod h1:O5PkD4apz2jZs4zqFdTr16e1dcaQCc5z6lkEnrrppuk= cosmossdk.io/store v1.1.1 h1:NA3PioJtWDVU7cHHeyvdva5J/ggyLDkyH0hGHl2804Y= cosmossdk.io/store v1.1.1/go.mod h1:8DwVTz83/2PSI366FERGbWSH7hL6sB7HbYp8bqksNwM= -cosmossdk.io/systemtests v1.0.0-rc.3 h1:W1ZdfHtWxbzRCiBwcMb1nMKkmUNyAcHapJOrfh1lX20= -cosmossdk.io/systemtests v1.0.0-rc.3/go.mod h1:B3RY1tY/iwLjQ9MUTz+GsiXV9gEdS8mfUvSQtWUwaAo= +cosmossdk.io/systemtests v1.0.0-rc.4.0.20241219115822-5bebfb028815 h1:ilHcBaGJrYhEKC2gO+DVBmWOPNt1VUWNQz3g7/9X4Ow= +cosmossdk.io/systemtests v1.0.0-rc.4.0.20241219115822-5bebfb028815/go.mod h1:tTDdBl9wnIyP4L1Gi1/GXD1onPfaAOsscWCw7JUuJ/A= cosmossdk.io/x/tx v1.0.0-alpha.3 h1:+55/JFH5QRqnFhOI2heH3DKsaNL0RpXcJOQNzUvHiaQ= cosmossdk.io/x/tx v1.0.0-alpha.3/go.mod h1:h4pQ/j6Gfu8goB1R3Jbl4qY4RjYVNAsoylcleTXdSRg= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= diff --git a/tests/systemtests/snapshots_test.go b/tests/systemtests/snapshots_test.go index 3377e55fb7af..6fae39749eb4 100644 --- a/tests/systemtests/snapshots_test.go +++ b/tests/systemtests/snapshots_test.go @@ -12,8 +12,10 @@ import ( systest "cosmossdk.io/systemtests" ) +const disabledLog = "--log_level=disabled" + func TestSnapshots(t *testing.T) { - t.Skip("Skip snapshots test, flaky due to pebbledb logs on CI") + t.Skip("Not persisting properly on CI") systest.Sut.ResetChain(t) cli := systest.NewCLIWrapper(t, systest.Sut, systest.Verbose) @@ -38,28 +40,25 @@ func TestSnapshots(t *testing.T) { } // export snapshot at height 5 - res := cli.RunCommandWithArgs(command, "export", "--height=5", fmt.Sprintf("--home=%s", node0Dir)) + res := cli.RunCommandWithArgs(command, "export", "--height=5", fmt.Sprintf("--home=%s", node0Dir), disabledLog) require.Contains(t, res, "Snapshot created at height 5") require.DirExists(t, fmt.Sprintf("%s/data/snapshots/5/3", node0Dir)) // Check snapshots list - res = cli. - WithRunErrorsIgnored(). - WithRunSingleOutput(). // pebbledb prints logs to stderr, we cannot override the logger in store/v2 and cosmos-db. This isn't problematic in a real-world scenario, but it makes it hard to test the output. - RunCommandWithArgs(command, "list", fmt.Sprintf("--home=%s", node0Dir)) + res = cli.RunCommandWithArgs(command, "list", fmt.Sprintf("--home=%s", node0Dir), disabledLog) require.Contains(t, res, "height: 5") // Dump snapshot - res = cli.RunCommandWithArgs(command, "dump", "5", "3", fmt.Sprintf("--home=%s", node0Dir), fmt.Sprintf("--output=%s/5-3.tar.gz", node0Dir)) + res = cli.RunCommandWithArgs(command, "dump", "5", "3", fmt.Sprintf("--home=%s", node0Dir), fmt.Sprintf("--output=%s/5-3.tar.gz", node0Dir), disabledLog) // Check if output file exist require.FileExists(t, fmt.Sprintf("%s/5-3.tar.gz", node0Dir)) // Delete snapshots - res = cli.RunCommandWithArgs(command, "delete", "5", "3", fmt.Sprintf("--home=%s", node0Dir)) + res = cli.RunCommandWithArgs(command, "delete", "5", "3", fmt.Sprintf("--home=%s", node0Dir), disabledLog) require.NoDirExists(t, fmt.Sprintf("%s/data/snapshots/5/3", node0Dir)) // Load snapshot from file - res = cli.RunCommandWithArgs(command, "load", fmt.Sprintf("%s/5-3.tar.gz", node0Dir), fmt.Sprintf("--home=%s", node0Dir)) + res = cli.RunCommandWithArgs(command, "load", fmt.Sprintf("%s/5-3.tar.gz", node0Dir), fmt.Sprintf("--home=%s", node0Dir), disabledLog) require.DirExists(t, fmt.Sprintf("%s/data/snapshots/5/3", node0Dir)) // Restore from snapshots @@ -73,14 +72,14 @@ func TestSnapshots(t *testing.T) { require.NoError(t, os.RemoveAll(fmt.Sprintf("%s/data/ss", node0Dir))) } - res = cli.RunCommandWithArgs(command, "restore", "5", "3", fmt.Sprintf("--home=%s", node0Dir)) + res = cli.RunCommandWithArgs(command, "restore", "5", "3", fmt.Sprintf("--home=%s", node0Dir), disabledLog) for _, dir := range restoreableDirs { require.DirExists(t, dir) } } func TestPrune(t *testing.T) { - t.Skip("Skip snapshots test, flaky due to pebbledb logs on CI") + t.Skip("Not persisting properly on CI") systest.Sut.ResetChain(t) cli := systest.NewCLIWrapper(t, systest.Sut, systest.Verbose) @@ -102,6 +101,6 @@ func TestPrune(t *testing.T) { } else { command = []string{"prune", "everything"} } - res := cli.RunCommandWithArgs(append(command, fmt.Sprintf("--home=%s", node0Dir))...) + res := cli.RunCommandWithArgs(append(command, fmt.Sprintf("--home=%s", node0Dir), disabledLog)...) require.Contains(t, res, "successfully pruned the application root multi stores") } diff --git a/tools/benchmark/go.sum b/tools/benchmark/go.sum index 2af0c7024b34..5ad8a0ea6334 100644 --- a/tools/benchmark/go.sum +++ b/tools/benchmark/go.sum @@ -98,8 +98,8 @@ github.com/cometbft/cometbft/api v1.0.0/go.mod h1:EkQiqVSu/p2ebrZEnB2z6Re7r8XNe/ github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/tools/confix/data/v2-app.toml b/tools/confix/data/v2-app.toml index 7d136a84c2c6..51c1e3298dfe 100644 --- a/tools/confix/data/v2-app.toml +++ b/tools/confix/data/v2-app.toml @@ -86,7 +86,7 @@ skip-fast-storage-upgrade = true # Enable enables the application telemetry functionality. When enabled, an in-memory sink is also enabled by default. Operators may also enabled other sinks such as Prometheus. enable = true # Address defines the metrics server address to bind to. -address = 'localhost:1327' +address = 'localhost:7180' # Prefixed with keys to separate services. service-name = '' # Enable prefixing gauge values with hostname. diff --git a/x/accounts/defaults/base/go.mod b/x/accounts/defaults/base/go.mod index 35c9d6decbed..b9e870b1759e 100644 --- a/x/accounts/defaults/base/go.mod +++ b/x/accounts/defaults/base/go.mod @@ -50,7 +50,7 @@ require ( github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cometbft/cometbft/api v1.0.0 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/cosmos-proto v1.0.0-beta.5 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect diff --git a/x/accounts/defaults/base/go.sum b/x/accounts/defaults/base/go.sum index 2f0011d6a5cf..0d5a1827a617 100644 --- a/x/accounts/defaults/base/go.sum +++ b/x/accounts/defaults/base/go.sum @@ -117,8 +117,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/accounts/defaults/lockup/go.sum b/x/accounts/defaults/lockup/go.sum index 2af0c7024b34..5ad8a0ea6334 100644 --- a/x/accounts/defaults/lockup/go.sum +++ b/x/accounts/defaults/lockup/go.sum @@ -98,8 +98,8 @@ github.com/cometbft/cometbft/api v1.0.0/go.mod h1:EkQiqVSu/p2ebrZEnB2z6Re7r8XNe/ github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/accounts/defaults/multisig/go.mod b/x/accounts/defaults/multisig/go.mod index c3130e05266f..ed9d917caca0 100644 --- a/x/accounts/defaults/multisig/go.mod +++ b/x/accounts/defaults/multisig/go.mod @@ -50,7 +50,7 @@ require ( github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cometbft/cometbft/api v1.0.0 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect diff --git a/x/accounts/defaults/multisig/go.sum b/x/accounts/defaults/multisig/go.sum index 2f0011d6a5cf..0d5a1827a617 100644 --- a/x/accounts/defaults/multisig/go.sum +++ b/x/accounts/defaults/multisig/go.sum @@ -117,8 +117,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/accounts/go.mod b/x/accounts/go.mod index 9844fbca3702..a126c9a80c7b 100644 --- a/x/accounts/go.mod +++ b/x/accounts/go.mod @@ -24,7 +24,7 @@ require ( github.com/cloudwego/base64x v0.1.4 // indirect github.com/cloudwego/iasm v0.2.0 // indirect github.com/cockroachdb/fifo v0.0.0-20240606204812-0bbfbd93a7ce // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect github.com/dgraph-io/ristretto/v2 v2.0.0 // indirect github.com/google/uuid v1.6.0 // indirect diff --git a/x/accounts/go.sum b/x/accounts/go.sum index 2f0011d6a5cf..0d5a1827a617 100644 --- a/x/accounts/go.sum +++ b/x/accounts/go.sum @@ -117,8 +117,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/authz/go.mod b/x/authz/go.mod index b59ec5d1554c..a4d05d11af2b 100644 --- a/x/authz/go.mod +++ b/x/authz/go.mod @@ -166,7 +166,7 @@ require ( github.com/cloudwego/base64x v0.1.4 // indirect github.com/cloudwego/iasm v0.2.0 // indirect github.com/cockroachdb/fifo v0.0.0-20240606204812-0bbfbd93a7ce // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/dgraph-io/ristretto/v2 v2.0.0 // indirect github.com/google/uuid v1.6.0 // indirect github.com/klauspost/cpuid/v2 v2.2.9 // indirect diff --git a/x/authz/go.sum b/x/authz/go.sum index 2f0011d6a5cf..0d5a1827a617 100644 --- a/x/authz/go.sum +++ b/x/authz/go.sum @@ -117,8 +117,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/bank/go.mod b/x/bank/go.mod index 68f3a756b98d..8a1a942988df 100644 --- a/x/bank/go.mod +++ b/x/bank/go.mod @@ -164,7 +164,7 @@ require ( github.com/cloudwego/base64x v0.1.4 // indirect github.com/cloudwego/iasm v0.2.0 // indirect github.com/cockroachdb/fifo v0.0.0-20240606204812-0bbfbd93a7ce // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/dgraph-io/ristretto/v2 v2.0.0 // indirect github.com/google/uuid v1.6.0 // indirect github.com/klauspost/cpuid/v2 v2.2.9 // indirect diff --git a/x/bank/go.sum b/x/bank/go.sum index 2f0011d6a5cf..0d5a1827a617 100644 --- a/x/bank/go.sum +++ b/x/bank/go.sum @@ -117,8 +117,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/circuit/go.mod b/x/circuit/go.mod index 57812f55cfdb..f79397fb4008 100644 --- a/x/circuit/go.mod +++ b/x/circuit/go.mod @@ -51,7 +51,7 @@ require ( github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cometbft/cometbft/api v1.0.0 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/cosmos-proto v1.0.0-beta.5 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect diff --git a/x/circuit/go.sum b/x/circuit/go.sum index 2f0011d6a5cf..0d5a1827a617 100644 --- a/x/circuit/go.sum +++ b/x/circuit/go.sum @@ -117,8 +117,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/consensus/go.mod b/x/consensus/go.mod index 37a11e8833b8..263ac1d1a106 100644 --- a/x/consensus/go.mod +++ b/x/consensus/go.mod @@ -52,7 +52,7 @@ require ( github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect diff --git a/x/consensus/go.sum b/x/consensus/go.sum index 2f0011d6a5cf..0d5a1827a617 100644 --- a/x/consensus/go.sum +++ b/x/consensus/go.sum @@ -117,8 +117,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/distribution/go.mod b/x/distribution/go.mod index befaa65c7a68..1787e05ab9ea 100644 --- a/x/distribution/go.mod +++ b/x/distribution/go.mod @@ -55,7 +55,7 @@ require ( github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cometbft/cometbft/api v1.0.0 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect diff --git a/x/distribution/go.sum b/x/distribution/go.sum index 2f0011d6a5cf..0d5a1827a617 100644 --- a/x/distribution/go.sum +++ b/x/distribution/go.sum @@ -117,8 +117,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/epochs/go.mod b/x/epochs/go.mod index 71debbefc663..6dcb1d331657 100644 --- a/x/epochs/go.mod +++ b/x/epochs/go.mod @@ -48,7 +48,7 @@ require ( github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cometbft/cometbft/api v1.0.0 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect diff --git a/x/epochs/go.sum b/x/epochs/go.sum index 2f0011d6a5cf..0d5a1827a617 100644 --- a/x/epochs/go.sum +++ b/x/epochs/go.sum @@ -117,8 +117,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/evidence/go.mod b/x/evidence/go.mod index ca4b24a188fc..cb3644c8a41c 100644 --- a/x/evidence/go.mod +++ b/x/evidence/go.mod @@ -55,7 +55,7 @@ require ( github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cometbft/cometbft/api v1.0.0 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect diff --git a/x/evidence/go.sum b/x/evidence/go.sum index 2f0011d6a5cf..0d5a1827a617 100644 --- a/x/evidence/go.sum +++ b/x/evidence/go.sum @@ -117,8 +117,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/feegrant/go.mod b/x/feegrant/go.mod index 2521cf7ccfc4..070adf7adf81 100644 --- a/x/feegrant/go.mod +++ b/x/feegrant/go.mod @@ -64,7 +64,7 @@ require ( github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cometbft/cometbft/api v1.0.0 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect diff --git a/x/feegrant/go.sum b/x/feegrant/go.sum index 6ee6e63398a3..353d76a21711 100644 --- a/x/feegrant/go.sum +++ b/x/feegrant/go.sum @@ -123,8 +123,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/gov/go.mod b/x/gov/go.mod index 3611933fd117..c6d68f9242aa 100644 --- a/x/gov/go.mod +++ b/x/gov/go.mod @@ -56,7 +56,7 @@ require ( github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cometbft/cometbft/api v1.0.0 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect diff --git a/x/gov/go.sum b/x/gov/go.sum index 6ee6e63398a3..353d76a21711 100644 --- a/x/gov/go.sum +++ b/x/gov/go.sum @@ -123,8 +123,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/group/go.mod b/x/group/go.mod index c05544b070b6..0c3c7847ee07 100644 --- a/x/group/go.mod +++ b/x/group/go.mod @@ -60,7 +60,7 @@ require ( github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cometbft/cometbft/api v1.0.0 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect diff --git a/x/group/go.sum b/x/group/go.sum index 5a112ab33586..bec6ad074cda 100644 --- a/x/group/go.sum +++ b/x/group/go.sum @@ -127,8 +127,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/mint/go.mod b/x/mint/go.mod index 6c5a943407b8..389269a55af4 100644 --- a/x/mint/go.mod +++ b/x/mint/go.mod @@ -166,7 +166,7 @@ require ( github.com/cloudwego/iasm v0.2.0 // indirect github.com/cockroachdb/fifo v0.0.0-20240606204812-0bbfbd93a7ce // indirect github.com/cometbft/cometbft/api v1.0.0 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/dgraph-io/ristretto/v2 v2.0.0 // indirect github.com/google/uuid v1.6.0 // indirect github.com/klauspost/cpuid/v2 v2.2.9 // indirect diff --git a/x/mint/go.sum b/x/mint/go.sum index 2f0011d6a5cf..0d5a1827a617 100644 --- a/x/mint/go.sum +++ b/x/mint/go.sum @@ -117,8 +117,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/nft/go.mod b/x/nft/go.mod index b643b19233a3..357fbbd1163e 100644 --- a/x/nft/go.mod +++ b/x/nft/go.mod @@ -53,7 +53,7 @@ require ( github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cometbft/cometbft/api v1.0.0 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect diff --git a/x/nft/go.sum b/x/nft/go.sum index 2f0011d6a5cf..0d5a1827a617 100644 --- a/x/nft/go.sum +++ b/x/nft/go.sum @@ -117,8 +117,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/params/go.sum b/x/params/go.sum index 87fc1bdf453a..74df568acf9e 100644 --- a/x/params/go.sum +++ b/x/params/go.sum @@ -107,8 +107,8 @@ github.com/cometbft/cometbft/api v1.0.0/go.mod h1:EkQiqVSu/p2ebrZEnB2z6Re7r8XNe/ github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/protocolpool/go.mod b/x/protocolpool/go.mod index 3e8ca5b9b999..f590efa64ed6 100644 --- a/x/protocolpool/go.mod +++ b/x/protocolpool/go.mod @@ -54,7 +54,7 @@ require ( github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cometbft/cometbft/api v1.0.0 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect diff --git a/x/protocolpool/go.sum b/x/protocolpool/go.sum index 2f0011d6a5cf..0d5a1827a617 100644 --- a/x/protocolpool/go.sum +++ b/x/protocolpool/go.sum @@ -117,8 +117,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/slashing/go.mod b/x/slashing/go.mod index 637bb40b780f..47cd299492be 100644 --- a/x/slashing/go.mod +++ b/x/slashing/go.mod @@ -56,7 +56,7 @@ require ( github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cometbft/cometbft/api v1.0.0 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect diff --git a/x/slashing/go.sum b/x/slashing/go.sum index fffc9853cc47..d14c327fc8ee 100644 --- a/x/slashing/go.sum +++ b/x/slashing/go.sum @@ -119,8 +119,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/staking/go.mod b/x/staking/go.mod index e7fd868ee21e..74c6d14e3533 100644 --- a/x/staking/go.mod +++ b/x/staking/go.mod @@ -166,7 +166,7 @@ require ( github.com/cloudwego/base64x v0.1.4 // indirect github.com/cloudwego/iasm v0.2.0 // indirect github.com/cockroachdb/fifo v0.0.0-20240606204812-0bbfbd93a7ce // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/dgraph-io/ristretto/v2 v2.0.0 // indirect github.com/google/uuid v1.6.0 // indirect github.com/klauspost/cpuid/v2 v2.2.9 // indirect diff --git a/x/staking/go.sum b/x/staking/go.sum index 2f0011d6a5cf..0d5a1827a617 100644 --- a/x/staking/go.sum +++ b/x/staking/go.sum @@ -117,8 +117,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY= diff --git a/x/upgrade/go.mod b/x/upgrade/go.mod index b7f0ef53e75e..75a82955c66e 100644 --- a/x/upgrade/go.mod +++ b/x/upgrade/go.mod @@ -71,7 +71,7 @@ require ( github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect github.com/cometbft/cometbft-db v1.0.1 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.0 // indirect + github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.3.4 // indirect diff --git a/x/upgrade/go.sum b/x/upgrade/go.sum index acd3e38dfe99..8af118967e5b 100644 --- a/x/upgrade/go.sum +++ b/x/upgrade/go.sum @@ -325,8 +325,8 @@ github.com/containerd/continuity v0.3.0/go.mod h1:wJEAIwKOm/pBZuBd0JmeTvnLquTB1A github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cosmos/btcutil v1.0.5 h1:t+ZFcX77LpKtDBhjucvnOH8C2l2ioGsBNEQ3jef8xFk= github.com/cosmos/btcutil v1.0.5/go.mod h1:IyB7iuqZMJlthe2tkIFL33xPyzbFYP0XVdS8P5lUPis= -github.com/cosmos/cosmos-db v1.1.0 h1:KLHNVQ73h7vawXTpj9UJ7ZR2IXv51tsEHkQJJ9EBDzI= -github.com/cosmos/cosmos-db v1.1.0/go.mod h1:t7c4A6cfGdpUwwVxrQ0gQLeRQqGUBJu0yvE4F/26REg= +github.com/cosmos/cosmos-db v1.1.1 h1:FezFSU37AlBC8S98NlSagL76oqBRWq/prTPvFcEJNCM= +github.com/cosmos/cosmos-db v1.1.1/go.mod h1:AghjcIPqdhSLP/2Z0yha5xPH3nLnskz81pBx3tcVSAw= github.com/cosmos/cosmos-proto v1.0.0-beta.5 h1:eNcayDLpip+zVLRLYafhzLvQlSmyab+RC5W7ZfmxJLA= github.com/cosmos/cosmos-proto v1.0.0-beta.5/go.mod h1:hQGLpiIUloJBMdQMMWb/4wRApmI9hjHH05nefC0Ojec= github.com/cosmos/go-bip39 v1.0.0 h1:pcomnQdrdH22njcAatO0yWojsUnCO3y2tNoV1cb6hHY=