From 1d8abfbbccca5174dd374d1ce3db132981b084d0 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Tue, 17 May 2022 15:30:02 +0200 Subject: [PATCH 01/29] Make baseapp changes --- baseapp/abci.go | 56 +- baseapp/abci_test.go | 84 +- baseapp/baseapp.go | 294 +++++- baseapp/baseapp_test.go | 1366 +++++++++------------------- baseapp/msg_service_router.go | 140 +++ baseapp/msg_service_router_test.go | 121 +++ baseapp/options.go | 15 +- baseapp/recovery.go | 77 ++ baseapp/recovery_test.go | 64 ++ baseapp/router.go | 41 + baseapp/router_test.go | 31 + baseapp/test_helpers.go | 66 +- simapp/app.go | 2 +- types/errors/abci.go | 28 + 14 files changed, 1324 insertions(+), 1061 deletions(-) create mode 100644 baseapp/msg_service_router.go create mode 100644 baseapp/msg_service_router_test.go create mode 100644 baseapp/recovery.go create mode 100644 baseapp/recovery_test.go create mode 100644 baseapp/router.go create mode 100644 baseapp/router_test.go diff --git a/baseapp/abci.go b/baseapp/abci.go index e03d4b587fb4..a5898b119fcf 100644 --- a/baseapp/abci.go +++ b/baseapp/abci.go @@ -8,6 +8,7 @@ import ( "sort" "strings" "syscall" + "time" "github.com/gogo/protobuf/proto" abci "github.com/tendermint/tendermint/abci/types" @@ -17,6 +18,7 @@ import ( "github.com/cosmos/cosmos-sdk/codec" snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" + "github.com/cosmos/cosmos-sdk/telemetry" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" "github.com/cosmos/cosmos-sdk/types/tx" @@ -235,11 +237,12 @@ func (app *BaseApp) EndBlock(req abci.RequestEndBlock) (res abci.ResponseEndBloc // CheckTx implements the ABCI interface and executes a tx in CheckTx mode. In // CheckTx mode, messages are not executed. This means messages are only validated -// and only the wired middlewares are executed. State is persisted to the BaseApp's -// internal CheckTx state if the middlewares' CheckTx pass. Otherwise, the ResponseCheckTx +// and only the AnteHandler is executed. State is persisted to the BaseApp's +// internal CheckTx state if the AnteHandler passes. Otherwise, the ResponseCheckTx // will contain releveant error information. Regardless of tx execution outcome, // the ResponseCheckTx will contain relevant gas execution context. func (app *BaseApp) CheckTx(req abci.RequestCheckTx) abci.ResponseCheckTx { + defer telemetry.MeasureSince(time.Now(), "abci", "check_tx") var mode runTxMode @@ -254,18 +257,18 @@ func (app *BaseApp) CheckTx(req abci.RequestCheckTx) abci.ResponseCheckTx { panic(fmt.Sprintf("unknown RequestCheckTx type: %s", req.Type)) } - ctx := app.getContextForTx(mode, req.Tx) - res, checkRes, err := app.txHandler.CheckTx(ctx, tx.Request{TxBytes: req.Tx}, tx.RequestCheckTx{Type: req.Type}) + gInfo, result, anteEvents, err := app.runTx(mode, req.Tx) if err != nil { - return sdkerrors.ResponseCheckTx(err, uint64(res.GasUsed), uint64(res.GasWanted), app.trace) + return sdkerrors.ResponseCheckTxWithEvents(err, gInfo.GasWanted, gInfo.GasUsed, anteEvents, app.trace) } - abciRes, err := convertTxResponseToCheckTx(res, checkRes) - if err != nil { - return sdkerrors.ResponseCheckTx(err, uint64(res.GasUsed), uint64(res.GasWanted), app.trace) + return abci.ResponseCheckTx{ + GasWanted: int64(gInfo.GasWanted), // TODO: Should type accept unsigned ints? + GasUsed: int64(gInfo.GasUsed), // TODO: Should type accept unsigned ints? + Log: result.Log, + Data: result.Data, + Events: sdk.MarkEventsToIndex(result.Events, app.indexEvents), } - - return abciRes } // DeliverTx implements the ABCI interface and executes a tx in DeliverTx mode. @@ -274,30 +277,31 @@ func (app *BaseApp) CheckTx(req abci.RequestCheckTx) abci.ResponseCheckTx { // Regardless of tx execution outcome, the ResponseDeliverTx will contain relevant // gas execution context. func (app *BaseApp) DeliverTx(req abci.RequestDeliverTx) abci.ResponseDeliverTx { + defer telemetry.MeasureSince(time.Now(), "abci", "deliver_tx") + + gInfo := sdk.GasInfo{} + resultStr := "successful" - var abciRes abci.ResponseDeliverTx defer func() { - for _, streamingListener := range app.abciListeners { - if err := streamingListener.ListenDeliverTx(app.deliverState.ctx, req, abciRes); err != nil { - app.logger.Error("DeliverTx listening hook failed", "err", err) - } - } + telemetry.IncrCounter(1, "tx", "count") + telemetry.IncrCounter(1, "tx", resultStr) + telemetry.SetGauge(float32(gInfo.GasUsed), "tx", "gas", "used") + telemetry.SetGauge(float32(gInfo.GasWanted), "tx", "gas", "wanted") }() - ctx := app.getContextForTx(runTxModeDeliver, req.Tx) - res, err := app.txHandler.DeliverTx(ctx, tx.Request{TxBytes: req.Tx}) + gInfo, result, anteEvents, err := app.runTx(runTxModeDeliver, req.Tx) if err != nil { - abciRes = sdkerrors.ResponseDeliverTx(err, uint64(res.GasUsed), uint64(res.GasWanted), app.trace) - return abciRes + resultStr = "failed" + return sdkerrors.ResponseDeliverTxWithEvents(err, gInfo.GasWanted, gInfo.GasUsed, anteEvents, app.trace) } - abciRes, err = convertTxResponseToDeliverTx(res) - if err != nil { - return sdkerrors.ResponseDeliverTx(err, uint64(res.GasUsed), uint64(res.GasWanted), app.trace) + return abci.ResponseDeliverTx{ + GasWanted: int64(gInfo.GasWanted), // TODO: Should type accept unsigned ints? + GasUsed: int64(gInfo.GasUsed), // TODO: Should type accept unsigned ints? + Log: result.Log, + Data: result.Data, + Events: sdk.MarkEventsToIndex(result.Events, app.indexEvents), } - - return abciRes - } // Commit implements the ABCI interface. It will commit all state that exists in diff --git a/baseapp/abci_test.go b/baseapp/abci_test.go index 58c29f0012c2..b382a5a38909 100644 --- a/baseapp/abci_test.go +++ b/baseapp/abci_test.go @@ -1,18 +1,15 @@ -package baseapp_test +package baseapp import ( "testing" "github.com/stretchr/testify/require" abci "github.com/tendermint/tendermint/abci/types" + tmproto "github.com/tendermint/tendermint/proto/tendermint/types" tmprototypes "github.com/tendermint/tendermint/proto/tendermint/types" dbm "github.com/tendermint/tm-db" - "github.com/cosmos/cosmos-sdk/baseapp" - pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types" - "github.com/cosmos/cosmos-sdk/snapshots" - snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" - "github.com/cosmos/cosmos-sdk/testutil" + sdk "github.com/cosmos/cosmos-sdk/types" ) func TestGetBlockRentionHeight(t *testing.T) { @@ -20,85 +17,82 @@ func TestGetBlockRentionHeight(t *testing.T) { db := dbm.NewMemDB() name := t.Name() - snapshotStore, err := snapshots.NewStore(dbm.NewMemDB(), testutil.GetTempDir(t)) - require.NoError(t, err) - testCases := map[string]struct { - bapp *baseapp.BaseApp + bapp *BaseApp maxAgeBlocks int64 commitHeight int64 expected int64 }{ "defaults": { - bapp: baseapp.NewBaseApp(name, logger, db), + bapp: NewBaseApp(name, logger, db, nil), maxAgeBlocks: 0, commitHeight: 499000, expected: 0, }, "pruning unbonding time only": { - bapp: baseapp.NewBaseApp(name, logger, db, baseapp.SetMinRetainBlocks(1)), + bapp: NewBaseApp(name, logger, db, nil, SetMinRetainBlocks(1)), maxAgeBlocks: 362880, commitHeight: 499000, expected: 136120, }, "pruning iavl snapshot only": { - bapp: baseapp.NewBaseApp( - name, logger, db, - baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)), - baseapp.SetMinRetainBlocks(1), - baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(10000, 1)), + bapp: NewBaseApp( + name, logger, db, nil, + SetPruning(sdk.PruningOptions{KeepEvery: 10000}), + SetMinRetainBlocks(1), ), maxAgeBlocks: 0, commitHeight: 499000, - expected: 489000, + expected: 490000, }, "pruning state sync snapshot only": { - bapp: baseapp.NewBaseApp( - name, logger, db, - baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(50000, 3)), - baseapp.SetMinRetainBlocks(1), + bapp: NewBaseApp( + name, logger, db, nil, + SetSnapshotInterval(50000), + SetSnapshotKeepRecent(3), + SetMinRetainBlocks(1), ), maxAgeBlocks: 0, commitHeight: 499000, expected: 349000, }, "pruning min retention only": { - bapp: baseapp.NewBaseApp( - name, logger, db, - baseapp.SetMinRetainBlocks(400000), + bapp: NewBaseApp( + name, logger, db, nil, + SetMinRetainBlocks(400000), ), maxAgeBlocks: 0, commitHeight: 499000, expected: 99000, }, "pruning all conditions": { - bapp: baseapp.NewBaseApp( - name, logger, db, - baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(0, 0)), - baseapp.SetMinRetainBlocks(400000), - baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(50000, 3)), + bapp: NewBaseApp( + name, logger, db, nil, + SetPruning(sdk.PruningOptions{KeepEvery: 10000}), + SetMinRetainBlocks(400000), + SetSnapshotInterval(50000), SetSnapshotKeepRecent(3), ), maxAgeBlocks: 362880, commitHeight: 499000, expected: 99000, }, "no pruning due to no persisted state": { - bapp: baseapp.NewBaseApp( - name, logger, db, - baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(0, 0)), - baseapp.SetMinRetainBlocks(400000), - baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(50000, 3)), + bapp: NewBaseApp( + name, logger, db, nil, + SetPruning(sdk.PruningOptions{KeepEvery: 10000}), + SetMinRetainBlocks(400000), + SetSnapshotInterval(50000), SetSnapshotKeepRecent(3), ), maxAgeBlocks: 362880, commitHeight: 10000, expected: 0, }, "disable pruning": { - bapp: baseapp.NewBaseApp( - name, logger, db, - baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(0, 0)), - baseapp.SetMinRetainBlocks(0), - baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(50000, 3)), + bapp: NewBaseApp( + name, logger, db, nil, + SetPruning(sdk.PruningOptions{KeepEvery: 10000}), + SetMinRetainBlocks(0), + SetSnapshotInterval(50000), SetSnapshotKeepRecent(3), ), maxAgeBlocks: 362880, commitHeight: 499000, @@ -111,7 +105,7 @@ func TestGetBlockRentionHeight(t *testing.T) { tc.bapp.SetParamStore(¶mStore{db: dbm.NewMemDB()}) tc.bapp.InitChain(abci.RequestInitChain{ - ConsensusParams: &tmprototypes.ConsensusParams{ + ConsensusParams: &abci.ConsensusParams{ Evidence: &tmprototypes.EvidenceParams{ MaxAgeNumBlocks: tc.maxAgeBlocks, }, @@ -134,12 +128,12 @@ func TestBaseAppCreateQueryContext(t *testing.T) { logger := defaultLogger() db := dbm.NewMemDB() name := t.Name() - app := baseapp.NewBaseApp(name, logger, db) + app := NewBaseApp(name, logger, db, nil) - app.BeginBlock(abci.RequestBeginBlock{Header: tmprototypes.Header{Height: 1}}) + app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: 1}}) app.Commit() - app.BeginBlock(abci.RequestBeginBlock{Header: tmprototypes.Header{Height: 2}}) + app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: 2}}) app.Commit() testCases := []struct { @@ -156,7 +150,7 @@ func TestBaseAppCreateQueryContext(t *testing.T) { for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { - _, err := app.CreateQueryContext(tc.height, tc.prove) + _, err := app.createQueryContext(tc.height, tc.prove) if tc.expErr { require.Error(t, err) } else { diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index 2dfa029b7c2f..9f62cb5e6120 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -1,10 +1,12 @@ package baseapp import ( - "context" "fmt" + "strings" + "github.com/gogo/protobuf/proto" abci "github.com/tendermint/tendermint/abci/types" + "github.com/tendermint/tendermint/crypto/tmhash" "github.com/tendermint/tendermint/libs/log" tmproto "github.com/tendermint/tendermint/proto/tendermint/types" dbm "github.com/tendermint/tm-db" @@ -15,7 +17,8 @@ import ( "github.com/cosmos/cosmos-sdk/store/rootmulti" storetypes "github.com/cosmos/cosmos-sdk/store/types" sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/types/tx" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" + "github.com/cosmos/cosmos-sdk/x/auth/migrations/legacytx" ) const ( @@ -48,11 +51,14 @@ type BaseApp struct { // nolint: maligned db dbm.DB // common DB backend cms sdk.CommitMultiStore // Main (uncached) state storeLoader StoreLoader // function to handle store loading, may be overridden with SetStoreLoader() + router sdk.Router // handle any kind of message queryRouter sdk.QueryRouter // router for redirecting query calls grpcQueryRouter *GRPCQueryRouter // router for redirecting gRPC query calls + msgServiceRouter *MsgServiceRouter // router for redirecting Msg service messages interfaceRegistry types.InterfaceRegistry + txDecoder sdk.TxDecoder // unmarshal []byte into sdk.Tx - txHandler tx.Handler // txHandler for {Deliver,Check}Tx and simulations + anteHandler sdk.AnteHandler // ante handler for fee and auth initChainer sdk.InitChainer // initialize state with validators and state blob beginBlocker sdk.BeginBlocker // logic to run before any txs endBlocker sdk.EndBlocker // logic to run after all txs, and to determine valset changes @@ -115,6 +121,9 @@ type BaseApp struct { // nolint: maligned // if BaseApp is passed to the upgrade keeper's NewKeeper method. appVersion uint64 + // recovery handler for app.runTx method + runTxRecoveryMiddleware recoveryMiddleware + // trace set will return full stack traces for errors in ABCI Log field trace bool @@ -133,17 +142,20 @@ type BaseApp struct { // nolint: maligned // // NOTE: The db is used to store the version number for now. func NewBaseApp( - name string, logger log.Logger, db dbm.DB, options ...func(*BaseApp), + name string, logger log.Logger, db dbm.DB, txDecoder sdk.TxDecoder, options ...func(*BaseApp), ) *BaseApp { app := &BaseApp{ - logger: logger, - name: name, - db: db, - cms: store.NewCommitMultiStore(db), - storeLoader: DefaultStoreLoader, - queryRouter: NewQueryRouter(), - grpcQueryRouter: NewGRPCQueryRouter(), - fauxMerkleMode: false, + logger: logger, + name: name, + db: db, + cms: store.NewCommitMultiStore(db), + storeLoader: DefaultStoreLoader, + router: NewRouter(), + queryRouter: NewQueryRouter(), + grpcQueryRouter: NewGRPCQueryRouter(), + msgServiceRouter: NewMsgServiceRouter(), + txDecoder: txDecoder, + fauxMerkleMode: false, } for _, option := range options { @@ -154,6 +166,8 @@ func NewBaseApp( app.cms.SetInterBlockCache(app.interBlockCache) } + app.runTxRecoveryMiddleware = newDefaultRecoveryMiddleware() + return app } @@ -182,6 +196,9 @@ func (app *BaseApp) Trace() bool { return app.trace } +// MsgServiceRouter returns the MsgServiceRouter of a BaseApp. +func (app *BaseApp) MsgServiceRouter() *MsgServiceRouter { return app.msgServiceRouter } + // MountStores mounts all IAVL or DB stores to the provided keys in the BaseApp // multistore. func (app *BaseApp) MountStores(keys ...storetypes.StoreKey) { @@ -346,6 +363,17 @@ func (app *BaseApp) setIndexEvents(ie []string) { } } +// Router returns the router of the BaseApp. +func (app *BaseApp) Router() sdk.Router { + if app.sealed { + // We cannot return a Router when the app is sealed because we can't have + // any routes modified which would cause unexpected routing behavior. + panic("Router() on sealed BaseApp") + } + + return app.router +} + // QueryRouter returns the QueryRouter of a BaseApp. func (app *BaseApp) QueryRouter() sdk.QueryRouter { return app.queryRouter } @@ -479,6 +507,22 @@ func (app *BaseApp) validateHeight(req abci.RequestBeginBlock) error { return nil } +// validateBasicTxMsgs executes basic validator calls for messages. +func validateBasicTxMsgs(msgs []sdk.Msg) error { + if len(msgs) == 0 { + return sdkerrors.Wrap(sdkerrors.ErrInvalidRequest, "must contain at least one message") + } + + for _, msg := range msgs { + err := msg.ValidateBasic() + if err != nil { + return err + } + } + + return nil +} + // Returns the applications's deliverState if app is in runTxModeDeliver, // otherwise it returns the application's checkstate. func (app *BaseApp) getState(mode runTxMode) *state { @@ -490,7 +534,7 @@ func (app *BaseApp) getState(mode runTxMode) *state { } // retrieve the context for the tx w/ txBytes and other memoized values. -func (app *BaseApp) getContextForTx(mode runTxMode, txBytes []byte) context.Context { +func (app *BaseApp) getContextForTx(mode runTxMode, txBytes []byte) sdk.Context { ctx := app.getState(mode).ctx. WithTxBytes(txBytes). WithVoteInfos(app.voteInfos) @@ -505,5 +549,227 @@ func (app *BaseApp) getContextForTx(mode runTxMode, txBytes []byte) context.Cont ctx, _ = ctx.CacheContext() } - return sdk.WrapSDKContext(ctx) + return ctx +} + +// cacheTxContext returns a new context based off of the provided context with +// a branched multi-store. +func (app *BaseApp) cacheTxContext(ctx sdk.Context, txBytes []byte) (sdk.Context, sdk.CacheMultiStore) { + ms := ctx.MultiStore() + // TODO: https://github.com/cosmos/cosmos-sdk/issues/2824 + msCache := ms.CacheMultiStore() + if msCache.TracingEnabled() { + msCache = msCache.SetTracingContext( + sdk.TraceContext( + map[string]interface{}{ + "txHash": fmt.Sprintf("%X", tmhash.Sum(txBytes)), + }, + ), + ).(sdk.CacheMultiStore) + } + + return ctx.WithMultiStore(msCache), msCache +} + +// runTx processes a transaction within a given execution mode, encoded transaction +// bytes, and the decoded transaction itself. All state transitions occur through +// a cached Context depending on the mode provided. State only gets persisted +// if all messages get executed successfully and the execution mode is DeliverTx. +// Note, gas execution info is always returned. A reference to a Result is +// returned if the tx does not run out of gas and if all the messages are valid +// and execute successfully. An error is returned otherwise. +func (app *BaseApp) runTx(mode runTxMode, txBytes []byte) (gInfo sdk.GasInfo, result *sdk.Result, anteEvents []abci.Event, err error) { + // NOTE: GasWanted should be returned by the AnteHandler. GasUsed is + // determined by the GasMeter. We need access to the context to get the gas + // meter so we initialize upfront. + var gasWanted uint64 + + ctx := app.getContextForTx(mode, txBytes) + ms := ctx.MultiStore() + + // only run the tx if there is block gas remaining + if mode == runTxModeDeliver && ctx.BlockGasMeter().IsOutOfGas() { + return gInfo, nil, nil, sdkerrors.Wrap(sdkerrors.ErrOutOfGas, "no block gas left to run tx") + } + + defer func() { + if r := recover(); r != nil { + recoveryMW := newOutOfGasRecoveryMiddleware(gasWanted, ctx, app.runTxRecoveryMiddleware) + err, result = processRecovery(r, recoveryMW), nil + } + + gInfo = sdk.GasInfo{GasWanted: gasWanted, GasUsed: ctx.GasMeter().GasConsumed()} + }() + + blockGasConsumed := false + // consumeBlockGas makes sure block gas is consumed at most once. It must happen after + // tx processing, and must be execute even if tx processing fails. Hence we use trick with `defer` + consumeBlockGas := func() { + if !blockGasConsumed { + blockGasConsumed = true + ctx.BlockGasMeter().ConsumeGas( + ctx.GasMeter().GasConsumedToLimit(), "block gas meter", + ) + } + } + + // If BlockGasMeter() panics it will be caught by the above recover and will + // return an error - in any case BlockGasMeter will consume gas past the limit. + // + // NOTE: This must exist in a separate defer function for the above recovery + // to recover from this one. + if mode == runTxModeDeliver { + defer consumeBlockGas() + } + + tx, err := app.txDecoder(txBytes) + if err != nil { + return sdk.GasInfo{}, nil, nil, err + } + + msgs := tx.GetMsgs() + if err := validateBasicTxMsgs(msgs); err != nil { + return sdk.GasInfo{}, nil, nil, err + } + + if app.anteHandler != nil { + var ( + anteCtx sdk.Context + msCache sdk.CacheMultiStore + ) + + // Branch context before AnteHandler call in case it aborts. + // This is required for both CheckTx and DeliverTx. + // Ref: https://github.com/cosmos/cosmos-sdk/issues/2772 + // + // NOTE: Alternatively, we could require that AnteHandler ensures that + // writes do not happen if aborted/failed. This may have some + // performance benefits, but it'll be more difficult to get right. + anteCtx, msCache = app.cacheTxContext(ctx, txBytes) + anteCtx = anteCtx.WithEventManager(sdk.NewEventManager()) + newCtx, err := app.anteHandler(anteCtx, tx, mode == runTxModeSimulate) + + if !newCtx.IsZero() { + // At this point, newCtx.MultiStore() is a store branch, or something else + // replaced by the AnteHandler. We want the original multistore. + // + // Also, in the case of the tx aborting, we need to track gas consumed via + // the instantiated gas meter in the AnteHandler, so we update the context + // prior to returning. + ctx = newCtx.WithMultiStore(ms) + } + + events := ctx.EventManager().Events() + + // GasMeter expected to be set in AnteHandler + gasWanted = ctx.GasMeter().Limit() + + if err != nil { + return gInfo, nil, nil, err + } + + msCache.Write() + anteEvents = events.ToABCIEvents() + } + + // Create a new Context based off of the existing Context with a MultiStore branch + // in case message processing fails. At this point, the MultiStore + // is a branch of a branch. + runMsgCtx, msCache := app.cacheTxContext(ctx, txBytes) + + // Attempt to execute all messages and only update state if all messages pass + // and we're in DeliverTx. Note, runMsgs will never return a reference to a + // Result if any single message fails or does not have a registered Handler. + result, err = app.runMsgs(runMsgCtx, msgs, mode) + if err == nil && mode == runTxModeDeliver { + // When block gas exceeds, it'll panic and won't commit the cached store. + consumeBlockGas() + + msCache.Write() + + if len(anteEvents) > 0 { + // append the events in the order of occurrence + result.Events = append(anteEvents, result.Events...) + } + } + + return gInfo, result, anteEvents, err +} + +// runMsgs iterates through a list of messages and executes them with the provided +// Context and execution mode. Messages will only be executed during simulation +// and DeliverTx. An error is returned if any single message fails or if a +// Handler does not exist for a given message route. Otherwise, a reference to a +// Result is returned. The caller must not commit state if an error is returned. +func (app *BaseApp) runMsgs(ctx sdk.Context, msgs []sdk.Msg, mode runTxMode) (*sdk.Result, error) { + msgLogs := make(sdk.ABCIMessageLogs, 0, len(msgs)) + events := sdk.EmptyEvents() + txMsgData := &sdk.TxMsgData{ + Data: make([]*sdk.MsgData, 0, len(msgs)), + } + + // NOTE: GasWanted is determined by the AnteHandler and GasUsed by the GasMeter. + for i, msg := range msgs { + // skip actual execution for (Re)CheckTx mode + if mode == runTxModeCheck || mode == runTxModeReCheck { + break + } + + var ( + msgResult *sdk.Result + eventMsgName string // name to use as value in event `message.action` + err error + ) + + if handler := app.msgServiceRouter.Handler(msg); handler != nil { + // ADR 031 request type routing + msgResult, err = handler(ctx, msg) + eventMsgName = sdk.MsgTypeURL(msg) + } else if legacyMsg, ok := msg.(legacytx.LegacyMsg); ok { + // legacy sdk.Msg routing + // Assuming that the app developer has migrated all their Msgs to + // proto messages and has registered all `Msg services`, then this + // path should never be called, because all those Msgs should be + // registered within the `msgServiceRouter` already. + msgRoute := legacyMsg.Route() + eventMsgName = legacyMsg.Type() + handler := app.router.Route(ctx, msgRoute) + if handler == nil { + return nil, sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "unrecognized message route: %s; message index: %d", msgRoute, i) + } + + msgResult, err = handler(ctx, msg) + } else { + return nil, sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "can't route message %+v", msg) + } + + if err != nil { + return nil, sdkerrors.Wrapf(err, "failed to execute message; message index: %d", i) + } + + msgEvents := sdk.Events{ + sdk.NewEvent(sdk.EventTypeMessage, sdk.NewAttribute(sdk.AttributeKeyAction, eventMsgName)), + } + msgEvents = msgEvents.AppendEvents(msgResult.GetEvents()) + + // append message events, data and logs + // + // Note: Each message result's data must be length-prefixed in order to + // separate each result. + events = events.AppendEvents(msgEvents) + + txMsgData.Data = append(txMsgData.Data, &sdk.MsgData{MsgType: sdk.MsgTypeURL(msg), Data: msgResult.Data}) + msgLogs = append(msgLogs, sdk.NewABCIMessageLog(uint32(i), msgResult.Log, msgEvents)) + } + + data, err := proto.Marshal(txMsgData) + if err != nil { + return nil, sdkerrors.Wrap(err, "failed to marshal tx data") + } + + return &sdk.Result{ + Data: data, + Log: strings.TrimSpace(msgLogs.String()), + Events: events.ToABCIEvents(), + }, nil } diff --git a/baseapp/baseapp_test.go b/baseapp/baseapp_test.go index 2c17071aefe8..beaaafb8641d 100644 --- a/baseapp/baseapp_test.go +++ b/baseapp/baseapp_test.go @@ -1,13 +1,13 @@ -package baseapp_test +package baseapp import ( "bytes" - "context" "encoding/binary" "encoding/json" "fmt" - "math" + "io/ioutil" "math/rand" + "os" "strings" "sync" "testing" @@ -20,60 +20,34 @@ import ( "github.com/tendermint/tendermint/libs/log" tmproto "github.com/tendermint/tendermint/proto/tendermint/types" dbm "github.com/tendermint/tm-db" - "google.golang.org/protobuf/proto" - "github.com/cosmos/cosmos-sdk/baseapp" "github.com/cosmos/cosmos-sdk/codec" - "github.com/cosmos/cosmos-sdk/codec/legacy" - codectypes "github.com/cosmos/cosmos-sdk/codec/types" - pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types" - "github.com/cosmos/cosmos-sdk/simapp" "github.com/cosmos/cosmos-sdk/snapshots" snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" "github.com/cosmos/cosmos-sdk/store/rootmulti" - storetypes "github.com/cosmos/cosmos-sdk/store/types" - "github.com/cosmos/cosmos-sdk/testutil" + store "github.com/cosmos/cosmos-sdk/store/types" "github.com/cosmos/cosmos-sdk/testutil/testdata" - "github.com/cosmos/cosmos-sdk/testutil/testdata_pulsar" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" - "github.com/cosmos/cosmos-sdk/x/auth/migrations/legacytx" + "github.com/cosmos/cosmos-sdk/x/auth/legacy/legacytx" ) var ( capKey1 = sdk.NewKVStoreKey("key1") capKey2 = sdk.NewKVStoreKey("key2") - - encCfg = simapp.MakeTestEncodingConfig() ) -func init() { - registerTestCodec(encCfg.Amino) -} - type paramStore struct { db *dbm.MemDB } -type setupConfig struct { - blocks uint64 - blockTxs int - snapshotInterval uint64 - snapshotKeepRecent uint32 - pruningOpts pruningtypes.PruningOptions -} - func (ps *paramStore) Set(_ sdk.Context, key []byte, value interface{}) { bz, err := json.Marshal(value) if err != nil { panic(err) } - if err := ps.db.Set(key, bz); err != nil { - panic(err) - } + ps.db.Set(key, bz) } func (ps *paramStore) Has(_ sdk.Context, key []byte) bool { @@ -101,34 +75,39 @@ func (ps *paramStore) Get(_ sdk.Context, key []byte, ptr interface{}) { } func defaultLogger() log.Logger { - logger, _ := log.NewDefaultLogger("plain", "info", false) - return logger.With("module", "sdk/app") + return log.MustNewDefaultLogger("plain", "info", false).With("module", "sdk/app") } -func newBaseApp(name string, options ...func(*baseapp.BaseApp)) *baseapp.BaseApp { +func newBaseApp(name string, options ...func(*BaseApp)) *BaseApp { logger := defaultLogger() db := dbm.NewMemDB() codec := codec.NewLegacyAmino() registerTestCodec(codec) - return baseapp.NewBaseApp(name, logger, db, options...) + return NewBaseApp(name, logger, db, testTxDecoder(codec), options...) } func registerTestCodec(cdc *codec.LegacyAmino) { + // register Tx, Msg + sdk.RegisterLegacyAminoCodec(cdc) + // register test types cdc.RegisterConcrete(&txTest{}, "cosmos-sdk/baseapp/txTest", nil) - legacy.RegisterAminoMsg(cdc, &msgCounter{}, "cosmos-sdk/baseapp/msgCounter") - legacy.RegisterAminoMsg(cdc, &msgCounter2{}, "cosmos-sdk/baseapp/msgCounter2") - legacy.RegisterAminoMsg(cdc, &msgKeyValue{}, "cosmos-sdk/baseapp/msgKeyValue") - legacy.RegisterAminoMsg(cdc, &msgNoRoute{}, "cosmos-sdk/baseapp/msgNoRoute") + cdc.RegisterConcrete(&msgCounter{}, "cosmos-sdk/baseapp/msgCounter", nil) + cdc.RegisterConcrete(&msgCounter2{}, "cosmos-sdk/baseapp/msgCounter2", nil) + cdc.RegisterConcrete(&msgKeyValue{}, "cosmos-sdk/baseapp/msgKeyValue", nil) + cdc.RegisterConcrete(&msgNoRoute{}, "cosmos-sdk/baseapp/msgNoRoute", nil) } // aminoTxEncoder creates a amino TxEncoder for testing purposes. -func aminoTxEncoder(cdc *codec.LegacyAmino) sdk.TxEncoder { +func aminoTxEncoder() sdk.TxEncoder { + cdc := codec.NewLegacyAmino() + registerTestCodec(cdc) + return legacytx.StdTxConfig{Cdc: cdc}.TxEncoder() } // simple one store baseapp -func setupBaseApp(t *testing.T, options ...func(*baseapp.BaseApp)) (*baseapp.BaseApp, error) { +func setupBaseApp(t *testing.T, options ...func(*BaseApp)) *BaseApp { app := newBaseApp(t.Name(), options...) require.Equal(t, t.Name(), app.Name()) @@ -137,69 +116,45 @@ func setupBaseApp(t *testing.T, options ...func(*baseapp.BaseApp)) (*baseapp.Bas // stores are mounted err := app.LoadLatestVersion() - return app, err -} - -// testTxHandler is a tx.Handler used for the mock app, it does not -// contain any signature verification logic. -func testTxHandler(options middleware.TxHandlerOptions, customTxHandlerMiddleware handlerFun) tx.Handler { - return middleware.ComposeMiddlewares( - middleware.NewRunMsgsTxHandler(options.MsgServiceRouter, options.LegacyRouter), - middleware.NewTxDecoderMiddleware(options.TxDecoder), - middleware.GasTxMiddleware, - middleware.RecoveryTxMiddleware, - middleware.NewIndexEventsTxMiddleware(options.IndexEvents), - middleware.ValidateBasicMiddleware, - middleware.ConsumeBlockGasMiddleware, - CustomTxHandlerMiddleware(customTxHandlerMiddleware), - ) + require.Nil(t, err) + return app } // simple one store baseapp with data and snapshots. Each tx is 1 MB in size (uncompressed). -func setupBaseAppWithSnapshots(t *testing.T, config *setupConfig) (*baseapp.BaseApp, error) { +func setupBaseAppWithSnapshots(t *testing.T, blocks uint, blockTxs int, options ...func(*BaseApp)) (*BaseApp, func()) { codec := codec.NewLegacyAmino() registerTestCodec(codec) - routerOpt := func(bapp *baseapp.BaseApp) { - legacyRouter := middleware.NewLegacyRouter() - legacyRouter.AddRoute(sdk.NewRoute(routeMsgKeyValue, func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { + routerOpt := func(bapp *BaseApp) { + bapp.Router().AddRoute(sdk.NewRoute(routeMsgKeyValue, func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { kv := msg.(*msgKeyValue) - bapp.CMS().GetCommitKVStore(capKey2).Set(kv.Key, kv.Value) - any, err := codectypes.NewAnyWithValue(msg) - if err != nil { - return nil, err - } - - return &sdk.Result{ - MsgResponses: []*codectypes.Any{any}, - }, nil + bapp.cms.GetCommitKVStore(capKey2).Set(kv.Key, kv.Value) + return &sdk.Result{}, nil })) - txHandler := testTxHandler( - middleware.TxHandlerOptions{ - LegacyRouter: legacyRouter, - MsgServiceRouter: middleware.NewMsgServiceRouter(encCfg.InterfaceRegistry), - TxDecoder: testTxDecoder(encCfg.Amino), - }, - func(ctx sdk.Context, tx sdk.Tx, simulate bool) (sdk.Context, error) { return ctx, nil }, - ) - bapp.SetTxHandler(txHandler) } + snapshotInterval := uint64(2) snapshotTimeout := 1 * time.Minute - snapshotStore, err := snapshots.NewStore(dbm.NewMemDB(), testutil.GetTempDir(t)) + snapshotDir, err := ioutil.TempDir("", "baseapp") require.NoError(t, err) - - app, err := setupBaseApp(t, routerOpt, baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(config.snapshotInterval, uint32(config.snapshotKeepRecent))), baseapp.SetPruning(config.pruningOpts)) - if err != nil { - return nil, err + snapshotStore, err := snapshots.NewStore(dbm.NewMemDB(), snapshotDir) + require.NoError(t, err) + teardown := func() { + os.RemoveAll(snapshotDir) } + app := setupBaseApp(t, append(options, + SetSnapshotStore(snapshotStore), + SetSnapshotInterval(snapshotInterval), + SetPruning(sdk.PruningOptions{KeepEvery: 1}), + routerOpt)...) + app.InitChain(abci.RequestInitChain{}) r := rand.New(rand.NewSource(3920758213583)) keyCounter := 0 - for height := int64(1); height <= int64(config.blocks); height++ { + for height := int64(1); height <= int64(blocks); height++ { app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: height}}) - for txNum := 0; txNum < config.blockTxs; txNum++ { + for txNum := 0; txNum < blockTxs; txNum++ { tx := txTest{Msgs: []sdk.Msg{}} for msgNum := 0; msgNum < 100; msgNum++ { key := []byte(fmt.Sprintf("%v", keyCounter)) @@ -209,7 +164,7 @@ func setupBaseAppWithSnapshots(t *testing.T, config *setupConfig) (*baseapp.Base tx.Msgs = append(tx.Msgs, msgKeyValue{Key: key, Value: value}) keyCounter++ } - txBytes, err := encCfg.Amino.Marshal(tx) + txBytes, err := codec.Marshal(tx) require.NoError(t, err) resp := app.DeliverTx(abci.RequestDeliverTx{Tx: txBytes}) require.True(t, resp.IsOK(), "%v", resp.String()) @@ -218,7 +173,7 @@ func setupBaseAppWithSnapshots(t *testing.T, config *setupConfig) (*baseapp.Base app.Commit() // Wait for snapshot to be taken, since it happens asynchronously. - if config.snapshotInterval > 0 && uint64(height)%config.snapshotInterval == 0 { + if uint64(height)%snapshotInterval == 0 { start := time.Now() for { if time.Since(start) > snapshotTimeout { @@ -234,81 +189,33 @@ func setupBaseAppWithSnapshots(t *testing.T, config *setupConfig) (*baseapp.Base } } - return app, nil + return app, teardown } func TestMountStores(t *testing.T) { - app, err := setupBaseApp(t) - require.NoError(t, err) + app := setupBaseApp(t) // check both stores - store1 := app.CMS().GetCommitKVStore(capKey1) + store1 := app.cms.GetCommitKVStore(capKey1) require.NotNil(t, store1) - store2 := app.CMS().GetCommitKVStore(capKey2) + store2 := app.cms.GetCommitKVStore(capKey2) require.NotNil(t, store2) } -type MockTxHandler struct { - T *testing.T -} - -func (th MockTxHandler) CheckTx(goCtx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - ctx := sdk.UnwrapSDKContext(goCtx) - require.NotNil(th.T, ctx.ConsensusParams()) - return tx.Response{}, tx.ResponseCheckTx{}, nil -} - -func (th MockTxHandler) DeliverTx(goCtx context.Context, req tx.Request) (tx.Response, error) { - ctx := sdk.UnwrapSDKContext(goCtx) - require.NotNil(th.T, ctx.ConsensusParams()) - return tx.Response{}, nil -} - -func (th MockTxHandler) SimulateTx(goCtx context.Context, req tx.Request) (tx.Response, error) { - ctx := sdk.UnwrapSDKContext(goCtx) - require.NotNil(th.T, ctx.ConsensusParams()) - return tx.Response{}, nil -} - -func TestConsensusParamsNotNil(t *testing.T) { - app, err := setupBaseApp(t, func(app *baseapp.BaseApp) { - app.SetBeginBlocker(func(ctx sdk.Context, req abci.RequestBeginBlock) abci.ResponseBeginBlock { - require.NotNil(t, ctx.ConsensusParams()) - return abci.ResponseBeginBlock{} - }) - }, func(app *baseapp.BaseApp) { - app.SetEndBlocker(func(ctx sdk.Context, req abci.RequestEndBlock) abci.ResponseEndBlock { - require.NotNil(t, ctx.ConsensusParams()) - return abci.ResponseEndBlock{} - }) - }, func(app *baseapp.BaseApp) { - app.SetTxHandler(MockTxHandler{T: t}) - }) - require.NoError(t, err) - - header := tmproto.Header{Height: 1} - app.BeginBlock(abci.RequestBeginBlock{Header: header}) - app.EndBlock(abci.RequestEndBlock{Height: header.Height}) - app.CheckTx(abci.RequestCheckTx{}) - app.DeliverTx(abci.RequestDeliverTx{}) - _, _, err = app.Simulate([]byte{}) - require.NoError(t, err) -} - // Test that we can make commits and then reload old versions. // Test that LoadLatestVersion actually does. func TestLoadVersion(t *testing.T) { logger := defaultLogger() - pruningOpt := baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)) + pruningOpt := SetPruning(store.PruneNothing) db := dbm.NewMemDB() name := t.Name() - app := baseapp.NewBaseApp(name, logger, db, pruningOpt) + app := NewBaseApp(name, logger, db, nil, pruningOpt) // make a cap key and mount the store err := app.LoadLatestVersion() // needed to make stores non-nil require.Nil(t, err) - emptyCommitID := storetypes.CommitID{} + emptyCommitID := sdk.CommitID{} // fresh store has zero/empty last commit lastHeight := app.LastBlockHeight() @@ -320,16 +227,16 @@ func TestLoadVersion(t *testing.T) { header := tmproto.Header{Height: 1} app.BeginBlock(abci.RequestBeginBlock{Header: header}) res := app.Commit() - commitID1 := storetypes.CommitID{Version: 1, Hash: res.Data} + commitID1 := sdk.CommitID{Version: 1, Hash: res.Data} // execute a block, collect commit ID header = tmproto.Header{Height: 2} app.BeginBlock(abci.RequestBeginBlock{Header: header}) res = app.Commit() - commitID2 := storetypes.CommitID{Version: 2, Hash: res.Data} + commitID2 := sdk.CommitID{Version: 2, Hash: res.Data} // reload with LoadLatestVersion - app = baseapp.NewBaseApp(name, logger, db, pruningOpt) + app = NewBaseApp(name, logger, db, nil, pruningOpt) app.MountStores() err = app.LoadLatestVersion() require.Nil(t, err) @@ -337,7 +244,7 @@ func TestLoadVersion(t *testing.T) { // reload with LoadVersion, see if you can commit the same block and get // the same result - app = baseapp.NewBaseApp(name, logger, db, pruningOpt) + app = NewBaseApp(name, logger, db, nil, pruningOpt) err = app.LoadVersion(1) require.Nil(t, err) testLoadVersionHelper(t, app, int64(1), commitID1) @@ -346,21 +253,21 @@ func TestLoadVersion(t *testing.T) { testLoadVersionHelper(t, app, int64(2), commitID2) } -func useDefaultLoader(app *baseapp.BaseApp) { - app.SetStoreLoader(baseapp.DefaultStoreLoader) +func useDefaultLoader(app *BaseApp) { + app.SetStoreLoader(DefaultStoreLoader) } func initStore(t *testing.T, db dbm.DB, storeKey string, k, v []byte) { - rs := rootmulti.NewStore(db, log.NewNopLogger()) - rs.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)) + rs := rootmulti.NewStore(db) + rs.SetPruning(store.PruneNothing) key := sdk.NewKVStoreKey(storeKey) - rs.MountStoreWithDB(key, storetypes.StoreTypeIAVL, nil) + rs.MountStoreWithDB(key, store.StoreTypeIAVL, nil) err := rs.LoadLatestVersion() require.Nil(t, err) require.Equal(t, int64(0), rs.LastCommitID().Version) // write some data in substore - kv, _ := rs.GetStore(key).(storetypes.KVStore) + kv, _ := rs.GetStore(key).(store.KVStore) require.NotNil(t, kv) kv.Set(k, v) commitID := rs.Commit() @@ -368,16 +275,16 @@ func initStore(t *testing.T, db dbm.DB, storeKey string, k, v []byte) { } func checkStore(t *testing.T, db dbm.DB, ver int64, storeKey string, k, v []byte) { - rs := rootmulti.NewStore(db, log.NewNopLogger()) - rs.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningDefault)) + rs := rootmulti.NewStore(db) + rs.SetPruning(store.PruneDefault) key := sdk.NewKVStoreKey(storeKey) - rs.MountStoreWithDB(key, storetypes.StoreTypeIAVL, nil) + rs.MountStoreWithDB(key, store.StoreTypeIAVL, nil) err := rs.LoadLatestVersion() require.Nil(t, err) require.Equal(t, ver, rs.LastCommitID().Version) // query data in substore - kv, _ := rs.GetStore(key).(storetypes.KVStore) + kv, _ := rs.GetStore(key).(store.KVStore) require.NotNil(t, kv) require.Equal(t, v, kv.Get(k)) } @@ -386,7 +293,7 @@ func checkStore(t *testing.T, db dbm.DB, ver int64, storeKey string, k, v []byte // Test that LoadLatestVersion actually does. func TestSetLoader(t *testing.T) { cases := map[string]struct { - setLoader func(*baseapp.BaseApp) + setLoader func(*BaseApp) origStoreKey string loadStoreKey string }{ @@ -412,11 +319,11 @@ func TestSetLoader(t *testing.T) { initStore(t, db, tc.origStoreKey, k, v) // load the app with the existing db - opts := []func(*baseapp.BaseApp){baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))} + opts := []func(*BaseApp){SetPruning(store.PruneNothing)} if tc.setLoader != nil { opts = append(opts, tc.setLoader) } - app := baseapp.NewBaseApp(t.Name(), defaultLogger(), db, opts...) + app := NewBaseApp(t.Name(), defaultLogger(), db, nil, opts...) app.MountStores(sdk.NewKVStoreKey(tc.loadStoreKey)) err := app.LoadLatestVersion() require.Nil(t, err) @@ -435,10 +342,10 @@ func TestSetLoader(t *testing.T) { func TestVersionSetterGetter(t *testing.T) { logger := defaultLogger() - pruningOpt := baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningDefault)) + pruningOpt := SetPruning(store.PruneDefault) db := dbm.NewMemDB() name := t.Name() - app := baseapp.NewBaseApp(name, logger, db, pruningOpt) + app := NewBaseApp(name, logger, db, nil, pruningOpt) require.Equal(t, "", app.Version()) res := app.Query(abci.RequestQuery{Path: "app/version"}) @@ -455,10 +362,10 @@ func TestVersionSetterGetter(t *testing.T) { func TestLoadVersionInvalid(t *testing.T) { logger := log.NewNopLogger() - pruningOpt := baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)) + pruningOpt := SetPruning(store.PruneNothing) db := dbm.NewMemDB() name := t.Name() - app := baseapp.NewBaseApp(name, logger, db, pruningOpt) + app := NewBaseApp(name, logger, db, nil, pruningOpt) err := app.LoadLatestVersion() require.Nil(t, err) @@ -470,10 +377,10 @@ func TestLoadVersionInvalid(t *testing.T) { header := tmproto.Header{Height: 1} app.BeginBlock(abci.RequestBeginBlock{Header: header}) res := app.Commit() - commitID1 := storetypes.CommitID{Version: 1, Hash: res.Data} + commitID1 := sdk.CommitID{Version: 1, Hash: res.Data} // create a new app with the stores mounted under the same cap key - app = baseapp.NewBaseApp(name, logger, db, pruningOpt) + app = NewBaseApp(name, logger, db, nil, pruningOpt) // require we can load the latest version err = app.LoadVersion(1) @@ -487,25 +394,24 @@ func TestLoadVersionInvalid(t *testing.T) { func TestLoadVersionPruning(t *testing.T) { logger := log.NewNopLogger() - pruningOptions := pruningtypes.NewCustomPruningOptions(10, 15) - pruningOpt := baseapp.SetPruning(pruningOptions) + pruningOptions := store.PruningOptions{ + KeepRecent: 2, + KeepEvery: 3, + Interval: 1, + } + pruningOpt := SetPruning(pruningOptions) db := dbm.NewMemDB() name := t.Name() - - snapshotStore, err := snapshots.NewStore(dbm.NewMemDB(), testutil.GetTempDir(t)) - require.NoError(t, err) - snapshotOpt := baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(3, 1)) - - app := baseapp.NewBaseApp(name, logger, db, pruningOpt, snapshotOpt) + app := NewBaseApp(name, logger, db, nil, pruningOpt) // make a cap key and mount the store capKey := sdk.NewKVStoreKey("key1") app.MountStores(capKey) - err = app.LoadLatestVersion() // needed to make stores non-nil + err := app.LoadLatestVersion() // needed to make stores non-nil require.Nil(t, err) - emptyCommitID := storetypes.CommitID{} + emptyCommitID := sdk.CommitID{} // fresh store has zero/empty last commit lastHeight := app.LastBlockHeight() @@ -513,36 +419,36 @@ func TestLoadVersionPruning(t *testing.T) { require.Equal(t, int64(0), lastHeight) require.Equal(t, emptyCommitID, lastID) - var lastCommitID storetypes.CommitID + var lastCommitID sdk.CommitID - // Commit 15 blocks, of which 15 (latest) is kept in addition to 5-14 inclusive - // (keep recent) and 3 (snapshot-interval). - for i := int64(1); i <= 15; i++ { + // Commit seven blocks, of which 7 (latest) is kept in addition to 6, 5 + // (keep recent) and 3 (keep every). + for i := int64(1); i <= 7; i++ { app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: i}}) res := app.Commit() - lastCommitID = storetypes.CommitID{Version: i, Hash: res.Data} + lastCommitID = sdk.CommitID{Version: i, Hash: res.Data} } - for _, v := range []int64{1, 2, 3, 4} { - _, err = app.CMS().CacheMultiStoreWithVersion(v) + for _, v := range []int64{1, 2, 4} { + _, err = app.cms.CacheMultiStoreWithVersion(v) require.NoError(t, err) } - for _, v := range []int64{3, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14} { - _, err = app.CMS().CacheMultiStoreWithVersion(v) + for _, v := range []int64{3, 5, 6, 7} { + _, err = app.cms.CacheMultiStoreWithVersion(v) require.NoError(t, err) } // reload with LoadLatestVersion, check it loads last version - app = baseapp.NewBaseApp(name, logger, db, pruningOpt, snapshotOpt) + app = NewBaseApp(name, logger, db, nil, pruningOpt) app.MountStores(capKey) err = app.LoadLatestVersion() require.Nil(t, err) - testLoadVersionHelper(t, app, int64(15), lastCommitID) + testLoadVersionHelper(t, app, int64(7), lastCommitID) } -func testLoadVersionHelper(t *testing.T, app *baseapp.BaseApp, expectedHeight int64, expectedID storetypes.CommitID) { +func testLoadVersionHelper(t *testing.T, app *BaseApp, expectedHeight int64, expectedID sdk.CommitID) { lastHeight := app.LastBlockHeight() lastID := app.LastCommitID() require.Equal(t, expectedHeight, lastHeight) @@ -552,16 +458,33 @@ func testLoadVersionHelper(t *testing.T, app *baseapp.BaseApp, expectedHeight in func TestOptionFunction(t *testing.T) { logger := defaultLogger() db := dbm.NewMemDB() - bap := baseapp.NewBaseApp("starting name", logger, db, testChangeNameHelper("new name")) - require.Equal(t, bap.GetName(), "new name", "BaseApp should have had name changed via option function") + bap := NewBaseApp("starting name", logger, db, nil, testChangeNameHelper("new name")) + require.Equal(t, bap.name, "new name", "BaseApp should have had name changed via option function") } -func testChangeNameHelper(name string) func(*baseapp.BaseApp) { - return func(bap *baseapp.BaseApp) { - bap.SetName(name) +func testChangeNameHelper(name string) func(*BaseApp) { + return func(bap *BaseApp) { + bap.name = name } } +// Test that txs can be unmarshalled and read and that +// correct error codes are returned when not +func TestTxDecoder(t *testing.T) { + codec := codec.NewLegacyAmino() + registerTestCodec(codec) + + app := newBaseApp(t.Name()) + tx := newTxCounter(1, 0) + txBytes := codec.MustMarshal(tx) + + dTx, err := app.txDecoder(txBytes) + require.NoError(t, err) + + cTx := dTx.(txTest) + require.Equal(t, tx.Counter, cTx.Counter) +} + // Test that Info returns the latest committed state. func TestInfo(t *testing.T) { app := newBaseApp(t.Name()) @@ -581,8 +504,7 @@ func TestInfo(t *testing.T) { } func TestBaseAppOptionSeal(t *testing.T) { - app, err := setupBaseApp(t) - require.NoError(t, err) + app := setupBaseApp(t) require.Panics(t, func() { app.SetName("") @@ -606,7 +528,7 @@ func TestBaseAppOptionSeal(t *testing.T) { app.SetEndBlocker(nil) }) require.Panics(t, func() { - app.SetTxHandler(nil) + app.SetAnteHandler(nil) }) require.Panics(t, func() { app.SetAddrPeerFilter(nil) @@ -617,12 +539,15 @@ func TestBaseAppOptionSeal(t *testing.T) { require.Panics(t, func() { app.SetFauxMerkleMode() }) + require.Panics(t, func() { + app.SetRouter(NewRouter()) + }) } func TestSetMinGasPrices(t *testing.T) { minGasPrices := sdk.DecCoins{sdk.NewInt64DecCoin("stake", 5000)} - app := newBaseApp(t.Name(), baseapp.SetMinGasPrices(minGasPrices.String())) - require.Equal(t, minGasPrices, app.MinGasPrices()) + app := newBaseApp(t.Name(), SetMinGasPrices(minGasPrices.String())) + require.Equal(t, minGasPrices, app.minGasPrices) } func TestInitChainer(t *testing.T) { @@ -631,7 +556,7 @@ func TestInitChainer(t *testing.T) { // we can reload the same app later db := dbm.NewMemDB() logger := defaultLogger() - app := baseapp.NewBaseApp(name, logger, db) + app := NewBaseApp(name, logger, db, nil) capKey := sdk.NewKVStoreKey("main") capKey2 := sdk.NewKVStoreKey("key2") app.MountStores(capKey, capKey2) @@ -674,10 +599,10 @@ func TestInitChainer(t *testing.T) { ) // assert that chainID is set correctly in InitChain - chainID := app.DeliverState().Context().ChainID() + chainID := app.deliverState.ctx.ChainID() require.Equal(t, "test-chain-id", chainID, "ChainID in deliverState not set correctly in InitChain") - chainID = app.CheckState().Context().ChainID() + chainID = app.checkState.ctx.ChainID() require.Equal(t, "test-chain-id", chainID, "ChainID in checkState not set correctly in InitChain") app.Commit() @@ -686,7 +611,7 @@ func TestInitChainer(t *testing.T) { require.Equal(t, value, res.Value) // reload app - app = baseapp.NewBaseApp(name, logger, db) + app = NewBaseApp(name, logger, db, nil) app.SetInitChainer(initChainer) app.MountStores(capKey, capKey2) err = app.LoadLatestVersion() // needed to make stores non-nil @@ -710,7 +635,7 @@ func TestInitChain_WithInitialHeight(t *testing.T) { name := t.Name() db := dbm.NewMemDB() logger := defaultLogger() - app := baseapp.NewBaseApp(name, logger, db) + app := NewBaseApp(name, logger, db, nil) app.InitChain( abci.RequestInitChain{ @@ -726,7 +651,7 @@ func TestBeginBlock_WithInitialHeight(t *testing.T) { name := t.Name() db := dbm.NewMemDB() logger := defaultLogger() - app := baseapp.NewBaseApp(name, logger, db) + app := NewBaseApp(name, logger, db, nil) app.InitChain( abci.RequestInitChain{ @@ -757,7 +682,6 @@ type txTest struct { Msgs []sdk.Msg Counter int64 FailOnAnte bool - GasLimit uint64 } func (tx *txTest) setFailOnAnte(fail bool) { @@ -766,7 +690,7 @@ func (tx *txTest) setFailOnAnte(fail bool) { func (tx *txTest) setFailOnHandler(fail bool) { for i, msg := range tx.Msgs { - tx.Msgs[i] = &msgCounter{msg.(*msgCounter).Counter, fail} + tx.Msgs[i] = msgCounter{msg.(msgCounter).Counter, fail} } } @@ -774,12 +698,6 @@ func (tx *txTest) setFailOnHandler(fail bool) { func (tx txTest) GetMsgs() []sdk.Msg { return tx.Msgs } func (tx txTest) ValidateBasic() error { return nil } -// Implements GasTx -func (tx txTest) GetGas() uint64 { return tx.GasLimit } - -// Implements TxWithTimeoutHeight -func (tx txTest) GetTimeoutHeight() uint64 { return 0 } - const ( routeMsgCounter = "msgCounter" routeMsgCounter2 = "msgCounter2" @@ -794,29 +712,29 @@ type msgCounter struct { } // dummy implementation of proto.Message -func (msg *msgCounter) Reset() {} -func (msg *msgCounter) String() string { return "TODO" } -func (msg *msgCounter) ProtoMessage() {} +func (msg msgCounter) Reset() {} +func (msg msgCounter) String() string { return "TODO" } +func (msg msgCounter) ProtoMessage() {} // Implements Msg -func (msg *msgCounter) Route() string { return routeMsgCounter } -func (msg *msgCounter) Type() string { return "counter1" } -func (msg *msgCounter) GetSignBytes() []byte { return nil } -func (msg *msgCounter) GetSigners() []sdk.AccAddress { return nil } -func (msg *msgCounter) ValidateBasic() error { +func (msg msgCounter) Route() string { return routeMsgCounter } +func (msg msgCounter) Type() string { return "counter1" } +func (msg msgCounter) GetSignBytes() []byte { return nil } +func (msg msgCounter) GetSigners() []sdk.AccAddress { return nil } +func (msg msgCounter) ValidateBasic() error { if msg.Counter >= 0 { return nil } return sdkerrors.Wrap(sdkerrors.ErrInvalidSequence, "counter should be a non-negative integer") } -func newTxCounter(counter int64, msgCounters ...int64) txTest { +func newTxCounter(counter int64, msgCounters ...int64) *txTest { msgs := make([]sdk.Msg, 0, len(msgCounters)) for _, c := range msgCounters { - msgs = append(msgs, &msgCounter{c, false}) + msgs = append(msgs, msgCounter{c, false}) } - return txTest{msgs, counter, false, math.MaxUint64} + return &txTest{msgs, counter, false} } // a msg we dont know how to route @@ -895,7 +813,7 @@ func testTxDecoder(cdc *codec.LegacyAmino) sdk.TxDecoder { } } -func customHandlerTxTest(t *testing.T, capKey storetypes.StoreKey, storeKey []byte) handlerFun { +func anteHandlerTxTest(t *testing.T, capKey sdk.StoreKey, storeKey []byte) sdk.AnteHandler { return func(ctx sdk.Context, tx sdk.Tx, simulate bool) (sdk.Context, error) { store := ctx.KVStore(capKey) txTest := tx.(txTest) @@ -910,7 +828,7 @@ func customHandlerTxTest(t *testing.T, capKey storetypes.StoreKey, storeKey []by } ctx.EventManager().EmitEvents( - counterEvent("post_handlers", txTest.Counter), + counterEvent("ante_handler", txTest.Counter), ) return ctx, nil @@ -926,7 +844,7 @@ func counterEvent(evType string, msgCount int64) sdk.Events { } } -func handlerMsgCounter(t *testing.T, capKey storetypes.StoreKey, deliverKey []byte) sdk.Handler { +func handlerMsgCounter(t *testing.T, capKey sdk.StoreKey, deliverKey []byte) sdk.Handler { return func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { ctx = ctx.WithEventManager(sdk.NewEventManager()) store := ctx.KVStore(capKey) @@ -953,14 +871,6 @@ func handlerMsgCounter(t *testing.T, capKey storetypes.StoreKey, deliverKey []by } res.Events = ctx.EventManager().Events().ToABCIEvents() - - any, err := codectypes.NewAnyWithValue(msg) - if err != nil { - return nil, err - } - - res.MsgResponses = []*codectypes.Any{any} - return res, nil } } @@ -992,7 +902,7 @@ func incrementingCounter(t *testing.T, store sdk.KVStore, counterKey []byte, cou return &sdk.Result{}, nil } -// --------------------------------------------------------------------- +//--------------------------------------------------------------------- // Tx processing - CheckTx, DeliverTx, SimulateTx. // These tests use the serialized tx as input, while most others will use the // Check(), Deliver(), Simulate() methods directly. @@ -1006,55 +916,49 @@ func TestCheckTx(t *testing.T) { // This ensures changes to the kvstore persist across successive CheckTx. counterKey := []byte("counter-key") - txHandlerOpt := func(bapp *baseapp.BaseApp) { - legacyRouter := middleware.NewLegacyRouter() + anteOpt := func(bapp *BaseApp) { bapp.SetAnteHandler(anteHandlerTxTest(t, capKey1, counterKey)) } + routerOpt := func(bapp *BaseApp) { // TODO: can remove this once CheckTx doesnt process msgs. - legacyRouter.AddRoute(sdk.NewRoute(routeMsgCounter, func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { + bapp.Router().AddRoute(sdk.NewRoute(routeMsgCounter, func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { return &sdk.Result{}, nil })) - txHandler := testTxHandler( - middleware.TxHandlerOptions{ - LegacyRouter: legacyRouter, - MsgServiceRouter: middleware.NewMsgServiceRouter(encCfg.InterfaceRegistry), - TxDecoder: testTxDecoder(encCfg.Amino), - }, - customHandlerTxTest(t, capKey1, counterKey), - ) - bapp.SetTxHandler(txHandler) } - app, err := setupBaseApp(t, txHandlerOpt) - require.NoError(t, err) + app := setupBaseApp(t, anteOpt, routerOpt) nTxs := int64(5) app.InitChain(abci.RequestInitChain{}) + // Create same codec used in txDecoder + codec := codec.NewLegacyAmino() + registerTestCodec(codec) + for i := int64(0); i < nTxs; i++ { tx := newTxCounter(i, 0) // no messages - txBytes, err := encCfg.Amino.Marshal(tx) + txBytes, err := codec.Marshal(tx) require.NoError(t, err) r := app.CheckTx(abci.RequestCheckTx{Tx: txBytes}) require.Empty(t, r.GetEvents()) - require.True(t, r.IsOK(), fmt.Sprintf("%+v", r)) + require.True(t, r.IsOK(), fmt.Sprintf("%v", r)) } - checkStateStore := app.CheckState().Context().KVStore(capKey1) + checkStateStore := app.checkState.ctx.KVStore(capKey1) storedCounter := getIntFromStore(checkStateStore, counterKey) - // Ensure storedCounter + // Ensure AnteHandler ran require.Equal(t, nTxs, storedCounter) // If a block is committed, CheckTx state should be reset. header := tmproto.Header{Height: 1} app.BeginBlock(abci.RequestBeginBlock{Header: header, Hash: []byte("hash")}) - require.NotNil(t, app.CheckState().Context().BlockGasMeter(), "block gas meter should have been set to checkState") - require.NotEmpty(t, app.CheckState().Context().HeaderHash()) + require.NotNil(t, app.checkState.ctx.BlockGasMeter(), "block gas meter should have been set to checkState") + require.NotEmpty(t, app.checkState.ctx.HeaderHash()) app.EndBlock(abci.RequestEndBlock{}) app.Commit() - checkStateStore = app.CheckState().Context().KVStore(capKey1) + checkStateStore = app.checkState.ctx.KVStore(capKey1) storedBytes := checkStateStore.Get(counterKey) require.Nil(t, storedBytes) } @@ -1062,30 +966,24 @@ func TestCheckTx(t *testing.T) { // Test that successive DeliverTx can see each others' effects // on the store, both within and across blocks. func TestDeliverTx(t *testing.T) { - // test increments in the post txHandler + // test increments in the ante anteKey := []byte("ante-key") + anteOpt := func(bapp *BaseApp) { bapp.SetAnteHandler(anteHandlerTxTest(t, capKey1, anteKey)) } + // test increments in the handler deliverKey := []byte("deliver-key") - - txHandlerOpt := func(bapp *baseapp.BaseApp) { - legacyRouter := middleware.NewLegacyRouter() + routerOpt := func(bapp *BaseApp) { r := sdk.NewRoute(routeMsgCounter, handlerMsgCounter(t, capKey1, deliverKey)) - legacyRouter.AddRoute(r) - txHandler := testTxHandler( - middleware.TxHandlerOptions{ - LegacyRouter: legacyRouter, - MsgServiceRouter: middleware.NewMsgServiceRouter(encCfg.InterfaceRegistry), - TxDecoder: testTxDecoder(encCfg.Amino), - }, - customHandlerTxTest(t, capKey1, anteKey), - ) - bapp.SetTxHandler(txHandler) + bapp.Router().AddRoute(r) } - app, err := setupBaseApp(t, txHandlerOpt) - require.NoError(t, err) + app := setupBaseApp(t, anteOpt, routerOpt) app.InitChain(abci.RequestInitChain{}) + // Create same codec used in txDecoder + codec := codec.NewLegacyAmino() + registerTestCodec(codec) + nBlocks := 3 txPerHeight := 5 @@ -1097,14 +995,14 @@ func TestDeliverTx(t *testing.T) { counter := int64(blockN*txPerHeight + i) tx := newTxCounter(counter, counter) - txBytes, err := encCfg.Amino.Marshal(tx) + txBytes, err := codec.Marshal(tx) require.NoError(t, err) res := app.DeliverTx(abci.RequestDeliverTx{Tx: txBytes}) require.True(t, res.IsOK(), fmt.Sprintf("%v", res)) events := res.GetEvents() require.Len(t, events, 3, "should contain ante handler, message type and counter events respectively") - require.Equal(t, sdk.MarkEventsToIndex(counterEvent("post_handlers", counter).ToABCIEvents(), map[string]struct{}{})[0], events[0], "ante handler event") + require.Equal(t, sdk.MarkEventsToIndex(counterEvent("ante_handler", counter).ToABCIEvents(), map[string]struct{}{})[0], events[0], "ante handler event") require.Equal(t, sdk.MarkEventsToIndex(counterEvent(sdk.EventTypeMessage, counter).ToABCIEvents(), map[string]struct{}{})[0], events[2], "msg handler update counter event") } @@ -1123,29 +1021,23 @@ func TestMultiMsgCheckTx(t *testing.T) { func TestMultiMsgDeliverTx(t *testing.T) { // increment the tx counter anteKey := []byte("ante-key") + anteOpt := func(bapp *BaseApp) { bapp.SetAnteHandler(anteHandlerTxTest(t, capKey1, anteKey)) } + // increment the msg counter deliverKey := []byte("deliver-key") deliverKey2 := []byte("deliver-key2") - - txHandlerOpt := func(bapp *baseapp.BaseApp) { - legacyRouter := middleware.NewLegacyRouter() + routerOpt := func(bapp *BaseApp) { r1 := sdk.NewRoute(routeMsgCounter, handlerMsgCounter(t, capKey1, deliverKey)) r2 := sdk.NewRoute(routeMsgCounter2, handlerMsgCounter(t, capKey1, deliverKey2)) - legacyRouter.AddRoute(r1) - legacyRouter.AddRoute(r2) - txHandler := testTxHandler( - middleware.TxHandlerOptions{ - LegacyRouter: legacyRouter, - MsgServiceRouter: middleware.NewMsgServiceRouter(encCfg.InterfaceRegistry), - TxDecoder: testTxDecoder(encCfg.Amino), - }, - customHandlerTxTest(t, capKey1, anteKey), - ) - bapp.SetTxHandler(txHandler) + bapp.Router().AddRoute(r1) + bapp.Router().AddRoute(r2) } - app, err := setupBaseApp(t, txHandlerOpt) - require.NoError(t, err) + app := setupBaseApp(t, anteOpt, routerOpt) + + // Create same codec used in txDecoder + codec := codec.NewLegacyAmino() + registerTestCodec(codec) // run a multi-msg tx // with all msgs the same route @@ -1153,12 +1045,12 @@ func TestMultiMsgDeliverTx(t *testing.T) { header := tmproto.Header{Height: 1} app.BeginBlock(abci.RequestBeginBlock{Header: header}) tx := newTxCounter(0, 0, 1, 2) - txBytes, err := encCfg.Amino.Marshal(tx) + txBytes, err := codec.Marshal(tx) require.NoError(t, err) res := app.DeliverTx(abci.RequestDeliverTx{Tx: txBytes}) require.True(t, res.IsOK(), fmt.Sprintf("%v", res)) - store := app.DeliverState().Context().KVStore(capKey1) + store := app.deliverState.ctx.KVStore(capKey1) // tx counter only incremented once txCounter := getIntFromStore(store, anteKey) @@ -1173,12 +1065,12 @@ func TestMultiMsgDeliverTx(t *testing.T) { tx = newTxCounter(1, 3) tx.Msgs = append(tx.Msgs, msgCounter2{0}) tx.Msgs = append(tx.Msgs, msgCounter2{1}) - txBytes, err = encCfg.Amino.Marshal(tx) + txBytes, err = codec.Marshal(tx) require.NoError(t, err) res = app.DeliverTx(abci.RequestDeliverTx{Tx: txBytes}) require.True(t, res.IsOK(), fmt.Sprintf("%v", res)) - store = app.DeliverState().Context().KVStore(capKey1) + store = app.deliverState.ctx.KVStore(capKey1) // tx counter only incremented once txCounter = getIntFromStore(store, anteKey) @@ -1205,36 +1097,29 @@ func TestConcurrentCheckDeliver(t *testing.T) { func TestSimulateTx(t *testing.T) { gasConsumed := uint64(5) - txHandlerOpt := func(bapp *baseapp.BaseApp) { - legacyRouter := middleware.NewLegacyRouter() + anteOpt := func(bapp *BaseApp) { + bapp.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx, simulate bool) (newCtx sdk.Context, err error) { + newCtx = ctx.WithGasMeter(sdk.NewGasMeter(gasConsumed)) + return + }) + } + + routerOpt := func(bapp *BaseApp) { r := sdk.NewRoute(routeMsgCounter, func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { ctx.GasMeter().ConsumeGas(gasConsumed, "test") - // Return dummy MsgResponse for msgCounter. - any, err := codectypes.NewAnyWithValue(&testdata.Dog{}) - if err != nil { - return nil, err - } - - return &sdk.Result{ - MsgResponses: []*codectypes.Any{any}, - }, nil + return &sdk.Result{}, nil }) - legacyRouter.AddRoute(r) - txHandler := testTxHandler( - middleware.TxHandlerOptions{ - LegacyRouter: legacyRouter, - MsgServiceRouter: middleware.NewMsgServiceRouter(encCfg.InterfaceRegistry), - TxDecoder: testTxDecoder(encCfg.Amino), - }, - func(ctx sdk.Context, tx sdk.Tx, simulate bool) (sdk.Context, error) { return ctx, nil }, - ) - bapp.SetTxHandler(txHandler) + bapp.Router().AddRoute(r) } - app, err := setupBaseApp(t, txHandlerOpt) - require.NoError(t, err) + + app := setupBaseApp(t, anteOpt, routerOpt) app.InitChain(abci.RequestInitChain{}) + // Create same codec used in txDecoder + cdc := codec.NewLegacyAmino() + registerTestCodec(cdc) + nBlocks := 3 for blockN := 0; blockN < nBlocks; blockN++ { count := int64(blockN + 1) @@ -1242,8 +1127,7 @@ func TestSimulateTx(t *testing.T) { app.BeginBlock(abci.RequestBeginBlock{Header: header}) tx := newTxCounter(count, count) - tx.GasLimit = gasConsumed - txBytes, err := encCfg.Amino.Marshal(tx) + txBytes, err := cdc.Marshal(tx) require.Nil(t, err) // simulate a message, check gas reported @@ -1280,33 +1164,19 @@ func TestSimulateTx(t *testing.T) { } func TestRunInvalidTransaction(t *testing.T) { - txHandlerOpt := func(bapp *baseapp.BaseApp) { - legacyRouter := middleware.NewLegacyRouter() + anteOpt := func(bapp *BaseApp) { + bapp.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx, simulate bool) (newCtx sdk.Context, err error) { + return + }) + } + routerOpt := func(bapp *BaseApp) { r := sdk.NewRoute(routeMsgCounter, func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { - any, err := codectypes.NewAnyWithValue(msg) - if err != nil { - return nil, err - } - - return &sdk.Result{ - MsgResponses: []*codectypes.Any{any}, - }, nil + return &sdk.Result{}, nil }) - legacyRouter.AddRoute(r) - txHandler := testTxHandler( - middleware.TxHandlerOptions{ - LegacyRouter: legacyRouter, - MsgServiceRouter: middleware.NewMsgServiceRouter(encCfg.InterfaceRegistry), - TxDecoder: testTxDecoder(encCfg.Amino), - }, - func(ctx sdk.Context, tx sdk.Tx, simulate bool) (newCtx sdk.Context, err error) { - return - }, - ) - bapp.SetTxHandler(txHandler) + bapp.Router().AddRoute(r) } - app, err := setupBaseApp(t, txHandlerOpt) - require.NoError(t, err) + + app := setupBaseApp(t, anteOpt, routerOpt) header := tmproto.Header{Height: 1} app.BeginBlock(abci.RequestBeginBlock{Header: header}) @@ -1314,7 +1184,8 @@ func TestRunInvalidTransaction(t *testing.T) { // transaction with no messages { emptyTx := &txTest{} - _, result, err := app.SimDeliver(aminoTxEncoder(encCfg.Amino), emptyTx) + _, result, err := app.Deliver(aminoTxEncoder(), emptyTx) + require.Error(t, err) require.Nil(t, result) space, code, _ := sdkerrors.ABCIInfo(err, false) @@ -1325,7 +1196,7 @@ func TestRunInvalidTransaction(t *testing.T) { // transaction where ValidateBasic fails { testCases := []struct { - tx txTest + tx *txTest fail bool }{ {newTxCounter(0, 0), false}, @@ -1340,7 +1211,7 @@ func TestRunInvalidTransaction(t *testing.T) { for _, testCase := range testCases { tx := testCase.tx - _, _, err := app.SimDeliver(aminoTxEncoder(encCfg.Amino), tx) + _, result, err := app.Deliver(aminoTxEncoder(), tx) if testCase.fail { require.Error(t, err) @@ -1349,15 +1220,15 @@ func TestRunInvalidTransaction(t *testing.T) { require.EqualValues(t, sdkerrors.ErrInvalidSequence.Codespace(), space, err) require.EqualValues(t, sdkerrors.ErrInvalidSequence.ABCICode(), code, err) } else { - require.NoError(t, err) + require.NotNil(t, result) } } } // transaction with no known route { - unknownRouteTx := txTest{[]sdk.Msg{&msgNoRoute{}}, 0, false, math.MaxUint64} - _, result, err := app.SimDeliver(aminoTxEncoder(encCfg.Amino), unknownRouteTx) + unknownRouteTx := txTest{[]sdk.Msg{msgNoRoute{}}, 0, false} + _, result, err := app.Deliver(aminoTxEncoder(), unknownRouteTx) require.Error(t, err) require.Nil(t, result) @@ -1365,8 +1236,8 @@ func TestRunInvalidTransaction(t *testing.T) { require.EqualValues(t, sdkerrors.ErrUnknownRequest.Codespace(), space, err) require.EqualValues(t, sdkerrors.ErrUnknownRequest.ABCICode(), code, err) - unknownRouteTx = txTest{[]sdk.Msg{&msgCounter{}, &msgNoRoute{}}, 0, false, math.MaxUint64} - _, result, err = app.SimDeliver(aminoTxEncoder(encCfg.Amino), unknownRouteTx) + unknownRouteTx = txTest{[]sdk.Msg{msgCounter{}, msgNoRoute{}}, 0, false} + _, result, err = app.Deliver(aminoTxEncoder(), unknownRouteTx) require.Error(t, err) require.Nil(t, result) @@ -1378,14 +1249,12 @@ func TestRunInvalidTransaction(t *testing.T) { // Transaction with an unregistered message { tx := newTxCounter(0, 0) - tx.Msgs = append(tx.Msgs, &msgNoDecode{}) + tx.Msgs = append(tx.Msgs, msgNoDecode{}) - // new codec so we can encode the tx, but we shouldn't be able to decode, - // because baseapp's codec is not aware of msgNoDecode. + // new codec so we can encode the tx, but we shouldn't be able to decode newCdc := codec.NewLegacyAmino() - sdk.RegisterLegacyAminoCodec(newCdc) // register Tx, Msg registerTestCodec(newCdc) - legacy.RegisterAminoMsg(newCdc, &msgNoDecode{}, "cosmos-sdk/baseapp/msgNoDecode") + newCdc.RegisterConcrete(&msgNoDecode{}, "cosmos-sdk/baseapp/msgNoDecode", nil) txBytes, err := newCdc.Marshal(tx) require.NoError(t, err) @@ -1399,47 +1268,49 @@ func TestRunInvalidTransaction(t *testing.T) { // Test that transactions exceeding gas limits fail func TestTxGasLimits(t *testing.T) { gasGranted := uint64(10) + anteOpt := func(bapp *BaseApp) { + bapp.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx, simulate bool) (newCtx sdk.Context, err error) { + newCtx = ctx.WithGasMeter(sdk.NewGasMeter(gasGranted)) + + // AnteHandlers must have their own defer/recover in order for the BaseApp + // to know how much gas was used! This is because the GasMeter is created in + // the AnteHandler, but if it panics the context won't be set properly in + // runTx's recover call. + defer func() { + if r := recover(); r != nil { + switch rType := r.(type) { + case sdk.ErrorOutOfGas: + err = sdkerrors.Wrapf(sdkerrors.ErrOutOfGas, "out of gas in location: %v", rType.Descriptor) + default: + panic(r) + } + } + }() + + count := tx.(txTest).Counter + newCtx.GasMeter().ConsumeGas(uint64(count), "counter-ante") + + return newCtx, nil + }) - ante := func(ctx sdk.Context, tx sdk.Tx, simulate bool) (sdk.Context, error) { - count := tx.(txTest).Counter - ctx.GasMeter().ConsumeGas(uint64(count), "counter-ante") - return ctx, nil } - txHandlerOpt := func(bapp *baseapp.BaseApp) { - legacyRouter := middleware.NewLegacyRouter() + routerOpt := func(bapp *BaseApp) { r := sdk.NewRoute(routeMsgCounter, func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { count := msg.(*msgCounter).Counter ctx.GasMeter().ConsumeGas(uint64(count), "counter-handler") - any, err := codectypes.NewAnyWithValue(msg) - if err != nil { - return nil, err - } - - return &sdk.Result{ - MsgResponses: []*codectypes.Any{any}, - }, nil + return &sdk.Result{}, nil }) - legacyRouter.AddRoute(r) - txHandler := testTxHandler( - middleware.TxHandlerOptions{ - LegacyRouter: legacyRouter, - MsgServiceRouter: middleware.NewMsgServiceRouter(encCfg.InterfaceRegistry), - TxDecoder: testTxDecoder(encCfg.Amino), - }, - ante, - ) - - bapp.SetTxHandler(txHandler) + bapp.Router().AddRoute(r) } - app, err := setupBaseApp(t, txHandlerOpt) - require.NoError(t, err) + + app := setupBaseApp(t, anteOpt, routerOpt) header := tmproto.Header{Height: 1} app.BeginBlock(abci.RequestBeginBlock{Header: header}) testCases := []struct { - tx txTest + tx *txTest gasUsed uint64 fail bool }{ @@ -1464,8 +1335,7 @@ func TestTxGasLimits(t *testing.T) { for i, tc := range testCases { tx := tc.tx - tx.GasLimit = gasGranted - gInfo, result, err := app.SimDeliver(aminoTxEncoder(encCfg.Amino), tx) + gInfo, result, err := app.Deliver(aminoTxEncoder(), tx) // check gas used and wanted require.Equal(t, tc.gasUsed, gInfo.GasUsed, fmt.Sprintf("tc #%d; gas: %v, result: %v, err: %s", i, gInfo, result, err)) @@ -1487,51 +1357,48 @@ func TestTxGasLimits(t *testing.T) { // Test that transactions exceeding gas limits fail func TestMaxBlockGasLimits(t *testing.T) { gasGranted := uint64(10) - ante := func(ctx sdk.Context, tx sdk.Tx, simulate bool) (sdk.Context, error) { - count := tx.(txTest).Counter - ctx.GasMeter().ConsumeGas(uint64(count), "counter-ante") + anteOpt := func(bapp *BaseApp) { + bapp.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx, simulate bool) (newCtx sdk.Context, err error) { + newCtx = ctx.WithGasMeter(sdk.NewGasMeter(gasGranted)) + + defer func() { + if r := recover(); r != nil { + switch rType := r.(type) { + case sdk.ErrorOutOfGas: + err = sdkerrors.Wrapf(sdkerrors.ErrOutOfGas, "out of gas in location: %v", rType.Descriptor) + default: + panic(r) + } + } + }() - return ctx, nil + count := tx.(txTest).Counter + newCtx.GasMeter().ConsumeGas(uint64(count), "counter-ante") + + return + }) } - txHandlerOpt := func(bapp *baseapp.BaseApp) { - legacyRouter := middleware.NewLegacyRouter() + routerOpt := func(bapp *BaseApp) { r := sdk.NewRoute(routeMsgCounter, func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { count := msg.(*msgCounter).Counter ctx.GasMeter().ConsumeGas(uint64(count), "counter-handler") - - any, err := codectypes.NewAnyWithValue(msg) - if err != nil { - return nil, err - } - - return &sdk.Result{ - MsgResponses: []*codectypes.Any{any}, - }, nil + return &sdk.Result{}, nil }) - legacyRouter.AddRoute(r) - txHandler := testTxHandler( - middleware.TxHandlerOptions{ - LegacyRouter: legacyRouter, - MsgServiceRouter: middleware.NewMsgServiceRouter(encCfg.InterfaceRegistry), - TxDecoder: testTxDecoder(encCfg.Amino), - }, - ante, - ) - bapp.SetTxHandler(txHandler) + bapp.Router().AddRoute(r) } - app, err := setupBaseApp(t, txHandlerOpt) - require.NoError(t, err) + + app := setupBaseApp(t, anteOpt, routerOpt) app.InitChain(abci.RequestInitChain{ - ConsensusParams: &tmproto.ConsensusParams{ - Block: &tmproto.BlockParams{ + ConsensusParams: &abci.ConsensusParams{ + Block: &abci.BlockParams{ MaxGas: 100, }, }, }) testCases := []struct { - tx txTest + tx *txTest numDelivers int gasUsedPerDeliver uint64 fail bool @@ -1551,7 +1418,6 @@ func TestMaxBlockGasLimits(t *testing.T) { for i, tc := range testCases { tx := tc.tx - tx.GasLimit = gasGranted // reset the block gas header := tmproto.Header{Height: app.LastBlockHeight() + 1} @@ -1559,9 +1425,9 @@ func TestMaxBlockGasLimits(t *testing.T) { // execute the transaction multiple times for j := 0; j < tc.numDelivers; j++ { - _, result, err := app.SimDeliver(aminoTxEncoder(encCfg.Amino), tx) + _, result, err := app.Deliver(aminoTxEncoder(), tx) - ctx := app.DeliverState().Context() + ctx := app.getState(runTxModeDeliver).ctx // check for failed transactions if tc.fail && (j+1) > tc.failAfterDeliver { @@ -1588,28 +1454,66 @@ func TestMaxBlockGasLimits(t *testing.T) { } } -func TestBaseAppMiddleware(t *testing.T) { +// Test custom panic handling within app.DeliverTx method +func TestCustomRunTxPanicHandler(t *testing.T) { + const customPanicMsg = "test panic" + anteErr := sdkerrors.Register("fakeModule", 100500, "fakeError") + + anteOpt := func(bapp *BaseApp) { + bapp.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx, simulate bool) (newCtx sdk.Context, err error) { + panic(sdkerrors.Wrap(anteErr, "anteHandler")) + }) + } + routerOpt := func(bapp *BaseApp) { + r := sdk.NewRoute(routeMsgCounter, func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { + return &sdk.Result{}, nil + }) + bapp.Router().AddRoute(r) + } + + app := setupBaseApp(t, anteOpt, routerOpt) + + header := tmproto.Header{Height: 1} + app.BeginBlock(abci.RequestBeginBlock{Header: header}) + + app.AddRunTxRecoveryHandler(func(recoveryObj interface{}) error { + err, ok := recoveryObj.(error) + if !ok { + return nil + } + + if anteErr.Is(err) { + panic(customPanicMsg) + } else { + return nil + } + }) + + // Transaction should panic with custom handler above + { + tx := newTxCounter(0, 0) + + require.PanicsWithValue(t, customPanicMsg, func() { app.Deliver(aminoTxEncoder(), tx) }) + } +} + +func TestBaseAppAnteHandler(t *testing.T) { anteKey := []byte("ante-key") - deliverKey := []byte("deliver-key") + anteOpt := func(bapp *BaseApp) { + bapp.SetAnteHandler(anteHandlerTxTest(t, capKey1, anteKey)) + } - txHandlerOpt := func(bapp *baseapp.BaseApp) { - legacyRouter := middleware.NewLegacyRouter() + deliverKey := []byte("deliver-key") + routerOpt := func(bapp *BaseApp) { r := sdk.NewRoute(routeMsgCounter, handlerMsgCounter(t, capKey1, deliverKey)) - legacyRouter.AddRoute(r) - txHandler := testTxHandler( - middleware.TxHandlerOptions{ - LegacyRouter: legacyRouter, - MsgServiceRouter: middleware.NewMsgServiceRouter(encCfg.InterfaceRegistry), - TxDecoder: testTxDecoder(encCfg.Amino), - }, - customHandlerTxTest(t, capKey1, anteKey), - ) - bapp.SetTxHandler(txHandler) + bapp.Router().AddRoute(r) } - app, err := setupBaseApp(t, txHandlerOpt) - require.NoError(t, err) + + cdc := codec.NewLegacyAmino() + app := setupBaseApp(t, anteOpt, routerOpt) app.InitChain(abci.RequestInitChain{}) + registerTestCodec(cdc) header := tmproto.Header{Height: app.LastBlockHeight() + 1} app.BeginBlock(abci.RequestBeginBlock{Header: header}) @@ -1617,16 +1521,16 @@ func TestBaseAppMiddleware(t *testing.T) { // execute a tx that will fail ante handler execution // // NOTE: State should not be mutated here. This will be implicitly checked by - // the next txs ante handler execution (customHandlerTxTest). + // the next txs ante handler execution (anteHandlerTxTest). tx := newTxCounter(0, 0) tx.setFailOnAnte(true) - txBytes, err := encCfg.Amino.Marshal(tx) + txBytes, err := cdc.Marshal(tx) require.NoError(t, err) res := app.DeliverTx(abci.RequestDeliverTx{Tx: txBytes}) require.Empty(t, res.Events) require.False(t, res.IsOK(), fmt.Sprintf("%v", res)) - ctx := app.DeliverState().Context() + ctx := app.getState(runTxModeDeliver).ctx store := ctx.KVStore(capKey1) require.Equal(t, int64(0), getIntFromStore(store, anteKey)) @@ -1635,14 +1539,15 @@ func TestBaseAppMiddleware(t *testing.T) { tx = newTxCounter(0, 0) tx.setFailOnHandler(true) - txBytes, err = encCfg.Amino.Marshal(tx) + txBytes, err = cdc.Marshal(tx) require.NoError(t, err) res = app.DeliverTx(abci.RequestDeliverTx{Tx: txBytes}) - require.Empty(t, res.Events) + // should emit ante event + require.NotEmpty(t, res.Events) require.False(t, res.IsOK(), fmt.Sprintf("%v", res)) - ctx = app.DeliverState().Context() + ctx = app.getState(runTxModeDeliver).ctx store = ctx.KVStore(capKey1) require.Equal(t, int64(1), getIntFromStore(store, anteKey)) require.Equal(t, int64(0), getIntFromStore(store, deliverKey)) @@ -1651,14 +1556,14 @@ func TestBaseAppMiddleware(t *testing.T) { // implicitly checked by previous tx executions tx = newTxCounter(1, 0) - txBytes, err = encCfg.Amino.Marshal(tx) + txBytes, err = cdc.Marshal(tx) require.NoError(t, err) res = app.DeliverTx(abci.RequestDeliverTx{Tx: txBytes}) require.NotEmpty(t, res.Events) require.True(t, res.IsOK(), fmt.Sprintf("%v", res)) - ctx = app.DeliverState().Context() + ctx = app.getState(runTxModeDeliver).ctx store = ctx.KVStore(capKey1) require.Equal(t, int64(2), getIntFromStore(store, anteKey)) require.Equal(t, int64(1), getIntFromStore(store, deliverKey)) @@ -1670,39 +1575,48 @@ func TestBaseAppMiddleware(t *testing.T) { func TestGasConsumptionBadTx(t *testing.T) { gasWanted := uint64(5) - ante := func(ctx sdk.Context, tx sdk.Tx, simulate bool) (sdk.Context, error) { - txTest := tx.(txTest) - ctx.GasMeter().ConsumeGas(uint64(txTest.Counter), "counter-ante") - if txTest.FailOnAnte { - return ctx, sdkerrors.Wrap(sdkerrors.ErrUnauthorized, "ante handler failure") - } + anteOpt := func(bapp *BaseApp) { + bapp.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx, simulate bool) (newCtx sdk.Context, err error) { + newCtx = ctx.WithGasMeter(sdk.NewGasMeter(gasWanted)) + + defer func() { + if r := recover(); r != nil { + switch rType := r.(type) { + case sdk.ErrorOutOfGas: + log := fmt.Sprintf("out of gas in location: %v", rType.Descriptor) + err = sdkerrors.Wrap(sdkerrors.ErrOutOfGas, log) + default: + panic(r) + } + } + }() - return ctx, nil + txTest := tx.(txTest) + newCtx.GasMeter().ConsumeGas(uint64(txTest.Counter), "counter-ante") + if txTest.FailOnAnte { + return newCtx, sdkerrors.Wrap(sdkerrors.ErrUnauthorized, "ante handler failure") + } + + return + }) } - txHandlerOpt := func(bapp *baseapp.BaseApp) { - legacyRouter := middleware.NewLegacyRouter() + routerOpt := func(bapp *BaseApp) { r := sdk.NewRoute(routeMsgCounter, func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { count := msg.(*msgCounter).Counter ctx.GasMeter().ConsumeGas(uint64(count), "counter-handler") return &sdk.Result{}, nil }) - legacyRouter.AddRoute(r) - txHandler := testTxHandler( - middleware.TxHandlerOptions{ - LegacyRouter: legacyRouter, - MsgServiceRouter: middleware.NewMsgServiceRouter(encCfg.InterfaceRegistry), - TxDecoder: testTxDecoder(encCfg.Amino), - }, - ante, - ) - bapp.SetTxHandler(txHandler) + bapp.Router().AddRoute(r) } - app, err := setupBaseApp(t, txHandlerOpt) - require.NoError(t, err) + + cdc := codec.NewLegacyAmino() + registerTestCodec(cdc) + + app := setupBaseApp(t, anteOpt, routerOpt) app.InitChain(abci.RequestInitChain{ - ConsensusParams: &tmproto.ConsensusParams{ - Block: &tmproto.BlockParams{ + ConsensusParams: &abci.ConsensusParams{ + Block: &abci.BlockParams{ MaxGas: 9, }, }, @@ -1714,9 +1628,8 @@ func TestGasConsumptionBadTx(t *testing.T) { app.BeginBlock(abci.RequestBeginBlock{Header: header}) tx := newTxCounter(5, 0) - tx.GasLimit = gasWanted tx.setFailOnAnte(true) - txBytes, err := encCfg.Amino.Marshal(tx) + txBytes, err := cdc.Marshal(tx) require.NoError(t, err) res := app.DeliverTx(abci.RequestDeliverTx{Tx: txBytes}) @@ -1724,7 +1637,7 @@ func TestGasConsumptionBadTx(t *testing.T) { // require next tx to fail due to black gas limit tx = newTxCounter(5, 0) - txBytes, err = encCfg.Amino.Marshal(tx) + txBytes, err = cdc.Marshal(tx) require.NoError(t, err) res = app.DeliverTx(abci.RequestDeliverTx{Tx: txBytes}) @@ -1734,38 +1647,24 @@ func TestGasConsumptionBadTx(t *testing.T) { // Test that we can only query from the latest committed state. func TestQuery(t *testing.T) { key, value := []byte("hello"), []byte("goodbye") + anteOpt := func(bapp *BaseApp) { + bapp.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx, simulate bool) (newCtx sdk.Context, err error) { + store := ctx.KVStore(capKey1) + store.Set(key, value) + return + }) + } - txHandlerOpt := func(bapp *baseapp.BaseApp) { - legacyRouter := middleware.NewLegacyRouter() + routerOpt := func(bapp *BaseApp) { r := sdk.NewRoute(routeMsgCounter, func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { store := ctx.KVStore(capKey1) store.Set(key, value) - - any, err := codectypes.NewAnyWithValue(msg) - if err != nil { - return nil, err - } - return &sdk.Result{ - MsgResponses: []*codectypes.Any{any}, - }, nil + return &sdk.Result{}, nil }) - legacyRouter.AddRoute(r) - txHandler := testTxHandler( - middleware.TxHandlerOptions{ - LegacyRouter: legacyRouter, - MsgServiceRouter: middleware.NewMsgServiceRouter(encCfg.InterfaceRegistry), - TxDecoder: testTxDecoder(encCfg.Amino), - }, - func(ctx sdk.Context, tx sdk.Tx, simulate bool) (newCtx sdk.Context, err error) { - store := ctx.KVStore(capKey1) - store.Set(key, value) - return - }, - ) - bapp.SetTxHandler(txHandler) + bapp.Router().AddRoute(r) } - app, err := setupBaseApp(t, txHandlerOpt) - require.NoError(t, err) + + app := setupBaseApp(t, anteOpt, routerOpt) app.InitChain(abci.RequestInitChain{}) @@ -1783,7 +1682,7 @@ func TestQuery(t *testing.T) { require.Equal(t, 0, len(res.Value)) // query is still empty after a CheckTx - _, resTx, err := app.SimCheck(aminoTxEncoder(encCfg.Amino), tx) + _, resTx, err := app.Check(aminoTxEncoder(), tx) require.NoError(t, err) require.NotNil(t, resTx) res = app.Query(query) @@ -1793,7 +1692,7 @@ func TestQuery(t *testing.T) { header := tmproto.Header{Height: app.LastBlockHeight() + 1} app.BeginBlock(abci.RequestBeginBlock{Header: header}) - _, resTx, err = app.SimDeliver(aminoTxEncoder(encCfg.Amino), tx) + _, resTx, err = app.Deliver(aminoTxEncoder(), tx) require.NoError(t, err) require.NotNil(t, resTx) res = app.Query(query) @@ -1806,16 +1705,14 @@ func TestQuery(t *testing.T) { } func TestGRPCQuery(t *testing.T) { - grpcQueryOpt := func(bapp *baseapp.BaseApp) { + grpcQueryOpt := func(bapp *BaseApp) { testdata.RegisterQueryServer( bapp.GRPCQueryRouter(), testdata.QueryImpl{}, ) } - app, err := setupBaseApp(t, grpcQueryOpt) - require.NoError(t, err) - app.GRPCQueryRouter().SetInterfaceRegistry(codectypes.NewInterfaceRegistry()) + app := setupBaseApp(t, grpcQueryOpt) app.InitChain(abci.RequestInitChain{}) header := tmproto.Header{Height: app.LastBlockHeight() + 1} @@ -1841,60 +1738,23 @@ func TestGRPCQuery(t *testing.T) { require.Equal(t, "Hello foo!", res.Greeting) } -func TestGRPCQueryPulsar(t *testing.T) { - grpcQueryOpt := func(bapp *baseapp.BaseApp) { - testdata_pulsar.RegisterQueryServer( - bapp.GRPCQueryRouter(), - testdata_pulsar.QueryImpl{}, - ) - } - - app, err := setupBaseApp(t, grpcQueryOpt) - require.NoError(t, err) - app.GRPCQueryRouter().SetInterfaceRegistry(codectypes.NewInterfaceRegistry()) - - app.InitChain(abci.RequestInitChain{}) - header := tmproto.Header{Height: app.LastBlockHeight() + 1} - app.BeginBlock(abci.RequestBeginBlock{Header: header}) - app.Commit() - - req := &testdata_pulsar.SayHelloRequest{Name: "foo"} - reqBz, err := proto.Marshal(req) - require.NoError(t, err) - - reqQuery := abci.RequestQuery{ - Data: reqBz, - Path: "/testdata.Query/SayHello", - } - - resQuery := app.Query(reqQuery) - - require.Equal(t, abci.CodeTypeOK, resQuery.Code, resQuery) - - var res testdata_pulsar.SayHelloResponse - err = proto.Unmarshal(resQuery.Value, &res) - require.NoError(t, err) - require.Equal(t, "Hello foo!", res.Greeting) -} - // Test p2p filter queries func TestP2PQuery(t *testing.T) { - addrPeerFilterOpt := func(bapp *baseapp.BaseApp) { + addrPeerFilterOpt := func(bapp *BaseApp) { bapp.SetAddrPeerFilter(func(addrport string) abci.ResponseQuery { require.Equal(t, "1.1.1.1:8000", addrport) return abci.ResponseQuery{Code: uint32(3)} }) } - idPeerFilterOpt := func(bapp *baseapp.BaseApp) { + idPeerFilterOpt := func(bapp *BaseApp) { bapp.SetIDPeerFilter(func(id string) abci.ResponseQuery { require.Equal(t, "testid", id) return abci.ResponseQuery{Code: uint32(4)} }) } - app, err := setupBaseApp(t, addrPeerFilterOpt, idPeerFilterOpt) - require.NoError(t, err) + app := setupBaseApp(t, addrPeerFilterOpt, idPeerFilterOpt) addrQuery := abci.RequestQuery{ Path: "/p2p/filter/addr/1.1.1.1:8000", @@ -1910,35 +1770,26 @@ func TestP2PQuery(t *testing.T) { } func TestGetMaximumBlockGas(t *testing.T) { - app, err := setupBaseApp(t) - require.NoError(t, err) + app := setupBaseApp(t) app.InitChain(abci.RequestInitChain{}) ctx := app.NewContext(true, tmproto.Header{}) - app.StoreConsensusParams(ctx, &tmproto.ConsensusParams{Block: &tmproto.BlockParams{MaxGas: 0}}) - require.Equal(t, uint64(0), app.GetMaximumBlockGas(ctx)) + app.StoreConsensusParams(ctx, &abci.ConsensusParams{Block: &abci.BlockParams{MaxGas: 0}}) + require.Equal(t, uint64(0), app.getMaximumBlockGas(ctx)) - app.StoreConsensusParams(ctx, &tmproto.ConsensusParams{Block: &tmproto.BlockParams{MaxGas: -1}}) - require.Equal(t, uint64(0), app.GetMaximumBlockGas(ctx)) + app.StoreConsensusParams(ctx, &abci.ConsensusParams{Block: &abci.BlockParams{MaxGas: -1}}) + require.Equal(t, uint64(0), app.getMaximumBlockGas(ctx)) - app.StoreConsensusParams(ctx, &tmproto.ConsensusParams{Block: &tmproto.BlockParams{MaxGas: 5000000}}) - require.Equal(t, uint64(5000000), app.GetMaximumBlockGas(ctx)) + app.StoreConsensusParams(ctx, &abci.ConsensusParams{Block: &abci.BlockParams{MaxGas: 5000000}}) + require.Equal(t, uint64(5000000), app.getMaximumBlockGas(ctx)) - app.StoreConsensusParams(ctx, &tmproto.ConsensusParams{Block: &tmproto.BlockParams{MaxGas: -5000000}}) - require.Panics(t, func() { app.GetMaximumBlockGas(ctx) }) + app.StoreConsensusParams(ctx, &abci.ConsensusParams{Block: &abci.BlockParams{MaxGas: -5000000}}) + require.Panics(t, func() { app.getMaximumBlockGas(ctx) }) } func TestListSnapshots(t *testing.T) { - setupConfig := &setupConfig{ - blocks: 5, - blockTxs: 4, - snapshotInterval: 2, - snapshotKeepRecent: 2, - pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), - } - - app, err := setupBaseAppWithSnapshots(t, setupConfig) - require.NoError(t, err) + app, teardown := setupBaseAppWithSnapshots(t, 5, 4) + defer teardown() resp := app.ListSnapshots(abci.RequestListSnapshots{}) for _, s := range resp.Snapshots { @@ -1948,153 +1799,14 @@ func TestListSnapshots(t *testing.T) { s.Metadata = nil } assert.Equal(t, abci.ResponseListSnapshots{Snapshots: []*abci.Snapshot{ - {Height: 4, Format: snapshottypes.CurrentFormat, Chunks: 2}, - {Height: 2, Format: snapshottypes.CurrentFormat, Chunks: 1}, + {Height: 4, Format: 1, Chunks: 2}, + {Height: 2, Format: 1, Chunks: 1}, }}, resp) } -func TestSnapshotWithPruning(t *testing.T) { - testcases := map[string]struct { - config *setupConfig - expectedSnapshots []*abci.Snapshot - expectedErr error - }{ - "prune nothing with snapshot": { - config: &setupConfig{ - blocks: 20, - blockTxs: 2, - snapshotInterval: 5, - snapshotKeepRecent: 1, - pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), - }, - expectedSnapshots: []*abci.Snapshot{ - {Height: 20, Format: 2, Chunks: 5}, - }, - }, - "prune everything with snapshot": { - config: &setupConfig{ - blocks: 20, - blockTxs: 2, - snapshotInterval: 5, - snapshotKeepRecent: 1, - pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningEverything), - }, - expectedSnapshots: []*abci.Snapshot{ - {Height: 20, Format: 2, Chunks: 5}, - }, - }, - "default pruning with snapshot": { - config: &setupConfig{ - blocks: 20, - blockTxs: 2, - snapshotInterval: 5, - snapshotKeepRecent: 1, - pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningDefault), - }, - expectedSnapshots: []*abci.Snapshot{ - {Height: 20, Format: 2, Chunks: 5}, - }, - }, - "custom": { - config: &setupConfig{ - blocks: 25, - blockTxs: 2, - snapshotInterval: 5, - snapshotKeepRecent: 2, - pruningOpts: pruningtypes.NewCustomPruningOptions(12, 12), - }, - expectedSnapshots: []*abci.Snapshot{ - {Height: 25, Format: 2, Chunks: 6}, - {Height: 20, Format: 2, Chunks: 5}, - }, - }, - "no snapshots": { - config: &setupConfig{ - blocks: 10, - blockTxs: 2, - snapshotInterval: 0, // 0 implies disable snapshots - pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), - }, - expectedSnapshots: []*abci.Snapshot{}, - }, - "keep all snapshots": { - config: &setupConfig{ - blocks: 10, - blockTxs: 2, - snapshotInterval: 3, - snapshotKeepRecent: 0, // 0 implies keep all snapshots - pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), - }, - expectedSnapshots: []*abci.Snapshot{ - {Height: 9, Format: 2, Chunks: 2}, - {Height: 6, Format: 2, Chunks: 2}, - {Height: 3, Format: 2, Chunks: 1}, - }, - }, - } - - for name, tc := range testcases { - t.Run(name, func(t *testing.T) { - app, err := setupBaseAppWithSnapshots(t, tc.config) - - if tc.expectedErr != nil { - require.Error(t, err) - require.Equal(t, tc.expectedErr.Error(), err.Error()) - return - } - require.NoError(t, err) - - resp := app.ListSnapshots(abci.RequestListSnapshots{}) - for _, s := range resp.Snapshots { - assert.NotEmpty(t, s.Hash) - assert.NotEmpty(t, s.Metadata) - s.Hash = nil - s.Metadata = nil - } - fmt.Println(resp) - assert.Equal(t, abci.ResponseListSnapshots{Snapshots: tc.expectedSnapshots}, resp) - - // Validate that heights were pruned correctly by querying the state at the last height that should be present relative to latest - // and the first height that should be pruned. - // - // Exceptions: - // * Prune nothing: should be able to query all heights (we only test first and latest) - // * Prune default: should be able to query all heights (we only test first and latest) - // * The reason for default behaving this way is that we only commit 20 heights but default has 100_000 keep-recent - var lastExistingHeight int64 - if tc.config.pruningOpts.GetPruningStrategy() == pruningtypes.PruningNothing || tc.config.pruningOpts.GetPruningStrategy() == pruningtypes.PruningDefault { - lastExistingHeight = 1 - } else { - // Integer division rounds down so by multiplying back we get the last height at which we pruned - lastExistingHeight = int64((tc.config.blocks/tc.config.pruningOpts.Interval)*tc.config.pruningOpts.Interval - tc.config.pruningOpts.KeepRecent) - } - - // Query 1 - res := app.Query(abci.RequestQuery{Path: fmt.Sprintf("/store/%s/key", capKey2.Name()), Data: []byte("0"), Height: lastExistingHeight}) - require.NotNil(t, res, "height: %d", lastExistingHeight) - require.NotNil(t, res.Value, "height: %d", lastExistingHeight) - - // Query 2 - res = app.Query(abci.RequestQuery{Path: fmt.Sprintf("/store/%s/key", capKey2.Name()), Data: []byte("0"), Height: lastExistingHeight - 1}) - require.NotNil(t, res, "height: %d", lastExistingHeight-1) - if tc.config.pruningOpts.GetPruningStrategy() == pruningtypes.PruningNothing || tc.config.pruningOpts.GetPruningStrategy() == pruningtypes.PruningDefault { - // With prune nothing or default, we query height 0 which translates to the latest height. - require.NotNil(t, res.Value, "height: %d", lastExistingHeight-1) - } - }) - } -} - func TestLoadSnapshotChunk(t *testing.T) { - setupConfig := &setupConfig{ - blocks: 2, - blockTxs: 5, - snapshotInterval: 2, - snapshotKeepRecent: 2, - pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), - } - app, err := setupBaseAppWithSnapshots(t, setupConfig) - require.NoError(t, err) + app, teardown := setupBaseAppWithSnapshots(t, 2, 5) + defer teardown() testcases := map[string]struct { height uint64 @@ -2102,13 +1814,13 @@ func TestLoadSnapshotChunk(t *testing.T) { chunk uint32 expectEmpty bool }{ - "Existing snapshot": {2, snapshottypes.CurrentFormat, 1, false}, - "Missing height": {100, snapshottypes.CurrentFormat, 1, true}, - "Missing format": {2, 3, 1, true}, - "Missing chunk": {2, snapshottypes.CurrentFormat, 9, true}, - "Zero height": {0, snapshottypes.CurrentFormat, 1, true}, + "Existing snapshot": {2, 1, 1, false}, + "Missing height": {100, 1, 1, true}, + "Missing format": {2, 2, 1, true}, + "Missing chunk": {2, 1, 9, true}, + "Zero height": {0, 1, 1, true}, "Zero format": {2, 0, 1, true}, - "Zero chunk": {2, snapshottypes.CurrentFormat, 0, false}, + "Zero chunk": {2, 1, 0, false}, } for name, tc := range testcases { tc := tc @@ -2129,15 +1841,8 @@ func TestLoadSnapshotChunk(t *testing.T) { func TestOfferSnapshot_Errors(t *testing.T) { // Set up app before test cases, since it's fairly expensive. - setupConfig := &setupConfig{ - blocks: 0, - blockTxs: 0, - snapshotInterval: 2, - snapshotKeepRecent: 2, - pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), - } - app, err := setupBaseAppWithSnapshots(t, setupConfig) - require.NoError(t, err) + app, teardown := setupBaseAppWithSnapshots(t, 0, 0) + defer teardown() m := snapshottypes.Metadata{ChunkHashes: [][]byte{{1}, {2}, {3}}} metadata, err := m.Marshal() @@ -2153,13 +1858,13 @@ func TestOfferSnapshot_Errors(t *testing.T) { Height: 1, Format: 9, Chunks: 3, Hash: hash, Metadata: metadata, }, abci.ResponseOfferSnapshot_REJECT_FORMAT}, "incorrect chunk count": {&abci.Snapshot{ - Height: 1, Format: snapshottypes.CurrentFormat, Chunks: 2, Hash: hash, Metadata: metadata, + Height: 1, Format: 1, Chunks: 2, Hash: hash, Metadata: metadata, }, abci.ResponseOfferSnapshot_REJECT}, "no chunks": {&abci.Snapshot{ - Height: 1, Format: snapshottypes.CurrentFormat, Chunks: 0, Hash: hash, Metadata: metadata, + Height: 1, Format: 1, Chunks: 0, Hash: hash, Metadata: metadata, }, abci.ResponseOfferSnapshot_REJECT}, "invalid metadata serialization": {&abci.Snapshot{ - Height: 1, Format: snapshottypes.CurrentFormat, Chunks: 0, Hash: hash, Metadata: []byte{3, 1, 4}, + Height: 1, Format: 1, Chunks: 0, Hash: hash, Metadata: []byte{3, 1, 4}, }, abci.ResponseOfferSnapshot_REJECT}, } for name, tc := range testcases { @@ -2191,25 +1896,11 @@ func TestOfferSnapshot_Errors(t *testing.T) { } func TestApplySnapshotChunk(t *testing.T) { - setupConfig1 := &setupConfig{ - blocks: 4, - blockTxs: 10, - snapshotInterval: 2, - snapshotKeepRecent: 2, - pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), - } - source, err := setupBaseAppWithSnapshots(t, setupConfig1) - require.NoError(t, err) + source, teardown := setupBaseAppWithSnapshots(t, 4, 10) + defer teardown() - setupConfig2 := &setupConfig{ - blocks: 0, - blockTxs: 0, - snapshotInterval: 2, - snapshotKeepRecent: 2, - pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), - } - target, err := setupBaseAppWithSnapshots(t, setupConfig2) - require.NoError(t, err) + target, teardown := setupBaseAppWithSnapshots(t, 0, 0) + defer teardown() // Fetch latest snapshot to restore respList := source.ListSnapshots(abci.RequestListSnapshots{}) @@ -2276,23 +1967,25 @@ func (rtr *testCustomRouter) Route(ctx sdk.Context, path string) sdk.Handler { } func TestWithRouter(t *testing.T) { + // test increments in the ante + anteKey := []byte("ante-key") + anteOpt := func(bapp *BaseApp) { bapp.SetAnteHandler(anteHandlerTxTest(t, capKey1, anteKey)) } + // test increments in the handler deliverKey := []byte("deliver-key") - - txHandlerOpt := func(bapp *baseapp.BaseApp) { - customRouter := &testCustomRouter{routes: sync.Map{}} + routerOpt := func(bapp *BaseApp) { + bapp.SetRouter(&testCustomRouter{routes: sync.Map{}}) r := sdk.NewRoute(routeMsgCounter, handlerMsgCounter(t, capKey1, deliverKey)) - customRouter.AddRoute(r) - txHandler := middleware.ComposeMiddlewares( - middleware.NewRunMsgsTxHandler(middleware.NewMsgServiceRouter(encCfg.InterfaceRegistry), customRouter), - middleware.NewTxDecoderMiddleware(testTxDecoder(encCfg.Amino)), - ) - bapp.SetTxHandler(txHandler) + bapp.Router().AddRoute(r) } - app, err := setupBaseApp(t, txHandlerOpt) - require.NoError(t, err) + + app := setupBaseApp(t, anteOpt, routerOpt) app.InitChain(abci.RequestInitChain{}) + // Create same codec used in txDecoder + codec := codec.NewLegacyAmino() + registerTestCodec(codec) + nBlocks := 3 txPerHeight := 5 @@ -2304,7 +1997,7 @@ func TestWithRouter(t *testing.T) { counter := int64(blockN*txPerHeight + i) tx := newTxCounter(counter, counter) - txBytes, err := encCfg.Amino.Marshal(tx) + txBytes, err := codec.Marshal(tx) require.NoError(t, err) res := app.DeliverTx(abci.RequestDeliverTx{Tx: txBytes}) @@ -2321,13 +2014,13 @@ func TestBaseApp_EndBlock(t *testing.T) { name := t.Name() logger := defaultLogger() - cp := &tmproto.ConsensusParams{ - Block: &tmproto.BlockParams{ + cp := &abci.ConsensusParams{ + Block: &abci.BlockParams{ MaxGas: 5000000, }, } - app := baseapp.NewBaseApp(name, logger, db) + app := NewBaseApp(name, logger, db, nil) app.SetParamStore(¶mStore{db: dbm.NewMemDB()}) app.InitChain(abci.RequestInitChain{ ConsensusParams: cp, @@ -2347,164 +2040,3 @@ func TestBaseApp_EndBlock(t *testing.T) { require.Equal(t, int64(100), res.GetValidatorUpdates()[0].Power) require.Equal(t, cp.Block.MaxGas, res.ConsensusParamUpdates.Block.MaxGas) } - -func TestBaseApp_Init(t *testing.T) { - db := dbm.NewMemDB() - name := t.Name() - logger := defaultLogger() - - snapshotStore, err := snapshots.NewStore(dbm.NewMemDB(), testutil.GetTempDir(t)) - require.NoError(t, err) - - testCases := map[string]struct { - bapp *baseapp.BaseApp - expectedPruning pruningtypes.PruningOptions - expectedSnapshot snapshottypes.SnapshotOptions - expectedErr error - }{ - "snapshot but no pruning": { - baseapp.NewBaseApp(name, logger, db, - baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 2)), - ), - pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), - snapshottypes.NewSnapshotOptions(1500, 2), - // if no pruning is set, the default is PruneNothing - nil, - }, - "pruning everything only": { - baseapp.NewBaseApp(name, logger, db, - baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningEverything)), - ), - pruningtypes.NewPruningOptions(pruningtypes.PruningEverything), - snapshottypes.NewSnapshotOptions(snapshottypes.SnapshotIntervalOff, 0), - nil, - }, - "pruning nothing only": { - baseapp.NewBaseApp(name, logger, db, - baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)), - ), - pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), - snapshottypes.NewSnapshotOptions(snapshottypes.SnapshotIntervalOff, 0), - nil, - }, - "pruning default only": { - baseapp.NewBaseApp(name, logger, db, - baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningDefault)), - ), - pruningtypes.NewPruningOptions(pruningtypes.PruningDefault), - snapshottypes.NewSnapshotOptions(snapshottypes.SnapshotIntervalOff, 0), - nil, - }, - "pruning custom only": { - baseapp.NewBaseApp(name, logger, db, - baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(10, 10)), - ), - pruningtypes.NewCustomPruningOptions(10, 10), - snapshottypes.NewSnapshotOptions(snapshottypes.SnapshotIntervalOff, 0), - nil, - }, - "pruning everything and snapshots": { - baseapp.NewBaseApp(name, logger, db, - baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningEverything)), - baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 2)), - ), - pruningtypes.NewPruningOptions(pruningtypes.PruningEverything), - snapshottypes.NewSnapshotOptions(1500, 2), - nil, - }, - "pruning nothing and snapshots": { - baseapp.NewBaseApp(name, logger, db, - baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)), - baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 2)), - ), - pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), - snapshottypes.NewSnapshotOptions(1500, 2), - nil, - }, - "pruning default and snapshots": { - baseapp.NewBaseApp(name, logger, db, - baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningDefault)), - baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 2)), - ), - pruningtypes.NewPruningOptions(pruningtypes.PruningDefault), - snapshottypes.NewSnapshotOptions(1500, 2), - nil, - }, - "pruning custom and snapshots": { - baseapp.NewBaseApp(name, logger, db, - baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(10, 10)), - baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 2)), - ), - pruningtypes.NewCustomPruningOptions(10, 10), - snapshottypes.NewSnapshotOptions(1500, 2), - nil, - }, - "error custom pruning 0 interval": { - baseapp.NewBaseApp(name, logger, db, - baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(10, 0)), - baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 2)), - ), - pruningtypes.NewCustomPruningOptions(10, 0), - snapshottypes.NewSnapshotOptions(1500, 2), - pruningtypes.ErrPruningIntervalZero, - }, - "error custom pruning too small interval": { - baseapp.NewBaseApp(name, logger, db, - baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(10, 9)), - baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 2)), - ), - pruningtypes.NewCustomPruningOptions(10, 9), - snapshottypes.NewSnapshotOptions(1500, 2), - pruningtypes.ErrPruningIntervalTooSmall, - }, - "error custom pruning too small keep recent": { - baseapp.NewBaseApp(name, logger, db, - baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(1, 10)), - baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 2)), - ), - pruningtypes.NewCustomPruningOptions(9, 10), - snapshottypes.NewSnapshotOptions(1500, 2), - pruningtypes.ErrPruningKeepRecentTooSmall, - }, - "snapshot zero interval - manager not set": { - baseapp.NewBaseApp(name, logger, db, - baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(10, 10)), - baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(snapshottypes.SnapshotIntervalOff, 2)), - ), - pruningtypes.NewCustomPruningOptions(10, 10), - snapshottypes.NewSnapshotOptions(snapshottypes.SnapshotIntervalOff, 0), - nil, - }, - "snapshot zero keep recent - allowed": { - baseapp.NewBaseApp(name, logger, db, - baseapp.SetPruning(pruningtypes.NewCustomPruningOptions(10, 10)), - baseapp.SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(1500, 0)), - ), - pruningtypes.NewCustomPruningOptions(10, 10), - snapshottypes.NewSnapshotOptions(1500, 0), // 0 snapshot-keep-recent means keep all - nil, - }, - } - - for _, tc := range testCases { - // Init and validate - require.Equal(t, tc.expectedErr, tc.bapp.Init()) - if tc.expectedErr != nil { - continue - } - - // Check that settings were set correctly - actualPruning := tc.bapp.CMS().GetPruning() - require.Equal(t, tc.expectedPruning, actualPruning) - - snapshotManager := tc.bapp.GetSnapshotManager() - if tc.expectedSnapshot.Interval == snapshottypes.SnapshotIntervalOff { - require.Nil(t, snapshotManager) - continue - } - require.NotNil(t, snapshotManager) - - require.Equal(t, tc.expectedSnapshot.Interval, snapshotManager.GetInterval()) - require.Equal(t, tc.expectedSnapshot.KeepRecent, snapshotManager.GetKeepRecent()) - } -} diff --git a/baseapp/msg_service_router.go b/baseapp/msg_service_router.go new file mode 100644 index 000000000000..1b7f8f89bf73 --- /dev/null +++ b/baseapp/msg_service_router.go @@ -0,0 +1,140 @@ +package baseapp + +import ( + "context" + "fmt" + + gogogrpc "github.com/gogo/protobuf/grpc" + "github.com/gogo/protobuf/proto" + "google.golang.org/grpc" + + codectypes "github.com/cosmos/cosmos-sdk/codec/types" + sdk "github.com/cosmos/cosmos-sdk/types" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" +) + +// MsgServiceRouter routes fully-qualified Msg service methods to their handler. +type MsgServiceRouter struct { + interfaceRegistry codectypes.InterfaceRegistry + routes map[string]MsgServiceHandler +} + +var _ gogogrpc.Server = &MsgServiceRouter{} + +// NewMsgServiceRouter creates a new MsgServiceRouter. +func NewMsgServiceRouter() *MsgServiceRouter { + return &MsgServiceRouter{ + routes: map[string]MsgServiceHandler{}, + } +} + +// MsgServiceHandler defines a function type which handles Msg service message. +type MsgServiceHandler = func(ctx sdk.Context, req sdk.Msg) (*sdk.Result, error) + +// Handler returns the MsgServiceHandler for a given msg or nil if not found. +func (msr *MsgServiceRouter) Handler(msg sdk.Msg) MsgServiceHandler { + return msr.routes[sdk.MsgTypeURL(msg)] +} + +// HandlerByTypeURL returns the MsgServiceHandler for a given query route path or nil +// if not found. +func (msr *MsgServiceRouter) HandlerByTypeURL(typeURL string) MsgServiceHandler { + return msr.routes[typeURL] +} + +// RegisterService implements the gRPC Server.RegisterService method. sd is a gRPC +// service description, handler is an object which implements that gRPC service. +// +// This function PANICs: +// - if it is called before the service `Msg`s have been registered using +// RegisterInterfaces, +// - or if a service is being registered twice. +func (msr *MsgServiceRouter) RegisterService(sd *grpc.ServiceDesc, handler interface{}) { + // Adds a top-level query handler based on the gRPC service name. + for _, method := range sd.Methods { + fqMethod := fmt.Sprintf("/%s/%s", sd.ServiceName, method.MethodName) + methodHandler := method.Handler + + var requestTypeName string + + // NOTE: This is how we pull the concrete request type for each handler for registering in the InterfaceRegistry. + // This approach is maybe a bit hacky, but less hacky than reflecting on the handler object itself. + // We use a no-op interceptor to avoid actually calling into the handler itself. + _, _ = methodHandler(nil, context.Background(), func(i interface{}) error { + msg, ok := i.(sdk.Msg) + if !ok { + // We panic here because there is no other alternative and the app cannot be initialized correctly + // this should only happen if there is a problem with code generation in which case the app won't + // work correctly anyway. + panic(fmt.Errorf("can't register request type %T for service method %s", i, fqMethod)) + } + + requestTypeName = sdk.MsgTypeURL(msg) + return nil + }, noopInterceptor) + + // Check that the service Msg fully-qualified method name has already + // been registered (via RegisterInterfaces). If the user registers a + // service without registering according service Msg type, there might be + // some unexpected behavior down the road. Since we can't return an error + // (`Server.RegisterService` interface restriction) we panic (at startup). + reqType, err := msr.interfaceRegistry.Resolve(requestTypeName) + if err != nil || reqType == nil { + panic( + fmt.Errorf( + "type_url %s has not been registered yet. "+ + "Before calling RegisterService, you must register all interfaces by calling the `RegisterInterfaces` "+ + "method on module.BasicManager. Each module should call `msgservice.RegisterMsgServiceDesc` inside its "+ + "`RegisterInterfaces` method with the `_Msg_serviceDesc` generated by proto-gen", + requestTypeName, + ), + ) + } + + // Check that each service is only registered once. If a service is + // registered more than once, then we should error. Since we can't + // return an error (`Server.RegisterService` interface restriction) we + // panic (at startup). + _, found := msr.routes[requestTypeName] + if found { + panic( + fmt.Errorf( + "msg service %s has already been registered. Please make sure to only register each service once. "+ + "This usually means that there are conflicting modules registering the same msg service", + fqMethod, + ), + ) + } + + msr.routes[requestTypeName] = func(ctx sdk.Context, req sdk.Msg) (*sdk.Result, error) { + ctx = ctx.WithEventManager(sdk.NewEventManager()) + interceptor := func(goCtx context.Context, _ interface{}, _ *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) { + goCtx = context.WithValue(goCtx, sdk.SdkContextKey, ctx) + return handler(goCtx, req) + } + // Call the method handler from the service description with the handler object. + // We don't do any decoding here because the decoding was already done. + res, err := methodHandler(handler, sdk.WrapSDKContext(ctx), noopDecoder, interceptor) + if err != nil { + return nil, err + } + + resMsg, ok := res.(proto.Message) + if !ok { + return nil, sdkerrors.Wrapf(sdkerrors.ErrInvalidType, "Expecting proto.Message, got %T", resMsg) + } + + return sdk.WrapServiceResult(ctx, resMsg, err) + } + } +} + +// SetInterfaceRegistry sets the interface registry for the router. +func (msr *MsgServiceRouter) SetInterfaceRegistry(interfaceRegistry codectypes.InterfaceRegistry) { + msr.interfaceRegistry = interfaceRegistry +} + +func noopDecoder(_ interface{}) error { return nil } +func noopInterceptor(_ context.Context, _ interface{}, _ *grpc.UnaryServerInfo, _ grpc.UnaryHandler) (interface{}, error) { + return nil, nil +} diff --git a/baseapp/msg_service_router_test.go b/baseapp/msg_service_router_test.go new file mode 100644 index 000000000000..471fa17d9ea6 --- /dev/null +++ b/baseapp/msg_service_router_test.go @@ -0,0 +1,121 @@ +package baseapp_test + +import ( + "testing" + + "github.com/stretchr/testify/require" + abci "github.com/tendermint/tendermint/abci/types" + "github.com/tendermint/tendermint/libs/log" + tmproto "github.com/tendermint/tendermint/proto/tendermint/types" + dbm "github.com/tendermint/tm-db" + + "github.com/cosmos/cosmos-sdk/baseapp" + "github.com/cosmos/cosmos-sdk/client/tx" + "github.com/cosmos/cosmos-sdk/simapp" + "github.com/cosmos/cosmos-sdk/testutil/testdata" + "github.com/cosmos/cosmos-sdk/types/tx/signing" + authsigning "github.com/cosmos/cosmos-sdk/x/auth/signing" +) + +func TestRegisterMsgService(t *testing.T) { + db := dbm.NewMemDB() + + // Create an encoding config that doesn't register testdata Msg services. + encCfg := simapp.MakeTestEncodingConfig() + app := baseapp.NewBaseApp("test", log.NewDefaultLogger("plain", "info", false), db, encCfg.TxConfig.TxDecoder()) + app.SetInterfaceRegistry(encCfg.InterfaceRegistry) + require.Panics(t, func() { + testdata.RegisterMsgServer( + app.MsgServiceRouter(), + testdata.MsgServerImpl{}, + ) + }) + + // Register testdata Msg services, and rerun `RegisterService`. + testdata.RegisterInterfaces(encCfg.InterfaceRegistry) + require.NotPanics(t, func() { + testdata.RegisterMsgServer( + app.MsgServiceRouter(), + testdata.MsgServerImpl{}, + ) + }) +} + +func TestRegisterMsgServiceTwice(t *testing.T) { + // Setup baseapp. + db := dbm.NewMemDB() + encCfg := simapp.MakeTestEncodingConfig() + app := baseapp.NewBaseApp("test", log.NewDefaultLogger("plain", "info", false), db, encCfg.TxConfig.TxDecoder()) + app.SetInterfaceRegistry(encCfg.InterfaceRegistry) + testdata.RegisterInterfaces(encCfg.InterfaceRegistry) + + // First time registering service shouldn't panic. + require.NotPanics(t, func() { + testdata.RegisterMsgServer( + app.MsgServiceRouter(), + testdata.MsgServerImpl{}, + ) + }) + + // Second time should panic. + require.Panics(t, func() { + testdata.RegisterMsgServer( + app.MsgServiceRouter(), + testdata.MsgServerImpl{}, + ) + }) +} + +func TestMsgService(t *testing.T) { + priv, _, _ := testdata.KeyTestPubAddr() + encCfg := simapp.MakeTestEncodingConfig() + testdata.RegisterInterfaces(encCfg.InterfaceRegistry) + db := dbm.NewMemDB() + app := baseapp.NewBaseApp("test", log.NewDefaultLogger("plain", "info", false), db, encCfg.TxConfig.TxDecoder()) + app.SetInterfaceRegistry(encCfg.InterfaceRegistry) + testdata.RegisterMsgServer( + app.MsgServiceRouter(), + testdata.MsgServerImpl{}, + ) + _ = app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: 1}}) + + msg := testdata.MsgCreateDog{Dog: &testdata.Dog{Name: "Spot"}} + txBuilder := encCfg.TxConfig.NewTxBuilder() + txBuilder.SetFeeAmount(testdata.NewTestFeeAmount()) + txBuilder.SetGasLimit(testdata.NewTestGasLimit()) + err := txBuilder.SetMsgs(&msg) + require.NoError(t, err) + + // First round: we gather all the signer infos. We use the "set empty + // signature" hack to do that. + sigV2 := signing.SignatureV2{ + PubKey: priv.PubKey(), + Data: &signing.SingleSignatureData{ + SignMode: encCfg.TxConfig.SignModeHandler().DefaultMode(), + Signature: nil, + }, + Sequence: 0, + } + + err = txBuilder.SetSignatures(sigV2) + require.NoError(t, err) + + // Second round: all signer infos are set, so each signer can sign. + signerData := authsigning.SignerData{ + ChainID: "test", + AccountNumber: 0, + Sequence: 0, + } + sigV2, err = tx.SignWithPrivKey( + encCfg.TxConfig.SignModeHandler().DefaultMode(), signerData, + txBuilder, priv, encCfg.TxConfig, 0) + require.NoError(t, err) + err = txBuilder.SetSignatures(sigV2) + require.NoError(t, err) + + // Send the tx to the app + txBytes, err := encCfg.TxConfig.TxEncoder()(txBuilder.GetTx()) + require.NoError(t, err) + res := app.DeliverTx(abci.RequestDeliverTx{Tx: txBytes}) + require.Equal(t, abci.CodeTypeOK, res.Code, "res=%+v", res) +} diff --git a/baseapp/options.go b/baseapp/options.go index 9ac0d0e33cc0..e0960a7cc2a2 100644 --- a/baseapp/options.go +++ b/baseapp/options.go @@ -12,7 +12,6 @@ import ( snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" "github.com/cosmos/cosmos-sdk/store" sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/types/tx" ) // File for storing in-package BaseApp optional functions, @@ -146,12 +145,12 @@ func (app *BaseApp) SetEndBlocker(endBlocker sdk.EndBlocker) { app.endBlocker = endBlocker } -func (app *BaseApp) SetTxHandler(txHandler tx.Handler) { +func (app *BaseApp) SetAnteHandler(ah sdk.AnteHandler) { if app.sealed { - panic("SetTxHandler() on sealed BaseApp") + panic("SetAnteHandler() on sealed BaseApp") } - app.txHandler = txHandler + app.anteHandler = ah } func (app *BaseApp) SetAddrPeerFilter(pf sdk.PeerFilter) { @@ -193,6 +192,14 @@ func (app *BaseApp) SetStoreLoader(loader StoreLoader) { app.storeLoader = loader } +// SetRouter allows us to customize the router. +func (app *BaseApp) SetRouter(router sdk.Router) { + if app.sealed { + panic("SetRouter() on sealed BaseApp") + } + app.router = router +} + // SetSnapshot sets the snapshot store and options. func (app *BaseApp) SetSnapshot(snapshotStore *snapshots.Store, opts snapshottypes.SnapshotOptions) { if app.sealed { diff --git a/baseapp/recovery.go b/baseapp/recovery.go new file mode 100644 index 000000000000..7f0687800c65 --- /dev/null +++ b/baseapp/recovery.go @@ -0,0 +1,77 @@ +package baseapp + +import ( + "fmt" + "runtime/debug" + + sdk "github.com/cosmos/cosmos-sdk/types" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" +) + +// RecoveryHandler handles recovery() object. +// Return a non-nil error if recoveryObj was processed. +// Return nil if recoveryObj was not processed. +type RecoveryHandler func(recoveryObj interface{}) error + +// recoveryMiddleware is wrapper for RecoveryHandler to create chained recovery handling. +// returns (recoveryMiddleware, nil) if recoveryObj was not processed and should be passed to the next middleware in chain. +// returns (nil, error) if recoveryObj was processed and middleware chain processing should be stopped. +type recoveryMiddleware func(recoveryObj interface{}) (recoveryMiddleware, error) + +// processRecovery processes recoveryMiddleware chain for recovery() object. +// Chain processing stops on non-nil error or when chain is processed. +func processRecovery(recoveryObj interface{}, middleware recoveryMiddleware) error { + if middleware == nil { + return nil + } + + next, err := middleware(recoveryObj) + if err != nil { + return err + } + + return processRecovery(recoveryObj, next) +} + +// newRecoveryMiddleware creates a RecoveryHandler middleware. +func newRecoveryMiddleware(handler RecoveryHandler, next recoveryMiddleware) recoveryMiddleware { + return func(recoveryObj interface{}) (recoveryMiddleware, error) { + if err := handler(recoveryObj); err != nil { + return nil, err + } + + return next, nil + } +} + +// newOutOfGasRecoveryMiddleware creates a standard OutOfGas recovery middleware for app.runTx method. +func newOutOfGasRecoveryMiddleware(gasWanted uint64, ctx sdk.Context, next recoveryMiddleware) recoveryMiddleware { + handler := func(recoveryObj interface{}) error { + err, ok := recoveryObj.(sdk.ErrorOutOfGas) + if !ok { + return nil + } + + return sdkerrors.Wrap( + sdkerrors.ErrOutOfGas, fmt.Sprintf( + "out of gas in location: %v; gasWanted: %d, gasUsed: %d", + err.Descriptor, gasWanted, ctx.GasMeter().GasConsumed(), + ), + ) + } + + return newRecoveryMiddleware(handler, next) +} + +// newDefaultRecoveryMiddleware creates a default (last in chain) recovery middleware for app.runTx method. +func newDefaultRecoveryMiddleware() recoveryMiddleware { + handler := func(recoveryObj interface{}) error { + return sdkerrors.Wrap( + sdkerrors.ErrPanic, fmt.Sprintf( + "recovered: %v\nstack:\n%v", recoveryObj, string(debug.Stack()), + ), + ) + } + + return newRecoveryMiddleware(handler, nil) +} diff --git a/baseapp/recovery_test.go b/baseapp/recovery_test.go new file mode 100644 index 000000000000..b75892c63818 --- /dev/null +++ b/baseapp/recovery_test.go @@ -0,0 +1,64 @@ +package baseapp + +import ( + "fmt" + "testing" + + "github.com/stretchr/testify/require" +) + +// Test that recovery chain produces expected error at specific middleware layer +func TestRecoveryChain(t *testing.T) { + createError := func(id int) error { + return fmt.Errorf("error from id: %d", id) + } + + createHandler := func(id int, handle bool) RecoveryHandler { + return func(_ interface{}) error { + if handle { + return createError(id) + } + return nil + } + } + + // check recovery chain [1] -> 2 -> 3 + { + mw := newRecoveryMiddleware(createHandler(3, false), nil) + mw = newRecoveryMiddleware(createHandler(2, false), mw) + mw = newRecoveryMiddleware(createHandler(1, true), mw) + receivedErr := processRecovery(nil, mw) + + require.Equal(t, createError(1), receivedErr) + } + + // check recovery chain 1 -> [2] -> 3 + { + mw := newRecoveryMiddleware(createHandler(3, false), nil) + mw = newRecoveryMiddleware(createHandler(2, true), mw) + mw = newRecoveryMiddleware(createHandler(1, false), mw) + receivedErr := processRecovery(nil, mw) + + require.Equal(t, createError(2), receivedErr) + } + + // check recovery chain 1 -> 2 -> [3] + { + mw := newRecoveryMiddleware(createHandler(3, true), nil) + mw = newRecoveryMiddleware(createHandler(2, false), mw) + mw = newRecoveryMiddleware(createHandler(1, false), mw) + receivedErr := processRecovery(nil, mw) + + require.Equal(t, createError(3), receivedErr) + } + + // check recovery chain 1 -> 2 -> 3 + { + mw := newRecoveryMiddleware(createHandler(3, false), nil) + mw = newRecoveryMiddleware(createHandler(2, false), mw) + mw = newRecoveryMiddleware(createHandler(1, false), mw) + receivedErr := processRecovery(nil, mw) + + require.Nil(t, receivedErr) + } +} diff --git a/baseapp/router.go b/baseapp/router.go new file mode 100644 index 000000000000..7e2e70a0c6f3 --- /dev/null +++ b/baseapp/router.go @@ -0,0 +1,41 @@ +package baseapp + +import ( + "fmt" + + sdk "github.com/cosmos/cosmos-sdk/types" +) + +type Router struct { + routes map[string]sdk.Handler +} + +var _ sdk.Router = NewRouter() + +// NewRouter returns a reference to a new router. +func NewRouter() *Router { + return &Router{ + routes: make(map[string]sdk.Handler), + } +} + +// AddRoute adds a route path to the router with a given handler. The route must +// be alphanumeric. +func (rtr *Router) AddRoute(route sdk.Route) sdk.Router { + if !sdk.IsAlphaNumeric(route.Path()) { + panic("route expressions can only contain alphanumeric characters") + } + if rtr.routes[route.Path()] != nil { + panic(fmt.Sprintf("route %s has already been initialized", route.Path())) + } + + rtr.routes[route.Path()] = route.Handler() + return rtr +} + +// Route returns a handler for a given route path. +// +// TODO: Handle expressive matches. +func (rtr *Router) Route(_ sdk.Context, path string) sdk.Handler { + return rtr.routes[path] +} diff --git a/baseapp/router_test.go b/baseapp/router_test.go new file mode 100644 index 000000000000..1e11dc0ca089 --- /dev/null +++ b/baseapp/router_test.go @@ -0,0 +1,31 @@ +package baseapp + +import ( + "testing" + + "github.com/stretchr/testify/require" + + sdk "github.com/cosmos/cosmos-sdk/types" +) + +var testHandler = func(_ sdk.Context, _ sdk.Msg) (*sdk.Result, error) { + return &sdk.Result{}, nil +} + +func TestRouter(t *testing.T) { + rtr := NewRouter() + + // require panic on invalid route + require.Panics(t, func() { + rtr.AddRoute(sdk.NewRoute("*", testHandler)) + }) + + rtr.AddRoute(sdk.NewRoute("testRoute", testHandler)) + h := rtr.Route(sdk.Context{}, "testRoute") + require.NotNil(t, h) + + // require panic on duplicate route + require.Panics(t, func() { + rtr.AddRoute(sdk.NewRoute("testRoute", testHandler)) + }) +} diff --git a/baseapp/test_helpers.go b/baseapp/test_helpers.go index 6b770499c70b..003d3e79bd6c 100644 --- a/baseapp/test_helpers.go +++ b/baseapp/test_helpers.go @@ -1,81 +1,39 @@ package baseapp import ( - abci "github.com/tendermint/tendermint/abci/types" tmproto "github.com/tendermint/tendermint/proto/tendermint/types" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/tx" ) // SimCheck defines a CheckTx helper function that used in tests and simulations. -func (app *BaseApp) SimCheck(txEncoder sdk.TxEncoder, sdkTx sdk.Tx) (sdk.GasInfo, *sdk.Result, error) { - // CheckTx expects tx bytes as argument, so we encode the tx argument into - // bytes. Note that CheckTx will actually decode those bytes again. But since +func (app *BaseApp) SimCheck(txEncoder sdk.TxEncoder, tx sdk.Tx) (sdk.GasInfo, *sdk.Result, error) { + // runTx expects tx bytes as argument, so we encode the tx argument into + // bytes. Note that runTx will actually decode those bytes again. But since // this helper is only used in tests/simulation, it's fine. - bz, err := txEncoder(sdkTx) + bz, err := txEncoder(tx) if err != nil { return sdk.GasInfo{}, nil, sdkerrors.Wrapf(sdkerrors.ErrInvalidRequest, "%s", err) } - - ctx := app.getContextForTx(runTxModeDeliver, bz) - res, _, err := app.txHandler.CheckTx(ctx, tx.Request{Tx: sdkTx, TxBytes: bz}, tx.RequestCheckTx{Type: abci.CheckTxType_New}) - gInfo := sdk.GasInfo{GasWanted: uint64(res.GasWanted), GasUsed: uint64(res.GasUsed)} - if err != nil { - return gInfo, nil, err - } - - data, err := makeABCIData(res) - if err != nil { - return gInfo, nil, err - } - - return gInfo, &sdk.Result{Data: data, Log: res.Log, Events: res.Events, MsgResponses: res.MsgResponses}, nil + gasInfo, result, _, err := app.runTx(runTxModeCheck, bz) + return gasInfo, result, err } // Simulate executes a tx in simulate mode to get result and gas info. func (app *BaseApp) Simulate(txBytes []byte) (sdk.GasInfo, *sdk.Result, error) { - ctx := app.getContextForTx(runTxModeSimulate, txBytes) - res, err := app.txHandler.SimulateTx(ctx, tx.Request{TxBytes: txBytes}) - gasInfo := sdk.GasInfo{ - GasWanted: res.GasWanted, - GasUsed: res.GasUsed, - } - if err != nil { - return gasInfo, nil, err - } - - data, err := makeABCIData(res) - if err != nil { - return gasInfo, nil, err - } - - return gasInfo, &sdk.Result{Data: data, Log: res.Log, Events: res.Events, MsgResponses: res.MsgResponses}, nil + gasInfo, result, _, err := app.runTx(runTxModeSimulate, txBytes) + return gasInfo, result, err } -// SimDeliver defines a DeliverTx helper function that used in tests and -// simulations. -func (app *BaseApp) SimDeliver(txEncoder sdk.TxEncoder, sdkTx sdk.Tx) (sdk.GasInfo, *sdk.Result, error) { +func (app *BaseApp) SimDeliver(txEncoder sdk.TxEncoder, tx sdk.Tx) (sdk.GasInfo, *sdk.Result, error) { // See comment for Check(). - bz, err := txEncoder(sdkTx) + bz, err := txEncoder(tx) if err != nil { return sdk.GasInfo{}, nil, sdkerrors.Wrapf(sdkerrors.ErrInvalidRequest, "%s", err) } - - ctx := app.getContextForTx(runTxModeDeliver, bz) - res, err := app.txHandler.DeliverTx(ctx, tx.Request{Tx: sdkTx, TxBytes: bz}) - gInfo := sdk.GasInfo{GasWanted: uint64(res.GasWanted), GasUsed: uint64(res.GasUsed)} - if err != nil { - return gInfo, nil, err - } - - data, err := makeABCIData(res) - if err != nil { - return gInfo, nil, err - } - - return gInfo, &sdk.Result{Data: data, Log: res.Log, Events: res.Events, MsgResponses: res.MsgResponses}, nil + gasInfo, result, _, err := app.runTx(runTxModeDeliver, bz) + return gasInfo, result, err } // Context with current {check, deliver}State of the app used by tests. diff --git a/simapp/app.go b/simapp/app.go index 799a95b60513..3fe90b1b6fa6 100644 --- a/simapp/app.go +++ b/simapp/app.go @@ -213,7 +213,7 @@ func NewSimApp( legacyAmino := encodingConfig.Amino interfaceRegistry := encodingConfig.InterfaceRegistry - bApp := baseapp.NewBaseApp(appName, logger, db, baseAppOptions...) + bApp := baseapp.NewBaseApp(appName, logger, db, encodingConfig.TxConfig.TxDecoder(), baseAppOptions...) bApp.SetCommitMultiStoreTracer(traceStore) bApp.SetVersion(version.Version) bApp.SetInterfaceRegistry(interfaceRegistry) diff --git a/types/errors/abci.go b/types/errors/abci.go index fef8dfa8ab69..729e3c554085 100644 --- a/types/errors/abci.go +++ b/types/errors/abci.go @@ -17,6 +17,20 @@ func ResponseCheckTx(err error, gw, gu uint64, debug bool) abci.ResponseCheckTx } } +// ResponseCheckTxWithEvents returns an ABCI ResponseCheckTx object with fields filled in +// from the given error, gas values and events. +func ResponseCheckTxWithEvents(err error, gw, gu uint64, events []abci.Event, debug bool) abci.ResponseCheckTx { + space, code, log := ABCIInfo(err, debug) + return abci.ResponseCheckTx{ + Codespace: space, + Code: code, + Log: log, + GasWanted: int64(gw), + GasUsed: int64(gu), + Events: events, + } +} + // ResponseDeliverTx returns an ABCI ResponseDeliverTx object with fields filled in // from the given error and gas values. func ResponseDeliverTx(err error, gw, gu uint64, debug bool) abci.ResponseDeliverTx { @@ -30,6 +44,20 @@ func ResponseDeliverTx(err error, gw, gu uint64, debug bool) abci.ResponseDelive } } +// ResponseDeliverTxWithEvents returns an ABCI ResponseDeliverTx object with fields filled in +// from the given error, gas values and events. +func ResponseDeliverTxWithEvents(err error, gw, gu uint64, events []abci.Event, debug bool) abci.ResponseDeliverTx { + space, code, log := ABCIInfo(err, debug) + return abci.ResponseDeliverTx{ + Codespace: space, + Code: code, + Log: log, + GasWanted: int64(gw), + GasUsed: int64(gu), + Events: events, + } +} + // QueryResult returns a ResponseQuery from an error. It will try to parse ABCI // info from the error. func QueryResult(err error, debug bool) abci.ResponseQuery { From 2b32ad5088b0f98b1ef4d21f5d679ebc8fb89b02 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Tue, 17 May 2022 15:59:06 +0200 Subject: [PATCH 02/29] Revert middlewares --- baseapp/abci.go | 46 +- baseapp/baseapp.go | 2 +- baseapp/baseapp_test.go | 2 +- baseapp/recovery.go | 2 +- baseapp/router.go | 4 +- proto/cosmos/base/abci/v1beta1/abci.proto | 2 +- server/mock/app.go | 22 +- server/mock/tx.go | 4 +- simapp/app.go | 36 +- simapp/app_test.go | 4 +- types/tx/middleware.go | 71 --- x/auth/ante/ante.go | 58 ++ .../middleware_test.go => ante/ante_test.go} | 385 +++++++------ x/auth/ante/basic.go | 206 +++++++ x/auth/ante/basic_test.go | 224 ++++++++ .../{middleware => ante}/expected_keepers.go | 4 +- x/auth/ante/ext.go | 36 ++ x/auth/ante/ext_test.go | 36 ++ x/auth/ante/fee.go | 140 +++++ x/auth/ante/fee_test.go | 104 ++++ x/auth/{middleware => ante}/feegrant_test.go | 65 +-- x/auth/ante/setup.go | 76 +++ x/auth/ante/setup_test.go | 99 ++++ x/auth/{middleware => ante}/sigverify.go | 538 ++++++------------ .../sigverify_benchmark_test.go | 4 +- x/auth/{middleware => ante}/sigverify_test.go | 296 +++++----- x/auth/ante/testutil_test.go | 200 +++++++ x/auth/middleware/basic.go | 359 ------------ x/auth/middleware/basic_test.go | 252 -------- x/auth/middleware/block_gas.go | 53 -- x/auth/middleware/branch_store.go | 70 --- x/auth/middleware/branch_store_test.go | 129 ----- x/auth/middleware/ext.go | 86 --- x/auth/middleware/ext_test.go | 54 -- x/auth/middleware/fee.go | 153 ----- x/auth/middleware/fee_test.go | 126 ---- x/auth/middleware/gas.go | 96 ---- x/auth/middleware/gas_test.go | 127 ----- x/auth/middleware/index_events.go | 61 -- x/auth/middleware/legacy_router.go | 41 -- x/auth/middleware/legacy_router_test.go | 32 -- x/auth/middleware/middleware.go | 115 ---- x/auth/middleware/msg_service_router.go | 136 ----- x/auth/middleware/msg_service_router_test.go | 55 -- x/auth/middleware/recovery.go | 78 --- x/auth/middleware/run_msgs.go | 117 ---- x/auth/middleware/run_msgs_test.go | 40 -- x/auth/middleware/testutil_test.go | 222 -------- x/auth/middleware/tips.go | 69 --- x/auth/middleware/tips_test.go | 206 ------- x/auth/middleware/tx.go | 77 --- x/auth/middleware/tx_test.go | 64 --- x/auth/middleware/validator_tx_fee.go | 59 -- x/auth/signing/verify_test.go | 4 +- x/auth/tx/builder.go | 4 +- x/authz/keeper/keeper.go | 6 +- x/feegrant/keeper/keeper.go | 4 +- x/gov/keeper/keeper.go | 8 +- x/group/keeper/keeper.go | 6 +- x/group/keeper/proposal_executor.go | 4 +- 60 files changed, 1791 insertions(+), 3788 deletions(-) delete mode 100644 types/tx/middleware.go create mode 100644 x/auth/ante/ante.go rename x/auth/{middleware/middleware_test.go => ante/ante_test.go} (66%) create mode 100644 x/auth/ante/basic.go create mode 100644 x/auth/ante/basic_test.go rename x/auth/{middleware => ante}/expected_keepers.go (85%) create mode 100644 x/auth/ante/ext.go create mode 100644 x/auth/ante/ext_test.go create mode 100644 x/auth/ante/fee.go create mode 100644 x/auth/ante/fee_test.go rename x/auth/{middleware => ante}/feegrant_test.go (78%) create mode 100644 x/auth/ante/setup.go create mode 100644 x/auth/ante/setup_test.go rename x/auth/{middleware => ante}/sigverify.go (51%) rename x/auth/{middleware => ante}/sigverify_benchmark_test.go (89%) rename x/auth/{middleware => ante}/sigverify_test.go (52%) create mode 100644 x/auth/ante/testutil_test.go delete mode 100644 x/auth/middleware/basic.go delete mode 100644 x/auth/middleware/basic_test.go delete mode 100644 x/auth/middleware/block_gas.go delete mode 100644 x/auth/middleware/branch_store.go delete mode 100644 x/auth/middleware/branch_store_test.go delete mode 100644 x/auth/middleware/ext.go delete mode 100644 x/auth/middleware/ext_test.go delete mode 100644 x/auth/middleware/fee.go delete mode 100644 x/auth/middleware/fee_test.go delete mode 100644 x/auth/middleware/gas.go delete mode 100644 x/auth/middleware/gas_test.go delete mode 100644 x/auth/middleware/index_events.go delete mode 100644 x/auth/middleware/legacy_router.go delete mode 100644 x/auth/middleware/legacy_router_test.go delete mode 100644 x/auth/middleware/middleware.go delete mode 100644 x/auth/middleware/msg_service_router.go delete mode 100644 x/auth/middleware/msg_service_router_test.go delete mode 100644 x/auth/middleware/recovery.go delete mode 100644 x/auth/middleware/run_msgs.go delete mode 100644 x/auth/middleware/run_msgs_test.go delete mode 100644 x/auth/middleware/testutil_test.go delete mode 100644 x/auth/middleware/tips.go delete mode 100644 x/auth/middleware/tips_test.go delete mode 100644 x/auth/middleware/tx.go delete mode 100644 x/auth/middleware/tx_test.go delete mode 100644 x/auth/middleware/validator_tx_fee.go diff --git a/baseapp/abci.go b/baseapp/abci.go index a5898b119fcf..260ff008f09a 100644 --- a/baseapp/abci.go +++ b/baseapp/abci.go @@ -21,7 +21,6 @@ import ( "github.com/cosmos/cosmos-sdk/telemetry" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/tx" ) // Supported ABCI Query prefixes @@ -262,11 +261,16 @@ func (app *BaseApp) CheckTx(req abci.RequestCheckTx) abci.ResponseCheckTx { return sdkerrors.ResponseCheckTxWithEvents(err, gInfo.GasWanted, gInfo.GasUsed, anteEvents, app.trace) } + data, err := makeABCIData(result) + if err != nil { + return sdkerrors.ResponseCheckTxWithEvents(err, gInfo.GasWanted, gInfo.GasUsed, anteEvents, app.trace) + } + return abci.ResponseCheckTx{ GasWanted: int64(gInfo.GasWanted), // TODO: Should type accept unsigned ints? GasUsed: int64(gInfo.GasUsed), // TODO: Should type accept unsigned ints? Log: result.Log, - Data: result.Data, + Data: data, Events: sdk.MarkEventsToIndex(result.Events, app.indexEvents), } } @@ -295,11 +299,16 @@ func (app *BaseApp) DeliverTx(req abci.RequestDeliverTx) abci.ResponseDeliverTx return sdkerrors.ResponseDeliverTxWithEvents(err, gInfo.GasWanted, gInfo.GasUsed, anteEvents, app.trace) } + data, err := makeABCIData(result) + if err != nil { + return sdkerrors.ResponseDeliverTxWithEvents(err, gInfo.GasWanted, gInfo.GasUsed, anteEvents, app.trace) + } + return abci.ResponseDeliverTx{ GasWanted: int64(gInfo.GasWanted), // TODO: Should type accept unsigned ints? GasUsed: int64(gInfo.GasUsed), // TODO: Should type accept unsigned ints? Log: result.Log, - Data: result.Data, + Data: data, Events: sdk.MarkEventsToIndex(result.Events, app.indexEvents), } } @@ -865,35 +874,6 @@ func SplitABCIQueryPath(requestPath string) (path []string) { } // makeABCIData generates the Data field to be sent to ABCI Check/DeliverTx. -func makeABCIData(txRes tx.Response) ([]byte, error) { +func makeABCIData(txRes *sdk.Result) ([]byte, error) { return proto.Marshal(&sdk.TxMsgData{MsgResponses: txRes.MsgResponses}) } - -// convertTxResponseToCheckTx converts a tx.Response into a abci.ResponseCheckTx. -func convertTxResponseToCheckTx(txRes tx.Response, checkRes tx.ResponseCheckTx) (abci.ResponseCheckTx, error) { - data, err := makeABCIData(txRes) - if err != nil { - return abci.ResponseCheckTx{}, nil - } - - return abci.ResponseCheckTx{ - Data: data, - Log: txRes.Log, - Events: txRes.Events, - Priority: checkRes.Priority, - }, nil -} - -// convertTxResponseToDeliverTx converts a tx.Response into a abci.ResponseDeliverTx. -func convertTxResponseToDeliverTx(txRes tx.Response) (abci.ResponseDeliverTx, error) { - data, err := makeABCIData(txRes) - if err != nil { - return abci.ResponseDeliverTx{}, nil - } - - return abci.ResponseDeliverTx{ - Data: data, - Log: txRes.Log, - Events: txRes.Events, - }, nil -} diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index 9f62cb5e6120..f629626aa33a 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -150,7 +150,7 @@ func NewBaseApp( db: db, cms: store.NewCommitMultiStore(db), storeLoader: DefaultStoreLoader, - router: NewRouter(), + router: NewLegacyRouter(), queryRouter: NewQueryRouter(), grpcQueryRouter: NewGRPCQueryRouter(), msgServiceRouter: NewMsgServiceRouter(), diff --git a/baseapp/baseapp_test.go b/baseapp/baseapp_test.go index beaaafb8641d..f12557e8dc8c 100644 --- a/baseapp/baseapp_test.go +++ b/baseapp/baseapp_test.go @@ -29,7 +29,7 @@ import ( "github.com/cosmos/cosmos-sdk/testutil/testdata" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/x/auth/legacy/legacytx" + "github.com/cosmos/cosmos-sdk/x/auth/migrations/legacytx" ) var ( diff --git a/baseapp/recovery.go b/baseapp/recovery.go index 7f0687800c65..8094d3dcda9c 100644 --- a/baseapp/recovery.go +++ b/baseapp/recovery.go @@ -33,7 +33,7 @@ func processRecovery(recoveryObj interface{}, middleware recoveryMiddleware) err return processRecovery(recoveryObj, next) } -// newRecoveryMiddleware creates a RecoveryHandler middleware. +// newRecoveryMiddleware creates a RecoveryHandler ante. func newRecoveryMiddleware(handler RecoveryHandler, next recoveryMiddleware) recoveryMiddleware { return func(recoveryObj interface{}) (recoveryMiddleware, error) { if err := handler(recoveryObj); err != nil { diff --git a/baseapp/router.go b/baseapp/router.go index 7e2e70a0c6f3..290d60afbc59 100644 --- a/baseapp/router.go +++ b/baseapp/router.go @@ -10,10 +10,10 @@ type Router struct { routes map[string]sdk.Handler } -var _ sdk.Router = NewRouter() +var _ sdk.Router = NewLegacyRouter() // NewRouter returns a reference to a new router. -func NewRouter() *Router { +func NewLegacyRouter() *Router { return &Router{ routes: make(map[string]sdk.Handler), } diff --git a/proto/cosmos/base/abci/v1beta1/abci.proto b/proto/cosmos/base/abci/v1beta1/abci.proto index a9de5258ddba..ddaa63561774 100644 --- a/proto/cosmos/base/abci/v1beta1/abci.proto +++ b/proto/cosmos/base/abci/v1beta1/abci.proto @@ -41,7 +41,7 @@ message TxResponse { string timestamp = 12; // Events defines all the events emitted by processing a transaction. Note, // these events include those emitted by processing all the messages and those - // emitted from the middleware. Whereas Logs contains the events, with + // emitted from the ante. Whereas Logs contains the events, with // additional metadata, emitted only by processing the messages. // // Since: cosmos-sdk 0.42.11, 0.44.5, 0.45 diff --git a/server/mock/app.go b/server/mock/app.go index 9fd4d72f480e..01ea62b09318 100644 --- a/server/mock/app.go +++ b/server/mock/app.go @@ -18,16 +18,16 @@ import ( storetypes "github.com/cosmos/cosmos-sdk/store/types" sdk "github.com/cosmos/cosmos-sdk/types" "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" + "github.com/cosmos/cosmos-sdk/x/auth/ante" ) -func testTxHandler(options middleware.TxHandlerOptions) tx.Handler { - return middleware.ComposeMiddlewares( - middleware.NewRunMsgsTxHandler(options.MsgServiceRouter, options.LegacyRouter), - middleware.NewTxDecoderMiddleware(options.TxDecoder), - middleware.GasTxMiddleware, - middleware.RecoveryTxMiddleware, - middleware.NewIndexEventsTxMiddleware(options.IndexEvents), +func testTxHandler(options ante.TxHandlerOptions) tx.Handler { + return ante.ComposeMiddlewares( + ante.NewRunMsgsTxHandler(options.MsgServiceRouter, options.LegacyRouter), + ante.NewTxDecoderMiddleware(options.TxDecoder), + ante.GasTxMiddleware, + ante.RecoveryTxMiddleware, + ante.NewIndexEventsTxMiddleware(options.IndexEvents), ) } @@ -53,14 +53,14 @@ func NewApp(rootDir string, logger log.Logger) (abci.Application, error) { // Set a Route. encCfg := simapp.MakeTestEncodingConfig() - legacyRouter := middleware.NewLegacyRouter() + legacyRouter := ante.NewLegacyRouter() // We're adding a test legacy route here, which accesses the kvstore // and simply sets the Msg's key/value pair in the kvstore. legacyRouter.AddRoute(sdk.NewRoute("kvstore", KVStoreHandler(capKeyMainStore))) txHandler := testTxHandler( - middleware.TxHandlerOptions{ + ante.TxHandlerOptions{ LegacyRouter: legacyRouter, - MsgServiceRouter: middleware.NewMsgServiceRouter(encCfg.InterfaceRegistry), + MsgServiceRouter: ante.NewMsgServiceRouter(encCfg.InterfaceRegistry), TxDecoder: decodeTx, }, ) diff --git a/server/mock/tx.go b/server/mock/tx.go index bfa6e6ad6b4b..90ef3981d93f 100644 --- a/server/mock/tx.go +++ b/server/mock/tx.go @@ -8,7 +8,7 @@ import ( sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" + "github.com/cosmos/cosmos-sdk/x/auth/ante" ) // kvstoreTx defines a tx for mock purposes. The `key` and `value` fields will @@ -27,7 +27,7 @@ func (msg *kvstoreTx) ProtoMessage() {} var _ sdk.Tx = &kvstoreTx{} var _ sdk.Msg = &kvstoreTx{} -var _ middleware.GasTx = &kvstoreTx{} +var _ ante.GasTx = &kvstoreTx{} func NewTx(key, value string) kvstoreTx { bytes := fmt.Sprintf("%s=%s", key, value) diff --git a/simapp/app.go b/simapp/app.go index 3fe90b1b6fa6..e9c61d2640c5 100644 --- a/simapp/app.go +++ b/simapp/app.go @@ -32,8 +32,8 @@ import ( "github.com/cosmos/cosmos-sdk/types/module" "github.com/cosmos/cosmos-sdk/version" "github.com/cosmos/cosmos-sdk/x/auth" + "github.com/cosmos/cosmos-sdk/x/auth/ante" authkeeper "github.com/cosmos/cosmos-sdk/x/auth/keeper" - authmiddleware "github.com/cosmos/cosmos-sdk/x/auth/middleware" authsims "github.com/cosmos/cosmos-sdk/x/auth/simulation" authtx "github.com/cosmos/cosmos-sdk/x/auth/tx" authtypes "github.com/cosmos/cosmos-sdk/x/auth/types" @@ -155,7 +155,7 @@ type SimApp struct { legacyAmino *codec.LegacyAmino appCodec codec.Codec interfaceRegistry types.InterfaceRegistry - msgSvcRouter *authmiddleware.MsgServiceRouter + msgSvcRouter *baseapp.MsgServiceRouter legacyRouter sdk.Router invCheckPeriod uint @@ -241,8 +241,8 @@ func NewSimApp( legacyAmino: legacyAmino, appCodec: appCodec, interfaceRegistry: interfaceRegistry, - legacyRouter: authmiddleware.NewLegacyRouter(), - msgSvcRouter: authmiddleware.NewMsgServiceRouter(interfaceRegistry), + legacyRouter: baseapp.NewLegacyRouter(), + msgSvcRouter: baseapp.NewMsgServiceRouter(), invCheckPeriod: invCheckPeriod, keys: keys, tkeys: tkeys, @@ -447,7 +447,7 @@ func NewSimApp( app.SetInitChainer(app.InitChainer) app.SetBeginBlocker(app.BeginBlocker) app.SetEndBlocker(app.EndBlocker) - app.setTxHandler(encodingConfig.TxConfig, cast.ToStringSlice(appOpts.Get(server.FlagIndexEvents))) + app.setAnteHandler(encodingConfig.TxConfig, cast.ToStringSlice(appOpts.Get(server.FlagIndexEvents))) if loadLatest { if err := app.LoadLatestVersion(); err != nil { @@ -458,28 +458,26 @@ func NewSimApp( return app } -func (app *SimApp) setTxHandler(txConfig client.TxConfig, indexEventsStr []string) { +func (app *SimApp) setAnteHandler(txConfig client.TxConfig, indexEventsStr []string) { indexEvents := map[string]struct{}{} for _, e := range indexEventsStr { indexEvents[e] = struct{}{} } - txHandler, err := authmiddleware.NewDefaultTxHandler(authmiddleware.TxHandlerOptions{ - Debug: app.Trace(), - IndexEvents: indexEvents, - LegacyRouter: app.legacyRouter, - MsgServiceRouter: app.msgSvcRouter, - AccountKeeper: app.AccountKeeper, - BankKeeper: app.BankKeeper, - FeegrantKeeper: app.FeeGrantKeeper, - SignModeHandler: txConfig.SignModeHandler(), - SigGasConsumer: authmiddleware.DefaultSigVerificationGasConsumer, - TxDecoder: txConfig.TxDecoder(), - }) + anteHandler, err := ante.NewAnteHandler( + ante.HandlerOptions{ + AccountKeeper: app.AccountKeeper, + BankKeeper: app.BankKeeper, + SignModeHandler: txConfig.SignModeHandler(), + FeegrantKeeper: app.FeeGrantKeeper, + SigGasConsumer: ante.DefaultSigVerificationGasConsumer, + }, + ) + if err != nil { panic(err) } - app.SetTxHandler(txHandler) + app.SetAnteHandler(anteHandler) } // Name returns the name of the App diff --git a/simapp/app_test.go b/simapp/app_test.go index 72acc777d9eb..ad8cd8b6b781 100644 --- a/simapp/app_test.go +++ b/simapp/app_test.go @@ -16,7 +16,7 @@ import ( sdk "github.com/cosmos/cosmos-sdk/types" "github.com/cosmos/cosmos-sdk/types/module" "github.com/cosmos/cosmos-sdk/x/auth" - authmiddleware "github.com/cosmos/cosmos-sdk/x/auth/middleware" + authmiddleware "github.com/cosmos/cosmos-sdk/x/auth/ante" "github.com/cosmos/cosmos-sdk/x/auth/vesting" authzmodule "github.com/cosmos/cosmos-sdk/x/authz/module" "github.com/cosmos/cosmos-sdk/x/bank" @@ -82,7 +82,7 @@ func TestRunMigrations(t *testing.T) { bApp := baseapp.NewBaseApp(appName, logger, db) bApp.SetCommitMultiStoreTracer(nil) bApp.SetInterfaceRegistry(encCfg.InterfaceRegistry) - msr := authmiddleware.NewMsgServiceRouter(encCfg.InterfaceRegistry) + msr := authante.NewMsgServiceRouter(encCfg.InterfaceRegistry) app.BaseApp = bApp app.configurator = module.NewConfigurator(app.appCodec, msr, app.GRPCQueryRouter()) diff --git a/types/tx/middleware.go b/types/tx/middleware.go deleted file mode 100644 index abe29be672d8..000000000000 --- a/types/tx/middleware.go +++ /dev/null @@ -1,71 +0,0 @@ -package tx - -import ( - context "context" - - codectypes "github.com/cosmos/cosmos-sdk/codec/types" - abci "github.com/tendermint/tendermint/abci/types" - - sdk "github.com/cosmos/cosmos-sdk/types" -) - -// RequestSimulateTx is the request type for the tx.Handler.RequestSimulateTx -// method. -type RequestSimulateTx struct { - TxBytes []byte -} - -// ResponseSimulateTx is the response type for the tx.Handler.RequestSimulateTx -// method. -type ResponseSimulateTx struct { - GasInfo sdk.GasInfo - Result *sdk.Result -} - -// Request is the tx request type used in middlewares. -// At least one of Tx or TxBytes must be set. If only TxBytes is set, then -// Tx will be populated by the TxDecoderMiddleware. If only Tx is set, then -// some middlewares (such as signature verification) will fail. -// -// In practice, the middleware stack is called from {Check,Deliver}Tx, which -// only passes the TxBytes. Then, the TxDecoderMiddleware decodes the bytes -// into the Tx field. -type Request struct { - Tx sdk.Tx - TxBytes []byte -} - -// Response is the tx response type used in middlewares. -type Response struct { - GasWanted uint64 - GasUsed uint64 - // MsgResponses is an array containing each Msg service handler's response - // type, packed in an Any. This will get proto-serialized into the `Data` field - // in the ABCI Check/DeliverTx responses. - MsgResponses []*codectypes.Any - Log string - Events []abci.Event -} - -// RequestCheckTx is the additional request type used in middlewares CheckTx -// method. -type RequestCheckTx struct { - Type abci.CheckTxType -} - -// RequestCheckTx is the additional response type used in middlewares CheckTx -// method. -type ResponseCheckTx struct { - Priority int64 -} - -// TxHandler defines the baseapp's CheckTx, DeliverTx and Simulate respective -// handlers. It is designed as a middleware stack. -type Handler interface { - CheckTx(ctx context.Context, req Request, checkReq RequestCheckTx) (Response, ResponseCheckTx, error) - DeliverTx(ctx context.Context, req Request) (Response, error) - SimulateTx(ctx context.Context, req Request) (Response, error) -} - -// TxMiddleware defines one layer of the TxHandler middleware stack. -type Middleware func(Handler) Handler diff --git a/x/auth/ante/ante.go b/x/auth/ante/ante.go new file mode 100644 index 000000000000..3b4aa6a56f11 --- /dev/null +++ b/x/auth/ante/ante.go @@ -0,0 +1,58 @@ +package ante + +import ( + sdk "github.com/cosmos/cosmos-sdk/types" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" + "github.com/cosmos/cosmos-sdk/types/tx/signing" + authsigning "github.com/cosmos/cosmos-sdk/x/auth/signing" + "github.com/cosmos/cosmos-sdk/x/auth/types" +) + +// HandlerOptions are the options required for constructing a default SDK AnteHandler. +type HandlerOptions struct { + AccountKeeper AccountKeeper + BankKeeper types.BankKeeper + FeegrantKeeper FeegrantKeeper + SignModeHandler authsigning.SignModeHandler + SigGasConsumer func(meter sdk.GasMeter, sig signing.SignatureV2, params types.Params) error +} + +// NewAnteHandler returns an AnteHandler that checks and increments sequence +// numbers, checks signatures & account numbers, and deducts fees from the first +// signer. +func NewAnteHandler(options HandlerOptions) (sdk.AnteHandler, error) { + if options.AccountKeeper == nil { + return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "account keeper is required for ante builder") + } + + if options.BankKeeper == nil { + return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "bank keeper is required for ante builder") + } + + if options.SignModeHandler == nil { + return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "sign mode handler is required for ante builder") + } + + var sigGasConsumer = options.SigGasConsumer + if sigGasConsumer == nil { + sigGasConsumer = DefaultSigVerificationGasConsumer + } + + anteDecorators := []sdk.AnteDecorator{ + NewSetUpContextDecorator(), // outermost AnteDecorator. SetUpContext must be called first + NewRejectExtensionOptionsDecorator(), + NewMempoolFeeDecorator(), + NewValidateBasicDecorator(), + NewTxTimeoutHeightDecorator(), + NewValidateMemoDecorator(options.AccountKeeper), + NewConsumeGasForTxSizeDecorator(options.AccountKeeper), + NewDeductFeeDecorator(options.AccountKeeper, options.BankKeeper, options.FeegrantKeeper), + NewSetPubKeyDecorator(options.AccountKeeper), // SetPubKeyDecorator must be called before all signature verification decorators + NewValidateSigCountDecorator(options.AccountKeeper), + NewSigGasConsumeDecorator(options.AccountKeeper, sigGasConsumer), + NewSigVerificationDecorator(options.AccountKeeper, options.SignModeHandler), + NewIncrementSequenceDecorator(options.AccountKeeper), + } + + return sdk.ChainAnteDecorators(anteDecorators...), nil +} diff --git a/x/auth/middleware/middleware_test.go b/x/auth/ante/ante_test.go similarity index 66% rename from x/auth/middleware/middleware_test.go rename to x/auth/ante/ante_test.go index d923b84b2c3d..d2af80904476 100644 --- a/x/auth/middleware/middleware_test.go +++ b/x/auth/ante/ante_test.go @@ -1,4 +1,4 @@ -package middleware_test +package ante_test import ( "encoding/json" @@ -7,6 +7,12 @@ import ( "strings" "testing" + "github.com/cosmos/cosmos-sdk/simapp" + + minttypes "github.com/cosmos/cosmos-sdk/x/mint/types" + + "github.com/stretchr/testify/require" + "github.com/cosmos/cosmos-sdk/crypto/keys/ed25519" kmultisig "github.com/cosmos/cosmos-sdk/crypto/keys/multisig" "github.com/cosmos/cosmos-sdk/crypto/keys/secp256k1" @@ -14,25 +20,17 @@ import ( "github.com/cosmos/cosmos-sdk/testutil/testdata" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/tx" "github.com/cosmos/cosmos-sdk/types/tx/signing" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" + "github.com/cosmos/cosmos-sdk/x/auth/ante" "github.com/cosmos/cosmos-sdk/x/auth/types" - "github.com/cosmos/cosmos-sdk/x/bank/testutil" - minttypes "github.com/cosmos/cosmos-sdk/x/mint/types" - "github.com/stretchr/testify/require" - abci "github.com/tendermint/tendermint/abci/types" ) -var testCoins = sdk.Coins{sdk.NewInt64Coin("atom", 10000000)} - // Test that simulate transaction accurately estimates gas cost -func (s *MWTestSuite) TestSimulateGasCost() { - ctx := s.SetupTest(false) // reset - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() +func (suite *AnteTestSuite) TestSimulateGasCost() { + suite.SetupTest(false) // reset // Same data for every test cases - accounts := s.createTestAccounts(ctx, 3, testCoins) + accounts := suite.CreateTestAccounts(3) msgs := []sdk.Msg{ testdata.NewTestMsg(accounts[0].acc.GetAddress(), accounts[1].acc.GetAddress()), testdata.NewTestMsg(accounts[2].acc.GetAddress(), accounts[0].acc.GetAddress()), @@ -48,8 +46,8 @@ func (s *MWTestSuite) TestSimulateGasCost() { { "tx with 150atom fee", func() { - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(gasLimit) + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(gasLimit) }, true, true, @@ -58,11 +56,11 @@ func (s *MWTestSuite) TestSimulateGasCost() { { "with previously estimated gas", func() { - simulatedGas := ctx.GasMeter().GasConsumed() + simulatedGas := suite.ctx.GasMeter().GasConsumed() accSeqs = []uint64{1, 1, 1} - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(simulatedGas) + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(simulatedGas) }, false, true, @@ -71,18 +69,18 @@ func (s *MWTestSuite) TestSimulateGasCost() { } for _, tc := range testCases { - s.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() tc.malleate() - s.runTestCase(ctx, txBuilder, privs, msgs, feeAmount, gasLimit, accNums, accSeqs, ctx.ChainID(), tc) + suite.RunTestCase(privs, msgs, feeAmount, gasLimit, accNums, accSeqs, suite.ctx.ChainID(), tc) }) } } -// Test various error cases in the TxHandler control flow. -func (s *MWTestSuite) TestTxHandlerSigErrors() { - ctx := s.SetupTest(false) // reset - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() +// Test various error cases in the AnteHandler control flow. +func (suite *AnteTestSuite) TestAnteHandlerSigErrors() { + suite.SetupTest(false) // reset // Same data for every test cases priv0, _, addr0 := testdata.KeyTestPubAddr() @@ -109,12 +107,12 @@ func (s *MWTestSuite) TestTxHandlerSigErrors() { privs, accNums, accSeqs = []cryptotypes.PrivKey{}, []uint64{}, []uint64{} // Create tx manually to test the tx's signers - s.Require().NoError(txBuilder.SetMsgs(msgs...)) - tx, _, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) + suite.Require().NoError(suite.txBuilder.SetMsgs(msgs...)) + tx, err := suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) // tx.GetSigners returns addresses in correct order: addr1, addr2, addr3 expectedSigners := []sdk.AccAddress{addr0, addr1, addr2} - s.Require().Equal(expectedSigners, tx.GetSigners()) + suite.Require().Equal(expectedSigners, tx.GetSigners()) }, false, false, @@ -141,12 +139,12 @@ func (s *MWTestSuite) TestTxHandlerSigErrors() { { "save the first account, but second is still unrecognized", func() { - acc1 := s.app.AccountKeeper.NewAccountWithAddress(ctx, addr0) - s.app.AccountKeeper.SetAccount(ctx, acc1) - err := s.app.BankKeeper.MintCoins(ctx, minttypes.ModuleName, feeAmount) - s.Require().NoError(err) - err = s.app.BankKeeper.SendCoinsFromModuleToAccount(ctx, minttypes.ModuleName, addr0, feeAmount) - s.Require().NoError(err) + acc1 := suite.app.AccountKeeper.NewAccountWithAddress(suite.ctx, addr0) + suite.app.AccountKeeper.SetAccount(suite.ctx, acc1) + err := suite.app.BankKeeper.MintCoins(suite.ctx, minttypes.ModuleName, feeAmount) + suite.Require().NoError(err) + err = suite.app.BankKeeper.SendCoinsFromModuleToAccount(suite.ctx, minttypes.ModuleName, addr0, feeAmount) + suite.Require().NoError(err) }, false, false, @@ -155,21 +153,21 @@ func (s *MWTestSuite) TestTxHandlerSigErrors() { } for _, tc := range testCases { - s.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() tc.malleate() - s.runTestCase(ctx, txBuilder, privs, msgs, feeAmount, gasLimit, accNums, accSeqs, ctx.ChainID(), tc) + suite.RunTestCase(privs, msgs, feeAmount, gasLimit, accNums, accSeqs, suite.ctx.ChainID(), tc) }) } } // Test logic around account number checking with one signer and many signers. -func (s *MWTestSuite) TestTxHandlerAccountNumbers() { - ctx := s.SetupTest(false) // reset - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() +func (suite *AnteTestSuite) TestAnteHandlerAccountNumbers() { + suite.SetupTest(false) // reset // Same data for every test cases - accounts := s.createTestAccounts(ctx, 2, testCoins) + accounts := suite.CreateTestAccounts(2) feeAmount := testdata.NewTestFeeAmount() gasLimit := testdata.NewTestGasLimit() @@ -236,22 +234,22 @@ func (s *MWTestSuite) TestTxHandlerAccountNumbers() { } for _, tc := range testCases { - s.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() tc.malleate() - s.runTestCase(ctx, txBuilder, privs, msgs, feeAmount, gasLimit, accNums, accSeqs, ctx.ChainID(), tc) + suite.RunTestCase(privs, msgs, feeAmount, gasLimit, accNums, accSeqs, suite.ctx.ChainID(), tc) }) } } // Test logic around account number checking with many signers when BlockHeight is 0. -func (s *MWTestSuite) TestTxHandlerAccountNumbersAtBlockHeightZero() { - ctx := s.SetupTest(false) // setup - ctx = ctx.WithBlockHeight(0) - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() +func (suite *AnteTestSuite) TestAnteHandlerAccountNumbersAtBlockHeightZero() { + suite.SetupTest(false) // setup + suite.ctx = suite.ctx.WithBlockHeight(0) // Same data for every test cases - accounts := s.createTestAccounts(ctx, 2, testCoins) + accounts := suite.CreateTestAccounts(2) feeAmount := testdata.NewTestFeeAmount() gasLimit := testdata.NewTestGasLimit() @@ -320,21 +318,21 @@ func (s *MWTestSuite) TestTxHandlerAccountNumbersAtBlockHeightZero() { } for _, tc := range testCases { - s.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() tc.malleate() - s.runTestCase(ctx, txBuilder, privs, msgs, feeAmount, gasLimit, accNums, accSeqs, ctx.ChainID(), tc) + suite.RunTestCase(privs, msgs, feeAmount, gasLimit, accNums, accSeqs, suite.ctx.ChainID(), tc) }) } } // Test logic around sequence checking with one signer and many signers. -func (s *MWTestSuite) TestTxHandlerSequences() { - ctx := s.SetupTest(false) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() +func (suite *AnteTestSuite) TestAnteHandlerSequences() { + suite.SetupTest(false) // setup // Same data for every test cases - accounts := s.createTestAccounts(ctx, 3, testCoins) + accounts := suite.CreateTestAccounts(3) feeAmount := testdata.NewTestFeeAmount() gasLimit := testdata.NewTestGasLimit() @@ -432,24 +430,24 @@ func (s *MWTestSuite) TestTxHandlerSequences() { } for _, tc := range testCases { - s.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() tc.malleate() - s.runTestCase(ctx, txBuilder, privs, msgs, feeAmount, gasLimit, accNums, accSeqs, ctx.ChainID(), tc) + suite.RunTestCase(privs, msgs, feeAmount, gasLimit, accNums, accSeqs, suite.ctx.ChainID(), tc) }) } } // Test logic around fee deduction. -func (s *MWTestSuite) TestTxHandlerFees() { - ctx := s.SetupTest(false) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() +func (suite *AnteTestSuite) TestAnteHandlerFees() { + suite.SetupTest(false) // setup // Same data for every test cases priv0, _, addr0 := testdata.KeyTestPubAddr() - acc1 := s.app.AccountKeeper.NewAccountWithAddress(ctx, addr0) - s.app.AccountKeeper.SetAccount(ctx, acc1) + acc1 := suite.app.AccountKeeper.NewAccountWithAddress(suite.ctx, addr0) + suite.app.AccountKeeper.SetAccount(suite.ctx, acc1) msgs := []sdk.Msg{testdata.NewTestMsg(addr0)} feeAmount := testdata.NewTestFeeAmount() gasLimit := testdata.NewTestGasLimit() @@ -474,8 +472,8 @@ func (s *MWTestSuite) TestTxHandlerFees() { { "signer does not have enough funds to pay the fee", func() { - err := testutil.FundAccount(s.app.BankKeeper, ctx, addr0, sdk.NewCoins(sdk.NewInt64Coin("atom", 149))) - s.Require().NoError(err) + err := simapp.FundAccount(suite.app.BankKeeper, suite.ctx, addr0, sdk.NewCoins(sdk.NewInt64Coin("atom", 149))) + suite.Require().NoError(err) }, false, false, @@ -486,13 +484,13 @@ func (s *MWTestSuite) TestTxHandlerFees() { func() { accNums = []uint64{acc1.GetAccountNumber()} - modAcc := s.app.AccountKeeper.GetModuleAccount(ctx, types.FeeCollectorName) + modAcc := suite.app.AccountKeeper.GetModuleAccount(suite.ctx, types.FeeCollectorName) - s.Require().True(s.app.BankKeeper.GetAllBalances(ctx, modAcc.GetAddress()).Empty()) - require.True(sdk.IntEq(s.T(), s.app.BankKeeper.GetAllBalances(ctx, addr0).AmountOf("atom"), sdk.NewInt(149))) + suite.Require().True(suite.app.BankKeeper.GetAllBalances(suite.ctx, modAcc.GetAddress()).Empty()) + require.True(sdk.IntEq(suite.T(), suite.app.BankKeeper.GetAllBalances(suite.ctx, addr0).AmountOf("atom"), sdk.NewInt(149))) - err := testutil.FundAccount(s.app.BankKeeper, ctx, addr0, sdk.NewCoins(sdk.NewInt64Coin("atom", 1))) - s.Require().NoError(err) + err := simapp.FundAccount(suite.app.BankKeeper, suite.ctx, addr0, sdk.NewCoins(sdk.NewInt64Coin("atom", 1))) + suite.Require().NoError(err) }, false, true, @@ -501,10 +499,10 @@ func (s *MWTestSuite) TestTxHandlerFees() { { "signer doesn't have any more funds", func() { - modAcc := s.app.AccountKeeper.GetModuleAccount(ctx, types.FeeCollectorName) + modAcc := suite.app.AccountKeeper.GetModuleAccount(suite.ctx, types.FeeCollectorName) - require.True(sdk.IntEq(s.T(), s.app.BankKeeper.GetAllBalances(ctx, modAcc.GetAddress()).AmountOf("atom"), sdk.NewInt(150))) - require.True(sdk.IntEq(s.T(), s.app.BankKeeper.GetAllBalances(ctx, addr0).AmountOf("atom"), sdk.NewInt(0))) + require.True(sdk.IntEq(suite.T(), suite.app.BankKeeper.GetAllBalances(suite.ctx, modAcc.GetAddress()).AmountOf("atom"), sdk.NewInt(150))) + require.True(sdk.IntEq(suite.T(), suite.app.BankKeeper.GetAllBalances(suite.ctx, addr0).AmountOf("atom"), sdk.NewInt(0))) }, false, false, @@ -513,21 +511,22 @@ func (s *MWTestSuite) TestTxHandlerFees() { } for _, tc := range testCases { - s.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.Run(fmt.Sprintf("Case %s", tc.desc), func() { + + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() tc.malleate() - s.runTestCase(ctx, txBuilder, privs, msgs, feeAmount, gasLimit, accNums, accSeqs, ctx.ChainID(), tc) + suite.RunTestCase(privs, msgs, feeAmount, gasLimit, accNums, accSeqs, suite.ctx.ChainID(), tc) }) } } // Test logic around memo gas consumption. -func (s *MWTestSuite) TestTxHandlerMemoGas() { - ctx := s.SetupTest(false) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() +func (suite *AnteTestSuite) TestAnteHandlerMemoGas() { + suite.SetupTest(false) // setup // Same data for every test cases - accounts := s.createTestAccounts(ctx, 1, testCoins) + accounts := suite.CreateTestAccounts(1) msgs := []sdk.Msg{testdata.NewTestMsg(accounts[0].acc.GetAddress())} privs, accNums, accSeqs := []cryptotypes.PrivKey{accounts[0].priv}, []uint64{0}, []uint64{0} @@ -553,7 +552,7 @@ func (s *MWTestSuite) TestTxHandlerMemoGas() { func() { feeAmount = sdk.NewCoins(sdk.NewInt64Coin("atom", 0)) gasLimit = 801 - txBuilder.SetMemo("abcininasidniandsinasindiansdiansdinaisndiasndiadninsd") + suite.txBuilder.SetMemo("abcininasidniandsinasindiansdiansdinaisndiasndiadninsd") }, false, false, @@ -564,7 +563,7 @@ func (s *MWTestSuite) TestTxHandlerMemoGas() { func() { feeAmount = sdk.NewCoins(sdk.NewInt64Coin("atom", 0)) gasLimit = 50000 - txBuilder.SetMemo(strings.Repeat("01234567890", 500)) + suite.txBuilder.SetMemo(strings.Repeat("01234567890", 500)) }, false, false, @@ -575,7 +574,7 @@ func (s *MWTestSuite) TestTxHandlerMemoGas() { func() { feeAmount = sdk.NewCoins(sdk.NewInt64Coin("atom", 0)) gasLimit = 50000 - txBuilder.SetMemo(strings.Repeat("0123456789", 10)) + suite.txBuilder.SetMemo(strings.Repeat("0123456789", 10)) }, false, true, @@ -584,20 +583,20 @@ func (s *MWTestSuite) TestTxHandlerMemoGas() { } for _, tc := range testCases { - s.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() tc.malleate() - s.runTestCase(ctx, txBuilder, privs, msgs, feeAmount, gasLimit, accNums, accSeqs, ctx.ChainID(), tc) + suite.RunTestCase(privs, msgs, feeAmount, gasLimit, accNums, accSeqs, suite.ctx.ChainID(), tc) }) } } -func (s *MWTestSuite) TestTxHandlerMultiSigner() { - ctx := s.SetupTest(false) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() +func (suite *AnteTestSuite) TestAnteHandlerMultiSigner() { + suite.SetupTest(false) // setup // Same data for every test cases - accounts := s.createTestAccounts(ctx, 3, testCoins) + accounts := suite.CreateTestAccounts(3) msg1 := testdata.NewTestMsg(accounts[0].acc.GetAddress(), accounts[1].acc.GetAddress()) msg2 := testdata.NewTestMsg(accounts[2].acc.GetAddress(), accounts[0].acc.GetAddress()) msg3 := testdata.NewTestMsg(accounts[1].acc.GetAddress(), accounts[2].acc.GetAddress()) @@ -618,7 +617,7 @@ func (s *MWTestSuite) TestTxHandlerMultiSigner() { func() { msgs = []sdk.Msg{msg1, msg2, msg3} privs, accNums, accSeqs = []cryptotypes.PrivKey{accounts[0].priv, accounts[1].priv, accounts[2].priv}, []uint64{0, 1, 2}, []uint64{0, 0, 0} - txBuilder.SetMemo("Check signers are in expected order and different account numbers works") + suite.txBuilder.SetMemo("Check signers are in expected order and different account numbers works") }, false, true, @@ -657,20 +656,20 @@ func (s *MWTestSuite) TestTxHandlerMultiSigner() { } for _, tc := range testCases { - s.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() tc.malleate() - s.runTestCase(ctx, txBuilder, privs, msgs, feeAmount, gasLimit, accNums, accSeqs, ctx.ChainID(), tc) + suite.RunTestCase(privs, msgs, feeAmount, gasLimit, accNums, accSeqs, suite.ctx.ChainID(), tc) }) } } -func (s *MWTestSuite) TestTxHandlerBadSignBytes() { - ctx := s.SetupTest(true) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() +func (suite *AnteTestSuite) TestAnteHandlerBadSignBytes() { + suite.SetupTest(false) // setup // Same data for every test cases - accounts := s.createTestAccounts(ctx, 2, testCoins) + accounts := suite.CreateTestAccounts(2) msg0 := testdata.NewTestMsg(accounts[0].acc.GetAddress()) // Variable data per test case @@ -688,7 +687,7 @@ func (s *MWTestSuite) TestTxHandlerBadSignBytes() { { "test good tx and signBytes", func() { - chainID = ctx.ChainID() + chainID = suite.ctx.ChainID() feeAmount = testdata.NewTestFeeAmount() gasLimit = testdata.NewTestGasLimit() msgs = []sdk.Msg{msg0} @@ -711,7 +710,7 @@ func (s *MWTestSuite) TestTxHandlerBadSignBytes() { { "test wrong accSeqs", func() { - chainID = ctx.ChainID() // Back to correct chainID + chainID = suite.ctx.ChainID() // Back to correct chainID accSeqs = []uint64{2} }, false, @@ -783,20 +782,20 @@ func (s *MWTestSuite) TestTxHandlerBadSignBytes() { } for _, tc := range testCases { - s.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() tc.malleate() - s.runTestCase(ctx, txBuilder, privs, msgs, feeAmount, gasLimit, accNums, accSeqs, chainID, tc) + suite.RunTestCase(privs, msgs, feeAmount, gasLimit, accNums, accSeqs, chainID, tc) }) } } -func (s *MWTestSuite) TestTxHandlerSetPubKey() { - ctx := s.SetupTest(true) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() +func (suite *AnteTestSuite) TestAnteHandlerSetPubKey() { + suite.SetupTest(false) // setup // Same data for every test cases - accounts := s.createTestAccounts(ctx, 2, testCoins) + accounts := suite.CreateTestAccounts(2) feeAmount := testdata.NewTestFeeAmount() gasLimit := testdata.NewTestGasLimit() @@ -823,8 +822,8 @@ func (s *MWTestSuite) TestTxHandlerSetPubKey() { "make sure public key has been set (tx itself should fail because of replay protection)", func() { // Make sure public key has been set from previous test. - acc0 := s.app.AccountKeeper.GetAccount(ctx, accounts[0].acc.GetAddress()) - s.Require().Equal(acc0.GetPubKey(), accounts[0].priv.PubKey()) + acc0 := suite.app.AccountKeeper.GetAccount(suite.ctx, accounts[0].acc.GetAddress()) + suite.Require().Equal(acc0.GetPubKey(), accounts[0].priv.PubKey()) }, false, false, @@ -844,30 +843,30 @@ func (s *MWTestSuite) TestTxHandlerSetPubKey() { "make sure public key is not set, when tx has no pubkey or signature", func() { // Make sure public key has not been set from previous test. - acc1 := s.app.AccountKeeper.GetAccount(ctx, accounts[1].acc.GetAddress()) - s.Require().Nil(acc1.GetPubKey()) + acc1 := suite.app.AccountKeeper.GetAccount(suite.ctx, accounts[1].acc.GetAddress()) + suite.Require().Nil(acc1.GetPubKey()) privs, accNums, accSeqs = []cryptotypes.PrivKey{accounts[1].priv}, []uint64{1}, []uint64{0} msgs = []sdk.Msg{testdata.NewTestMsg(accounts[1].acc.GetAddress())} - txBuilder.SetMsgs(msgs...) - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(gasLimit) + suite.txBuilder.SetMsgs(msgs...) + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(gasLimit) // Manually create tx, and remove signature. - testTx, _, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) - txBuilder, err := s.clientCtx.TxConfig.WrapTxBuilder(testTx) - s.Require().NoError(err) - s.Require().NoError(txBuilder.SetSignatures()) + tx, err := suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) + txBuilder, err := suite.clientCtx.TxConfig.WrapTxBuilder(tx) + suite.Require().NoError(err) + suite.Require().NoError(txBuilder.SetSignatures()) - // Run txHandler manually, expect ErrNoSignatures. - _, _, err = s.txHandler.CheckTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: txBuilder.GetTx()}, tx.RequestCheckTx{}) - s.Require().Error(err) - s.Require().True(errors.Is(err, sdkerrors.ErrNoSignatures)) + // Run anteHandler manually, expect ErrNoSignatures. + _, err = suite.anteHandler(suite.ctx, txBuilder.GetTx(), false) + suite.Require().Error(err) + suite.Require().True(errors.Is(err, sdkerrors.ErrNoSignatures)) // Make sure public key has not been set. - acc1 = s.app.AccountKeeper.GetAccount(ctx, accounts[1].acc.GetAddress()) - s.Require().Nil(acc1.GetPubKey()) + acc1 = suite.app.AccountKeeper.GetAccount(suite.ctx, accounts[1].acc.GetAddress()) + suite.Require().Nil(acc1.GetPubKey()) // Set incorrect accSeq, to generate incorrect signature. privs, accNums, accSeqs = []cryptotypes.PrivKey{accounts[1].priv}, []uint64{1}, []uint64{1} @@ -879,10 +878,10 @@ func (s *MWTestSuite) TestTxHandlerSetPubKey() { { "make sure previous public key has been set after wrong signature", func() { - // Make sure public key has been set, as SetPubKeyMiddleware - // is called before all signature verification middlewares. - acc1 := s.app.AccountKeeper.GetAccount(ctx, accounts[1].acc.GetAddress()) - s.Require().Equal(acc1.GetPubKey(), accounts[1].priv.PubKey()) + // Make sure public key has been set, as SetPubKeyDecorator + // is called before all signature verification decorators. + acc1 := suite.app.AccountKeeper.GetAccount(suite.ctx, accounts[1].acc.GetAddress()) + suite.Require().Equal(acc1.GetPubKey(), accounts[1].priv.PubKey()) }, false, false, @@ -891,10 +890,11 @@ func (s *MWTestSuite) TestTxHandlerSetPubKey() { } for _, tc := range testCases { - s.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() tc.malleate() - s.runTestCase(ctx, txBuilder, privs, msgs, feeAmount, gasLimit, accNums, accSeqs, ctx.ChainID(), tc) + suite.RunTestCase(privs, msgs, feeAmount, gasLimit, accNums, accSeqs, suite.ctx.ChainID(), tc) }) } } @@ -964,17 +964,16 @@ func TestCountSubkeys(t *testing.T) { } for _, tc := range testCases { t.Run(tc.name, func(T *testing.T) { - require.Equal(t, tc.want, middleware.CountSubKeys(tc.args.pub)) + require.Equal(t, tc.want, ante.CountSubKeys(tc.args.pub)) }) } } -func (s *MWTestSuite) TestTxHandlerSigLimitExceeded() { - ctx := s.SetupTest(false) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() +func (suite *AnteTestSuite) TestAnteHandlerSigLimitExceeded() { + suite.SetupTest(false) // setup // Same data for every test cases - accounts := s.createTestAccounts(ctx, 8, testCoins) + accounts := suite.CreateTestAccounts(8) var addrs []sdk.AccAddress var privs []cryptotypes.PrivKey for i := 0; i < 8; i++ { @@ -997,25 +996,26 @@ func (s *MWTestSuite) TestTxHandlerSigLimitExceeded() { } for _, tc := range testCases { - s.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() tc.malleate() - s.runTestCase(ctx, txBuilder, privs, msgs, feeAmount, gasLimit, accNums, accSeqs, ctx.ChainID(), tc) + suite.RunTestCase(privs, msgs, feeAmount, gasLimit, accNums, accSeqs, suite.ctx.ChainID(), tc) }) } } // Test custom SignatureVerificationGasConsumer -func (s *MWTestSuite) TestCustomSignatureVerificationGasConsumer() { - ctx := s.SetupTest(false) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() - - txHandler, err := middleware.NewDefaultTxHandler( - middleware.TxHandlerOptions{ - AccountKeeper: s.app.AccountKeeper, - BankKeeper: s.app.BankKeeper, - FeegrantKeeper: s.app.FeeGrantKeeper, - SignModeHandler: s.clientCtx.TxConfig.SignModeHandler(), +func (suite *AnteTestSuite) TestCustomSignatureVerificationGasConsumer() { + suite.SetupTest(false) // setup + + // setup an ante handler that only accepts PubKeyEd25519 + anteHandler, err := ante.NewAnteHandler( + ante.HandlerOptions{ + AccountKeeper: suite.app.AccountKeeper, + BankKeeper: suite.app.BankKeeper, + FeegrantKeeper: suite.app.FeeGrantKeeper, + SignModeHandler: suite.clientCtx.TxConfig.SignModeHandler(), SigGasConsumer: func(meter sdk.GasMeter, sig signing.SignatureV2, params types.Params) error { switch pubkey := sig.PubKey.(type) { case *ed25519.PubKey: @@ -1025,22 +1025,21 @@ func (s *MWTestSuite) TestCustomSignatureVerificationGasConsumer() { return sdkerrors.Wrapf(sdkerrors.ErrInvalidPubKey, "unrecognized public key type: %T", pubkey) } }, - TxDecoder: s.clientCtx.TxConfig.TxDecoder(), }, ) - s.Require().NoError(err) - s.Require().NoError(err) + suite.Require().NoError(err) + suite.anteHandler = anteHandler // Same data for every test cases - accounts := s.createTestAccounts(ctx, 1, testCoins) - txBuilder.SetFeeAmount(testdata.NewTestFeeAmount()) - txBuilder.SetGasLimit(testdata.NewTestGasLimit()) - txBuilder.SetMsgs(testdata.NewTestMsg(accounts[0].acc.GetAddress())) + accounts := suite.CreateTestAccounts(1) + feeAmount := testdata.NewTestFeeAmount() + gasLimit := testdata.NewTestGasLimit() // Variable data per test case var ( accNums []uint64 + msgs []sdk.Msg privs []cryptotypes.PrivKey accSeqs []uint64 ) @@ -1049,6 +1048,7 @@ func (s *MWTestSuite) TestCustomSignatureVerificationGasConsumer() { { "verify that an secp256k1 account gets rejected", func() { + msgs = []sdk.Msg{testdata.NewTestMsg(accounts[0].acc.GetAddress())} privs, accNums, accSeqs = []cryptotypes.PrivKey{accounts[0].priv}, []uint64{0}, []uint64{0} }, false, @@ -1058,57 +1058,54 @@ func (s *MWTestSuite) TestCustomSignatureVerificationGasConsumer() { } for _, tc := range testCases { - s.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() tc.malleate() - testTx, txBytes, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) - _, err = txHandler.DeliverTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx, TxBytes: txBytes}) - s.Require().Error(err) - s.Require().True(errors.Is(err, tc.expErr)) + suite.RunTestCase(privs, msgs, feeAmount, gasLimit, accNums, accSeqs, suite.ctx.ChainID(), tc) }) } } -func (s *MWTestSuite) TestTxHandlerReCheck() { - ctx := s.SetupTest(false) // setup +func (suite *AnteTestSuite) TestAnteHandlerReCheck() { + suite.SetupTest(false) // setup // Set recheck=true - ctx = ctx.WithIsReCheckTx(true) - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() + suite.ctx = suite.ctx.WithIsReCheckTx(true) + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() // Same data for every test cases - accounts := s.createTestAccounts(ctx, 1, testCoins) + accounts := suite.CreateTestAccounts(1) feeAmount := testdata.NewTestFeeAmount() gasLimit := testdata.NewTestGasLimit() - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(gasLimit) + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(gasLimit) msg := testdata.NewTestMsg(accounts[0].acc.GetAddress()) msgs := []sdk.Msg{msg} - s.Require().NoError(txBuilder.SetMsgs(msgs...)) + suite.Require().NoError(suite.txBuilder.SetMsgs(msgs...)) - txBuilder.SetMemo("thisisatestmemo") + suite.txBuilder.SetMemo("thisisatestmemo") // test that operations skipped on recheck do not run privs, accNums, accSeqs := []cryptotypes.PrivKey{accounts[0].priv}, []uint64{0}, []uint64{0} - testTx, _, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) + tx, err := suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) - // make signature array empty which would normally cause ValidateBasicMiddleware and SigVerificationMiddleware fail - // since these middlewares don't run on recheck, the tx should pass the middleware - txBuilder, err = s.clientCtx.TxConfig.WrapTxBuilder(testTx) - s.Require().NoError(err) - s.Require().NoError(txBuilder.SetSignatures()) + // make signature array empty which would normally cause ValidateBasicDecorator and SigVerificationDecorator fail + // since these decorators don't run on recheck, the tx should pass the antehandler + txBuilder, err := suite.clientCtx.TxConfig.WrapTxBuilder(tx) + suite.Require().NoError(err) + suite.Require().NoError(txBuilder.SetSignatures()) - _, _, err = s.txHandler.CheckTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: txBuilder.GetTx()}, tx.RequestCheckTx{Type: abci.CheckTxType_Recheck}) - s.Require().Nil(err, "TxHandler errored on recheck unexpectedly: %v", err) + _, err = suite.anteHandler(suite.ctx, txBuilder.GetTx(), false) + suite.Require().Nil(err, "AnteHandler errored on recheck unexpectedly: %v", err) - testTx, _, err = s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) - txBytes, err := json.Marshal(testTx) - s.Require().Nil(err, "Error marshalling tx: %v", err) - ctx = ctx.WithTxBytes(txBytes) + tx, err = suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) + txBytes, err := json.Marshal(tx) + suite.Require().Nil(err, "Error marshalling tx: %v", err) + suite.ctx = suite.ctx.WithTxBytes(txBytes) // require that state machine param-dependent checking is still run on recheck since parameters can change between check and recheck testCases := []struct { @@ -1119,37 +1116,35 @@ func (s *MWTestSuite) TestTxHandlerReCheck() { {"txsize check", types.NewParams(types.DefaultMaxMemoCharacters, types.DefaultTxSigLimit, 10000000, types.DefaultSigVerifyCostED25519, types.DefaultSigVerifyCostSecp256k1)}, {"sig verify cost check", types.NewParams(types.DefaultMaxMemoCharacters, types.DefaultTxSigLimit, types.DefaultTxSizeCostPerByte, types.DefaultSigVerifyCostED25519, 100000000)}, } - for _, tc := range testCases { // set testcase parameters - s.app.AccountKeeper.SetParams(ctx, tc.params) + suite.app.AccountKeeper.SetParams(suite.ctx, tc.params) - _, _, err = s.txHandler.CheckTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx, TxBytes: txBytes}, tx.RequestCheckTx{Type: abci.CheckTxType_Recheck}) + _, err := suite.anteHandler(suite.ctx, tx, false) - s.Require().NotNil(err, "tx does not fail on recheck with updated params in test case: %s", tc.name) + suite.Require().NotNil(err, "tx does not fail on recheck with updated params in test case: %s", tc.name) // reset parameters to default values - s.app.AccountKeeper.SetParams(ctx, types.DefaultParams()) + suite.app.AccountKeeper.SetParams(suite.ctx, types.DefaultParams()) } // require that local mempool fee check is still run on recheck since validator may change minFee between check and recheck - // create new minimum gas price so txhandler fails on recheck - ctx = ctx.WithMinGasPrices([]sdk.DecCoin{{ + // create new minimum gas price so antehandler fails on recheck + suite.ctx = suite.ctx.WithMinGasPrices([]sdk.DecCoin{{ Denom: "dnecoin", // fee does not have this denom Amount: sdk.NewDec(5), }}) - _, _, err = s.txHandler.CheckTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}, tx.RequestCheckTx{}) - - s.Require().NotNil(err, "txhandler on recheck did not fail when mingasPrice was changed") + _, err = suite.anteHandler(suite.ctx, tx, false) + suite.Require().NotNil(err, "antehandler on recheck did not fail when mingasPrice was changed") // reset min gasprice - ctx = ctx.WithMinGasPrices(sdk.DecCoins{}) + suite.ctx = suite.ctx.WithMinGasPrices(sdk.DecCoins{}) - // remove funds for account so txhandler fails on recheck - s.app.AccountKeeper.SetAccount(ctx, accounts[0].acc) - balances := s.app.BankKeeper.GetAllBalances(ctx, accounts[0].acc.GetAddress()) - err = s.app.BankKeeper.SendCoinsFromAccountToModule(ctx, accounts[0].acc.GetAddress(), minttypes.ModuleName, balances) - s.Require().NoError(err) + // remove funds for account so antehandler fails on recheck + suite.app.AccountKeeper.SetAccount(suite.ctx, accounts[0].acc) + balances := suite.app.BankKeeper.GetAllBalances(suite.ctx, accounts[0].acc.GetAddress()) + err = suite.app.BankKeeper.SendCoinsFromAccountToModule(suite.ctx, accounts[0].acc.GetAddress(), minttypes.ModuleName, balances) + suite.Require().NoError(err) - _, _, err = s.txHandler.CheckTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}, tx.RequestCheckTx{}) - s.Require().NotNil(err, "txhandler on recheck did not fail once feePayer no longer has sufficient funds") + _, err = suite.anteHandler(suite.ctx, tx, false) + suite.Require().NotNil(err, "antehandler on recheck did not fail once feePayer no longer has sufficient funds") } diff --git a/x/auth/ante/basic.go b/x/auth/ante/basic.go new file mode 100644 index 000000000000..52c219f79e4d --- /dev/null +++ b/x/auth/ante/basic.go @@ -0,0 +1,206 @@ +package ante + +import ( + "github.com/cosmos/cosmos-sdk/codec/legacy" + "github.com/cosmos/cosmos-sdk/crypto/keys/multisig" + cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" + sdk "github.com/cosmos/cosmos-sdk/types" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" + "github.com/cosmos/cosmos-sdk/types/tx/signing" + "github.com/cosmos/cosmos-sdk/x/auth/migrations/legacytx" + authsigning "github.com/cosmos/cosmos-sdk/x/auth/signing" +) + +// ValidateBasicDecorator will call tx.ValidateBasic and return any non-nil error. +// If ValidateBasic passes, decorator calls next AnteHandler in chain. Note, +// ValidateBasicDecorator decorator will not get executed on ReCheckTx since it +// is not dependent on application state. +type ValidateBasicDecorator struct{} + +func NewValidateBasicDecorator() ValidateBasicDecorator { + return ValidateBasicDecorator{} +} + +func (vbd ValidateBasicDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (sdk.Context, error) { + // no need to validate basic on recheck tx, call next antehandler + if ctx.IsReCheckTx() { + return next(ctx, tx, simulate) + } + + if err := tx.ValidateBasic(); err != nil { + return ctx, err + } + + return next(ctx, tx, simulate) +} + +// ValidateMemoDecorator will validate memo given the parameters passed in +// If memo is too large decorator returns with error, otherwise call next AnteHandler +// CONTRACT: Tx must implement TxWithMemo interface +type ValidateMemoDecorator struct { + ak AccountKeeper +} + +func NewValidateMemoDecorator(ak AccountKeeper) ValidateMemoDecorator { + return ValidateMemoDecorator{ + ak: ak, + } +} + +func (vmd ValidateMemoDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (sdk.Context, error) { + memoTx, ok := tx.(sdk.TxWithMemo) + if !ok { + return ctx, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "invalid transaction type") + } + + params := vmd.ak.GetParams(ctx) + + memoLength := len(memoTx.GetMemo()) + if uint64(memoLength) > params.MaxMemoCharacters { + return ctx, sdkerrors.Wrapf(sdkerrors.ErrMemoTooLarge, + "maximum number of characters is %d but received %d characters", + params.MaxMemoCharacters, memoLength, + ) + } + + return next(ctx, tx, simulate) +} + +// ConsumeTxSizeGasDecorator will take in parameters and consume gas proportional +// to the size of tx before calling next AnteHandler. Note, the gas costs will be +// slightly over estimated due to the fact that any given signing account may need +// to be retrieved from state. +// +// CONTRACT: If simulate=true, then signatures must either be completely filled +// in or empty. +// CONTRACT: To use this decorator, signatures of transaction must be represented +// as legacytx.StdSignature otherwise simulate mode will incorrectly estimate gas cost. +type ConsumeTxSizeGasDecorator struct { + ak AccountKeeper +} + +func NewConsumeGasForTxSizeDecorator(ak AccountKeeper) ConsumeTxSizeGasDecorator { + return ConsumeTxSizeGasDecorator{ + ak: ak, + } +} + +func (cgts ConsumeTxSizeGasDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (sdk.Context, error) { + sigTx, ok := tx.(authsigning.SigVerifiableTx) + if !ok { + return ctx, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "invalid tx type") + } + params := cgts.ak.GetParams(ctx) + + ctx.GasMeter().ConsumeGas(params.TxSizeCostPerByte*sdk.Gas(len(ctx.TxBytes())), "txSize") + + // simulate gas cost for signatures in simulate mode + if simulate { + // in simulate mode, each element should be a nil signature + sigs, err := sigTx.GetSignaturesV2() + if err != nil { + return ctx, err + } + n := len(sigs) + + for i, signer := range sigTx.GetSigners() { + // if signature is already filled in, no need to simulate gas cost + if i < n && !isIncompleteSignature(sigs[i].Data) { + continue + } + + var pubkey cryptotypes.PubKey + + acc := cgts.ak.GetAccount(ctx, signer) + + // use placeholder simSecp256k1Pubkey if sig is nil + if acc == nil || acc.GetPubKey() == nil { + pubkey = simSecp256k1Pubkey + } else { + pubkey = acc.GetPubKey() + } + + // use stdsignature to mock the size of a full signature + simSig := legacytx.StdSignature{ //nolint:staticcheck // this will be removed when proto is ready + Signature: simSecp256k1Sig[:], + PubKey: pubkey, + } + + sigBz := legacy.Cdc.MustMarshal(simSig) + cost := sdk.Gas(len(sigBz) + 6) + + // If the pubkey is a multi-signature pubkey, then we estimate for the maximum + // number of signers. + if _, ok := pubkey.(*multisig.LegacyAminoPubKey); ok { + cost *= params.TxSigLimit + } + + ctx.GasMeter().ConsumeGas(params.TxSizeCostPerByte*cost, "txSize") + } + } + + return next(ctx, tx, simulate) +} + +// isIncompleteSignature tests whether SignatureData is fully filled in for simulation purposes +func isIncompleteSignature(data signing.SignatureData) bool { + if data == nil { + return true + } + + switch data := data.(type) { + case *signing.SingleSignatureData: + return len(data.Signature) == 0 + case *signing.MultiSignatureData: + if len(data.Signatures) == 0 { + return true + } + for _, s := range data.Signatures { + if isIncompleteSignature(s) { + return true + } + } + } + + return false +} + +type ( + // TxTimeoutHeightDecorator defines an AnteHandler decorator that checks for a + // tx height timeout. + TxTimeoutHeightDecorator struct{} + + // TxWithTimeoutHeight defines the interface a tx must implement in order for + // TxHeightTimeoutDecorator to process the tx. + TxWithTimeoutHeight interface { + sdk.Tx + + GetTimeoutHeight() uint64 + } +) + +// TxTimeoutHeightDecorator defines an AnteHandler decorator that checks for a +// tx height timeout. +func NewTxTimeoutHeightDecorator() TxTimeoutHeightDecorator { + return TxTimeoutHeightDecorator{} +} + +// AnteHandle implements an AnteHandler decorator for the TxHeightTimeoutDecorator +// type where the current block height is checked against the tx's height timeout. +// If a height timeout is provided (non-zero) and is less than the current block +// height, then an error is returned. +func (txh TxTimeoutHeightDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (sdk.Context, error) { + timeoutTx, ok := tx.(TxWithTimeoutHeight) + if !ok { + return ctx, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "expected tx to implement TxWithTimeoutHeight") + } + + timeoutHeight := timeoutTx.GetTimeoutHeight() + if timeoutHeight > 0 && uint64(ctx.BlockHeight()) > timeoutHeight { + return ctx, sdkerrors.Wrapf( + sdkerrors.ErrTxTimeoutHeight, "block height: %d, timeout height: %d", ctx.BlockHeight(), timeoutHeight, + ) + } + + return next(ctx, tx, simulate) +} diff --git a/x/auth/ante/basic_test.go b/x/auth/ante/basic_test.go new file mode 100644 index 000000000000..4a8cb830fdf6 --- /dev/null +++ b/x/auth/ante/basic_test.go @@ -0,0 +1,224 @@ +package ante_test + +import ( + "strings" + + cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" + "github.com/cosmos/cosmos-sdk/crypto/types/multisig" + "github.com/cosmos/cosmos-sdk/testutil/testdata" + sdk "github.com/cosmos/cosmos-sdk/types" + "github.com/cosmos/cosmos-sdk/types/tx/signing" + "github.com/cosmos/cosmos-sdk/x/auth/ante" +) + +func (suite *AnteTestSuite) TestValidateBasic() { + suite.SetupTest(true) // setup + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() + + // keys and addresses + priv1, _, addr1 := testdata.KeyTestPubAddr() + + // msg and signatures + msg := testdata.NewTestMsg(addr1) + feeAmount := testdata.NewTestFeeAmount() + gasLimit := testdata.NewTestGasLimit() + suite.Require().NoError(suite.txBuilder.SetMsgs(msg)) + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(gasLimit) + + privs, accNums, accSeqs := []cryptotypes.PrivKey{}, []uint64{}, []uint64{} + invalidTx, err := suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) + + vbd := ante.NewValidateBasicDecorator() + antehandler := sdk.ChainAnteDecorators(vbd) + _, err = antehandler(suite.ctx, invalidTx, false) + + suite.Require().NotNil(err, "Did not error on invalid tx") + + privs, accNums, accSeqs = []cryptotypes.PrivKey{priv1}, []uint64{0}, []uint64{0} + validTx, err := suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) + + _, err = antehandler(suite.ctx, validTx, false) + suite.Require().Nil(err, "ValidateBasicDecorator returned error on valid tx. err: %v", err) + + // test decorator skips on recheck + suite.ctx = suite.ctx.WithIsReCheckTx(true) + + // decorator should skip processing invalidTx on recheck and thus return nil-error + _, err = antehandler(suite.ctx, invalidTx, false) + + suite.Require().Nil(err, "ValidateBasicDecorator ran on ReCheck") +} + +func (suite *AnteTestSuite) TestValidateMemo() { + suite.SetupTest(true) // setup + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() + + // keys and addresses + priv1, _, addr1 := testdata.KeyTestPubAddr() + + // msg and signatures + msg := testdata.NewTestMsg(addr1) + feeAmount := testdata.NewTestFeeAmount() + gasLimit := testdata.NewTestGasLimit() + suite.Require().NoError(suite.txBuilder.SetMsgs(msg)) + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(gasLimit) + + privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{0}, []uint64{0} + suite.txBuilder.SetMemo(strings.Repeat("01234567890", 500)) + invalidTx, err := suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) + + // require that long memos get rejected + vmd := ante.NewValidateMemoDecorator(suite.app.AccountKeeper) + antehandler := sdk.ChainAnteDecorators(vmd) + _, err = antehandler(suite.ctx, invalidTx, false) + + suite.Require().NotNil(err, "Did not error on tx with high memo") + + suite.txBuilder.SetMemo(strings.Repeat("01234567890", 10)) + validTx, err := suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) + + // require small memos pass ValidateMemo Decorator + _, err = antehandler(suite.ctx, validTx, false) + suite.Require().Nil(err, "ValidateBasicDecorator returned error on valid tx. err: %v", err) +} + +func (suite *AnteTestSuite) TestConsumeGasForTxSize() { + suite.SetupTest(true) // setup + + // keys and addresses + priv1, _, addr1 := testdata.KeyTestPubAddr() + + // msg and signatures + msg := testdata.NewTestMsg(addr1) + feeAmount := testdata.NewTestFeeAmount() + gasLimit := testdata.NewTestGasLimit() + + cgtsd := ante.NewConsumeGasForTxSizeDecorator(suite.app.AccountKeeper) + antehandler := sdk.ChainAnteDecorators(cgtsd) + + testCases := []struct { + name string + sigV2 signing.SignatureV2 + }{ + {"SingleSignatureData", signing.SignatureV2{PubKey: priv1.PubKey()}}, + {"MultiSignatureData", signing.SignatureV2{PubKey: priv1.PubKey(), Data: multisig.NewMultisig(2)}}, + } + + for _, tc := range testCases { + suite.Run(tc.name, func() { + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() + suite.Require().NoError(suite.txBuilder.SetMsgs(msg)) + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(gasLimit) + suite.txBuilder.SetMemo(strings.Repeat("01234567890", 10)) + + privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{0}, []uint64{0} + tx, err := suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) + + txBytes, err := suite.clientCtx.TxConfig.TxJSONEncoder()(tx) + suite.Require().Nil(err, "Cannot marshal tx: %v", err) + + params := suite.app.AccountKeeper.GetParams(suite.ctx) + expectedGas := sdk.Gas(len(txBytes)) * params.TxSizeCostPerByte + + // Set suite.ctx with TxBytes manually + suite.ctx = suite.ctx.WithTxBytes(txBytes) + + // track how much gas is necessary to retrieve parameters + beforeGas := suite.ctx.GasMeter().GasConsumed() + suite.app.AccountKeeper.GetParams(suite.ctx) + afterGas := suite.ctx.GasMeter().GasConsumed() + expectedGas += afterGas - beforeGas + + beforeGas = suite.ctx.GasMeter().GasConsumed() + suite.ctx, err = antehandler(suite.ctx, tx, false) + suite.Require().Nil(err, "ConsumeTxSizeGasDecorator returned error: %v", err) + + // require that decorator consumes expected amount of gas + consumedGas := suite.ctx.GasMeter().GasConsumed() - beforeGas + suite.Require().Equal(expectedGas, consumedGas, "Decorator did not consume the correct amount of gas") + + // simulation must not underestimate gas of this decorator even with nil signatures + txBuilder, err := suite.clientCtx.TxConfig.WrapTxBuilder(tx) + suite.Require().NoError(err) + suite.Require().NoError(txBuilder.SetSignatures(tc.sigV2)) + tx = txBuilder.GetTx() + + simTxBytes, err := suite.clientCtx.TxConfig.TxJSONEncoder()(tx) + suite.Require().Nil(err, "Cannot marshal tx: %v", err) + // require that simulated tx is smaller than tx with signatures + suite.Require().True(len(simTxBytes) < len(txBytes), "simulated tx still has signatures") + + // Set suite.ctx with smaller simulated TxBytes manually + suite.ctx = suite.ctx.WithTxBytes(simTxBytes) + + beforeSimGas := suite.ctx.GasMeter().GasConsumed() + + // run antehandler with simulate=true + suite.ctx, err = antehandler(suite.ctx, tx, true) + consumedSimGas := suite.ctx.GasMeter().GasConsumed() - beforeSimGas + + // require that antehandler passes and does not underestimate decorator cost + suite.Require().Nil(err, "ConsumeTxSizeGasDecorator returned error: %v", err) + suite.Require().True(consumedSimGas >= expectedGas, "Simulate mode underestimates gas on AnteDecorator. Simulated cost: %d, expected cost: %d", consumedSimGas, expectedGas) + + }) + } + +} + +func (suite *AnteTestSuite) TestTxHeightTimeoutDecorator() { + suite.SetupTest(true) + + antehandler := sdk.ChainAnteDecorators(ante.NewTxTimeoutHeightDecorator()) + + // keys and addresses + priv1, _, addr1 := testdata.KeyTestPubAddr() + + // msg and signatures + msg := testdata.NewTestMsg(addr1) + feeAmount := testdata.NewTestFeeAmount() + gasLimit := testdata.NewTestGasLimit() + + testCases := []struct { + name string + timeout uint64 + height int64 + expectErr bool + }{ + {"default value", 0, 10, false}, + {"no timeout (greater height)", 15, 10, false}, + {"no timeout (same height)", 10, 10, false}, + {"timeout (smaller height)", 9, 10, true}, + } + + for _, tc := range testCases { + tc := tc + + suite.Run(tc.name, func() { + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() + + suite.Require().NoError(suite.txBuilder.SetMsgs(msg)) + + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(gasLimit) + suite.txBuilder.SetMemo(strings.Repeat("01234567890", 10)) + suite.txBuilder.SetTimeoutHeight(tc.timeout) + + privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{0}, []uint64{0} + tx, err := suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) + + ctx := suite.ctx.WithBlockHeight(tc.height) + _, err = antehandler(ctx, tx, true) + suite.Require().Equal(tc.expectErr, err != nil, err) + }) + } +} diff --git a/x/auth/middleware/expected_keepers.go b/x/auth/ante/expected_keepers.go similarity index 85% rename from x/auth/middleware/expected_keepers.go rename to x/auth/ante/expected_keepers.go index 33bb6339c1f3..4dbbbd21c713 100644 --- a/x/auth/middleware/expected_keepers.go +++ b/x/auth/ante/expected_keepers.go @@ -1,4 +1,4 @@ -package middleware +package ante import ( sdk "github.com/cosmos/cosmos-sdk/types" @@ -6,7 +6,7 @@ import ( ) // AccountKeeper defines the contract needed for AccountKeeper related APIs. -// Interface provides support to use non-sdk AccountKeeper for TxHandler's middlewares. +// Interface provides support to use non-sdk AccountKeeper for AnteHandler's decorators. type AccountKeeper interface { GetParams(ctx sdk.Context) (params types.Params) GetAccount(ctx sdk.Context, addr sdk.AccAddress) types.AccountI diff --git a/x/auth/ante/ext.go b/x/auth/ante/ext.go new file mode 100644 index 000000000000..362b8d32a971 --- /dev/null +++ b/x/auth/ante/ext.go @@ -0,0 +1,36 @@ +package ante + +import ( + codectypes "github.com/cosmos/cosmos-sdk/codec/types" + "github.com/cosmos/cosmos-sdk/types" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" +) + +type HasExtensionOptionsTx interface { + GetExtensionOptions() []*codectypes.Any + GetNonCriticalExtensionOptions() []*codectypes.Any +} + +// RejectExtensionOptionsDecorator is an AnteDecorator that rejects all extension +// options which can optionally be included in protobuf transactions. Users that +// need extension options should create a custom AnteHandler chain that handles +// needed extension options properly and rejects unknown ones. +type RejectExtensionOptionsDecorator struct{} + +// NewRejectExtensionOptionsDecorator creates a new RejectExtensionOptionsDecorator +func NewRejectExtensionOptionsDecorator() RejectExtensionOptionsDecorator { + return RejectExtensionOptionsDecorator{} +} + +var _ types.AnteDecorator = RejectExtensionOptionsDecorator{} + +// AnteHandle implements the AnteDecorator.AnteHandle method +func (r RejectExtensionOptionsDecorator) AnteHandle(ctx types.Context, tx types.Tx, simulate bool, next types.AnteHandler) (newCtx types.Context, err error) { + if hasExtOptsTx, ok := tx.(HasExtensionOptionsTx); ok { + if len(hasExtOptsTx.GetExtensionOptions()) != 0 { + return ctx, sdkerrors.ErrUnknownExtensionOptions + } + } + + return next(ctx, tx, simulate) +} diff --git a/x/auth/ante/ext_test.go b/x/auth/ante/ext_test.go new file mode 100644 index 000000000000..89ce6a7d649f --- /dev/null +++ b/x/auth/ante/ext_test.go @@ -0,0 +1,36 @@ +package ante_test + +import ( + "github.com/cosmos/cosmos-sdk/codec/types" + "github.com/cosmos/cosmos-sdk/testutil/testdata" + sdk "github.com/cosmos/cosmos-sdk/types" + "github.com/cosmos/cosmos-sdk/x/auth/ante" + "github.com/cosmos/cosmos-sdk/x/auth/tx" +) + +func (suite *AnteTestSuite) TestRejectExtensionOptionsDecorator() { + suite.SetupTest(true) // setup + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() + + reod := ante.NewRejectExtensionOptionsDecorator() + antehandler := sdk.ChainAnteDecorators(reod) + + // no extension options should not trigger an error + theTx := suite.txBuilder.GetTx() + _, err := antehandler(suite.ctx, theTx, false) + suite.Require().NoError(err) + + extOptsTxBldr, ok := suite.txBuilder.(tx.ExtensionOptionsTxBuilder) + if !ok { + // if we can't set extension options, this decorator doesn't apply and we're done + return + } + + // setting any extension option should cause an error + any, err := types.NewAnyWithValue(testdata.NewTestMsg()) + suite.Require().NoError(err) + extOptsTxBldr.SetExtensionOptions(any) + theTx = suite.txBuilder.GetTx() + _, err = antehandler(suite.ctx, theTx, false) + suite.Require().EqualError(err, "unknown extension options") +} diff --git a/x/auth/ante/fee.go b/x/auth/ante/fee.go new file mode 100644 index 000000000000..19e8258cfa73 --- /dev/null +++ b/x/auth/ante/fee.go @@ -0,0 +1,140 @@ +package ante + +import ( + "fmt" + + sdk "github.com/cosmos/cosmos-sdk/types" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" + "github.com/cosmos/cosmos-sdk/x/auth/types" +) + +// MempoolFeeDecorator will check if the transaction's fee is at least as large +// as the local validator's minimum gasFee (defined in validator config). +// If fee is too low, decorator returns error and tx is rejected from mempool. +// Note this only applies when ctx.CheckTx = true +// If fee is high enough or not CheckTx, then call next AnteHandler +// CONTRACT: Tx must implement FeeTx to use MempoolFeeDecorator +type MempoolFeeDecorator struct{} + +func NewMempoolFeeDecorator() MempoolFeeDecorator { + return MempoolFeeDecorator{} +} + +func (mfd MempoolFeeDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (newCtx sdk.Context, err error) { + feeTx, ok := tx.(sdk.FeeTx) + if !ok { + return ctx, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "Tx must be a FeeTx") + } + + feeCoins := feeTx.GetFee() + gas := feeTx.GetGas() + + // Ensure that the provided fees meet a minimum threshold for the validator, + // if this is a CheckTx. This is only for local mempool purposes, and thus + // is only ran on check tx. + if ctx.IsCheckTx() && !simulate { + minGasPrices := ctx.MinGasPrices() + if !minGasPrices.IsZero() { + requiredFees := make(sdk.Coins, len(minGasPrices)) + + // Determine the required fees by multiplying each required minimum gas + // price by the gas limit, where fee = ceil(minGasPrice * gasLimit). + glDec := sdk.NewDec(int64(gas)) + for i, gp := range minGasPrices { + fee := gp.Amount.Mul(glDec) + requiredFees[i] = sdk.NewCoin(gp.Denom, fee.Ceil().RoundInt()) + } + + if !feeCoins.IsAnyGTE(requiredFees) { + return ctx, sdkerrors.Wrapf(sdkerrors.ErrInsufficientFee, "insufficient fees; got: %s required: %s", feeCoins, requiredFees) + } + } + } + + return next(ctx, tx, simulate) +} + +// DeductFeeDecorator deducts fees from the first signer of the tx +// If the first signer does not have the funds to pay for the fees, return with InsufficientFunds error +// Call next AnteHandler if fees successfully deducted +// CONTRACT: Tx must implement FeeTx interface to use DeductFeeDecorator +type DeductFeeDecorator struct { + ak AccountKeeper + bankKeeper types.BankKeeper + feegrantKeeper FeegrantKeeper +} + +func NewDeductFeeDecorator(ak AccountKeeper, bk types.BankKeeper, fk FeegrantKeeper) DeductFeeDecorator { + return DeductFeeDecorator{ + ak: ak, + bankKeeper: bk, + feegrantKeeper: fk, + } +} + +func (dfd DeductFeeDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (newCtx sdk.Context, err error) { + feeTx, ok := tx.(sdk.FeeTx) + if !ok { + return ctx, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "Tx must be a FeeTx") + } + + if addr := dfd.ak.GetModuleAddress(types.FeeCollectorName); addr == nil { + return ctx, fmt.Errorf("Fee collector module account (%s) has not been set", types.FeeCollectorName) + } + + fee := feeTx.GetFee() + feePayer := feeTx.FeePayer() + feeGranter := feeTx.FeeGranter() + + deductFeesFrom := feePayer + + // if feegranter set deduct fee from feegranter account. + // this works with only when feegrant enabled. + if feeGranter != nil { + if dfd.feegrantKeeper == nil { + return ctx, sdkerrors.Wrap(sdkerrors.ErrInvalidRequest, "fee grants are not enabled") + } else if !feeGranter.Equals(feePayer) { + err := dfd.feegrantKeeper.UseGrantedFees(ctx, feeGranter, feePayer, fee, tx.GetMsgs()) + + if err != nil { + return ctx, sdkerrors.Wrapf(err, "%s not allowed to pay fees from %s", feeGranter, feePayer) + } + } + + deductFeesFrom = feeGranter + } + + deductFeesFromAcc := dfd.ak.GetAccount(ctx, deductFeesFrom) + if deductFeesFromAcc == nil { + return ctx, sdkerrors.Wrapf(sdkerrors.ErrUnknownAddress, "fee payer address: %s does not exist", deductFeesFrom) + } + + // deduct the fees + if !feeTx.GetFee().IsZero() { + err = DeductFees(dfd.bankKeeper, ctx, deductFeesFromAcc, feeTx.GetFee()) + if err != nil { + return ctx, err + } + } + + events := sdk.Events{sdk.NewEvent(sdk.EventTypeTx, + sdk.NewAttribute(sdk.AttributeKeyFee, feeTx.GetFee().String()), + )} + ctx.EventManager().EmitEvents(events) + + return next(ctx, tx, simulate) +} + +// DeductFees deducts fees from the given account. +func DeductFees(bankKeeper types.BankKeeper, ctx sdk.Context, acc types.AccountI, fees sdk.Coins) error { + if !fees.IsValid() { + return sdkerrors.Wrapf(sdkerrors.ErrInsufficientFee, "invalid fee amount: %s", fees) + } + + err := bankKeeper.SendCoinsFromAccountToModule(ctx, acc.GetAddress(), types.FeeCollectorName, fees) + if err != nil { + return sdkerrors.Wrapf(sdkerrors.ErrInsufficientFunds, err.Error()) + } + + return nil +} diff --git a/x/auth/ante/fee_test.go b/x/auth/ante/fee_test.go new file mode 100644 index 000000000000..7edd8fad9257 --- /dev/null +++ b/x/auth/ante/fee_test.go @@ -0,0 +1,104 @@ +package ante_test + +import ( + cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" + "github.com/cosmos/cosmos-sdk/simapp" + "github.com/cosmos/cosmos-sdk/testutil/testdata" + sdk "github.com/cosmos/cosmos-sdk/types" + "github.com/cosmos/cosmos-sdk/x/auth/ante" +) + +func (suite *AnteTestSuite) TestEnsureMempoolFees() { + suite.SetupTest(true) // setup + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() + + mfd := ante.NewMempoolFeeDecorator() + antehandler := sdk.ChainAnteDecorators(mfd) + + // keys and addresses + priv1, _, addr1 := testdata.KeyTestPubAddr() + + // msg and signatures + msg := testdata.NewTestMsg(addr1) + feeAmount := testdata.NewTestFeeAmount() + gasLimit := testdata.NewTestGasLimit() + suite.Require().NoError(suite.txBuilder.SetMsgs(msg)) + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(gasLimit) + + privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{0}, []uint64{0} + tx, err := suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) + + // Set high gas price so standard test fee fails + atomPrice := sdk.NewDecCoinFromDec("atom", sdk.NewDec(200).Quo(sdk.NewDec(100000))) + highGasPrice := []sdk.DecCoin{atomPrice} + suite.ctx = suite.ctx.WithMinGasPrices(highGasPrice) + + // Set IsCheckTx to true + suite.ctx = suite.ctx.WithIsCheckTx(true) + + // antehandler errors with insufficient fees + _, err = antehandler(suite.ctx, tx, false) + suite.Require().NotNil(err, "Decorator should have errored on too low fee for local gasPrice") + + // Set IsCheckTx to false + suite.ctx = suite.ctx.WithIsCheckTx(false) + + // antehandler should not error since we do not check minGasPrice in DeliverTx + _, err = antehandler(suite.ctx, tx, false) + suite.Require().Nil(err, "MempoolFeeDecorator returned error in DeliverTx") + + // Set IsCheckTx back to true for testing sufficient mempool fee + suite.ctx = suite.ctx.WithIsCheckTx(true) + + atomPrice = sdk.NewDecCoinFromDec("atom", sdk.NewDec(0).Quo(sdk.NewDec(100000))) + lowGasPrice := []sdk.DecCoin{atomPrice} + suite.ctx = suite.ctx.WithMinGasPrices(lowGasPrice) + + _, err = antehandler(suite.ctx, tx, false) + suite.Require().Nil(err, "Decorator should not have errored on fee higher than local gasPrice") +} + +func (suite *AnteTestSuite) TestDeductFees() { + suite.SetupTest(false) // setup + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() + + // keys and addresses + priv1, _, addr1 := testdata.KeyTestPubAddr() + + // msg and signatures + msg := testdata.NewTestMsg(addr1) + feeAmount := testdata.NewTestFeeAmount() + gasLimit := testdata.NewTestGasLimit() + suite.Require().NoError(suite.txBuilder.SetMsgs(msg)) + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(gasLimit) + + privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{0}, []uint64{0} + tx, err := suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) + + // Set account with insufficient funds + acc := suite.app.AccountKeeper.NewAccountWithAddress(suite.ctx, addr1) + suite.app.AccountKeeper.SetAccount(suite.ctx, acc) + coins := sdk.NewCoins(sdk.NewCoin("atom", sdk.NewInt(10))) + err = simapp.FundAccount(suite.app.BankKeeper, suite.ctx, addr1, coins) + suite.Require().NoError(err) + + dfd := ante.NewDeductFeeDecorator(suite.app.AccountKeeper, suite.app.BankKeeper, nil) + antehandler := sdk.ChainAnteDecorators(dfd) + + _, err = antehandler(suite.ctx, tx, false) + + suite.Require().NotNil(err, "Tx did not error when fee payer had insufficient funds") + + // Set account with sufficient funds + suite.app.AccountKeeper.SetAccount(suite.ctx, acc) + err = simapp.FundAccount(suite.app.BankKeeper, suite.ctx, addr1, sdk.NewCoins(sdk.NewCoin("atom", sdk.NewInt(200)))) + suite.Require().NoError(err) + + _, err = antehandler(suite.ctx, tx, false) + + suite.Require().Nil(err, "Tx errored after account has been set with sufficient funds") +} diff --git a/x/auth/middleware/feegrant_test.go b/x/auth/ante/feegrant_test.go similarity index 78% rename from x/auth/middleware/feegrant_test.go rename to x/auth/ante/feegrant_test.go index 547f57a589d1..b23c2cec619d 100644 --- a/x/auth/middleware/feegrant_test.go +++ b/x/auth/ante/feegrant_test.go @@ -1,4 +1,4 @@ -package middleware_test +package ante_test import ( "math/rand" @@ -10,35 +10,33 @@ import ( "github.com/cosmos/cosmos-sdk/client" "github.com/cosmos/cosmos-sdk/codec" cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" + + "github.com/cosmos/cosmos-sdk/simapp" "github.com/cosmos/cosmos-sdk/simapp/helpers" "github.com/cosmos/cosmos-sdk/testutil/testdata" sdk "github.com/cosmos/cosmos-sdk/types" "github.com/cosmos/cosmos-sdk/types/simulation" - txtypes "github.com/cosmos/cosmos-sdk/types/tx" "github.com/cosmos/cosmos-sdk/types/tx/signing" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" + "github.com/cosmos/cosmos-sdk/x/auth/ante" authsign "github.com/cosmos/cosmos-sdk/x/auth/signing" "github.com/cosmos/cosmos-sdk/x/auth/tx" authtypes "github.com/cosmos/cosmos-sdk/x/auth/types" - "github.com/cosmos/cosmos-sdk/x/bank/testutil" "github.com/cosmos/cosmos-sdk/x/feegrant" ) -func (s *MWTestSuite) TestDeductFeesNoDelegation() { - ctx := s.SetupTest(false) // setup - app := s.app +func (suite *AnteTestSuite) TestDeductFeesNoDelegation() { + suite.SetupTest(false) + // setup + app, ctx := suite.app, suite.ctx protoTxCfg := tx.NewTxConfig(codec.NewProtoCodec(app.InterfaceRegistry()), tx.DefaultSignModes) - txHandler := middleware.ComposeMiddlewares( - noopTxHandler, - middleware.DeductFeeMiddleware( - s.app.AccountKeeper, - s.app.BankKeeper, - s.app.FeeGrantKeeper, - nil, - ), - ) + // this just tests our handler + dfd := ante.NewDeductFeeDecorator(app.AccountKeeper, app.BankKeeper, app.FeeGrantKeeper) + feeAnteHandler := sdk.ChainAnteDecorators(dfd) + + // this tests the whole stack + anteHandlerStack := suite.anteHandler // keys and addresses priv1, _, addr1 := testdata.KeyTestPubAddr() @@ -48,24 +46,24 @@ func (s *MWTestSuite) TestDeductFeesNoDelegation() { priv5, _, addr5 := testdata.KeyTestPubAddr() // Set addr1 with insufficient funds - err := testutil.FundAccount(s.app.BankKeeper, ctx, addr1, []sdk.Coin{sdk.NewCoin("atom", sdk.NewInt(10))}) - s.Require().NoError(err) + err := simapp.FundAccount(suite.app.BankKeeper, suite.ctx, addr1, []sdk.Coin{sdk.NewCoin("atom", sdk.NewInt(10))}) + suite.Require().NoError(err) // Set addr2 with more funds - err = testutil.FundAccount(s.app.BankKeeper, ctx, addr2, []sdk.Coin{sdk.NewCoin("atom", sdk.NewInt(99999))}) - s.Require().NoError(err) + err = simapp.FundAccount(suite.app.BankKeeper, suite.ctx, addr2, []sdk.Coin{sdk.NewCoin("atom", sdk.NewInt(99999))}) + suite.Require().NoError(err) // grant fee allowance from `addr2` to `addr3` (plenty to pay) err = app.FeeGrantKeeper.GrantAllowance(ctx, addr2, addr3, &feegrant.BasicAllowance{ SpendLimit: sdk.NewCoins(sdk.NewInt64Coin("atom", 500)), }) - s.Require().NoError(err) + suite.Require().NoError(err) // grant low fee allowance (20atom), to check the tx requesting more than allowed. err = app.FeeGrantKeeper.GrantAllowance(ctx, addr2, addr4, &feegrant.BasicAllowance{ SpendLimit: sdk.NewCoins(sdk.NewInt64Coin("atom", 20)), }) - s.Require().NoError(err) + suite.Require().NoError(err) cases := map[string]struct { signerKey cryptotypes.PrivKey @@ -136,7 +134,7 @@ func (s *MWTestSuite) TestDeductFeesNoDelegation() { for name, stc := range cases { tc := stc // to make scopelint happy - s.T().Run(name, func(t *testing.T) { + suite.T().Run(name, func(t *testing.T) { fee := sdk.NewCoins(sdk.NewInt64Coin("atom", tc.fee)) msgs := []sdk.Msg{testdata.NewTestMsg(tc.signer)} @@ -146,23 +144,20 @@ func (s *MWTestSuite) TestDeductFeesNoDelegation() { accNums, seqs = []uint64{acc.GetAccountNumber()}, []uint64{acc.GetSequence()} } - testTx, err := genTxWithFeeGranter(protoTxCfg, msgs, fee, helpers.DefaultGenTxGas, ctx.ChainID(), accNums, seqs, tc.feeAccount, privs...) - s.Require().NoError(err) - - // tests only feegrant middleware - _, err = txHandler.DeliverTx(sdk.WrapSDKContext(ctx), txtypes.Request{Tx: testTx}) + tx, err := genTxWithFeeGranter(protoTxCfg, msgs, fee, helpers.DefaultGenTxGas, ctx.ChainID(), accNums, seqs, tc.feeAccount, privs...) + suite.Require().NoError(err) + _, err = feeAnteHandler(ctx, tx, false) // tests only feegrant ante if tc.valid { - s.Require().NoError(err) + suite.Require().NoError(err) } else { - s.Require().Error(err) + suite.Require().Error(err) } - // tests while stack - _, err = s.txHandler.DeliverTx(sdk.WrapSDKContext(ctx), txtypes.Request{Tx: testTx}) + _, err = anteHandlerStack(ctx, tx, false) // tests while stack if tc.valid { - s.Require().NoError(err) + suite.Require().NoError(err) } else { - s.Require().Error(err) + suite.Require().Error(err) } }) } @@ -213,11 +208,9 @@ func genTxWithFeeGranter(gen client.TxConfig, msgs []sdk.Msg, feeAmt sdk.Coins, // 2nd round: once all signer infos are set, every signer can sign. for i, p := range priv { signerData := authsign.SignerData{ - Address: sdk.AccAddress(p.PubKey().Address()).String(), ChainID: chainID, AccountNumber: accNums[i], Sequence: accSeqs[i], - PubKey: p.PubKey(), } signBytes, err := gen.SignModeHandler().GetSignBytes(signMode, signerData, tx.GetTx()) if err != nil { diff --git a/x/auth/ante/setup.go b/x/auth/ante/setup.go new file mode 100644 index 000000000000..6d6d4344c0c3 --- /dev/null +++ b/x/auth/ante/setup.go @@ -0,0 +1,76 @@ +package ante + +import ( + "fmt" + + sdk "github.com/cosmos/cosmos-sdk/types" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" + "github.com/cosmos/cosmos-sdk/x/auth/migrations/legacytx" +) + +var ( + _ GasTx = (*legacytx.StdTx)(nil) // assert StdTx implements GasTx +) + +// GasTx defines a Tx with a GetGas() method which is needed to use SetUpContextDecorator +type GasTx interface { + sdk.Tx + GetGas() uint64 +} + +// SetUpContextDecorator sets the GasMeter in the Context and wraps the next AnteHandler with a defer clause +// to recover from any downstream OutOfGas panics in the AnteHandler chain to return an error with information +// on gas provided and gas used. +// CONTRACT: Must be first decorator in the chain +// CONTRACT: Tx must implement GasTx interface +type SetUpContextDecorator struct{} + +func NewSetUpContextDecorator() SetUpContextDecorator { + return SetUpContextDecorator{} +} + +func (sud SetUpContextDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (newCtx sdk.Context, err error) { + // all transactions must implement GasTx + gasTx, ok := tx.(GasTx) + if !ok { + // Set a gas meter with limit 0 as to prevent an infinite gas meter attack + // during runTx. + newCtx = SetGasMeter(simulate, ctx, 0) + return newCtx, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "Tx must be GasTx") + } + + newCtx = SetGasMeter(simulate, ctx, gasTx.GetGas()) + + // Decorator will catch an OutOfGasPanic caused in the next antehandler + // AnteHandlers must have their own defer/recover in order for the BaseApp + // to know how much gas was used! This is because the GasMeter is created in + // the AnteHandler, but if it panics the context won't be set properly in + // runTx's recover call. + defer func() { + if r := recover(); r != nil { + switch rType := r.(type) { + case sdk.ErrorOutOfGas: + log := fmt.Sprintf( + "out of gas in location: %v; gasWanted: %d, gasUsed: %d", + rType.Descriptor, gasTx.GetGas(), newCtx.GasMeter().GasConsumed()) + + err = sdkerrors.Wrap(sdkerrors.ErrOutOfGas, log) + default: + panic(r) + } + } + }() + + return next(newCtx, tx, simulate) +} + +// SetGasMeter returns a new context with a gas meter set from a given context. +func SetGasMeter(simulate bool, ctx sdk.Context, gasLimit uint64) sdk.Context { + // In various cases such as simulation and during the genesis block, we do not + // meter any gas utilization. + if simulate || ctx.BlockHeight() == 0 { + return ctx.WithGasMeter(sdk.NewInfiniteGasMeter()) + } + + return ctx.WithGasMeter(sdk.NewGasMeter(gasLimit)) +} diff --git a/x/auth/ante/setup_test.go b/x/auth/ante/setup_test.go new file mode 100644 index 000000000000..4942665cac04 --- /dev/null +++ b/x/auth/ante/setup_test.go @@ -0,0 +1,99 @@ +package ante_test + +import ( + cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" + "github.com/cosmos/cosmos-sdk/testutil/testdata" + sdk "github.com/cosmos/cosmos-sdk/types" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" + "github.com/cosmos/cosmos-sdk/x/auth/ante" +) + +func (suite *AnteTestSuite) TestSetup() { + suite.SetupTest(true) // setup + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() + + // keys and addresses + priv1, _, addr1 := testdata.KeyTestPubAddr() + + // msg and signatures + msg := testdata.NewTestMsg(addr1) + feeAmount := testdata.NewTestFeeAmount() + gasLimit := testdata.NewTestGasLimit() + suite.Require().NoError(suite.txBuilder.SetMsgs(msg)) + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(gasLimit) + + privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{0}, []uint64{0} + tx, err := suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) + + sud := ante.NewSetUpContextDecorator() + antehandler := sdk.ChainAnteDecorators(sud) + + // Set height to non-zero value for GasMeter to be set + suite.ctx = suite.ctx.WithBlockHeight(1) + + // Context GasMeter Limit not set + suite.Require().Equal(uint64(0), suite.ctx.GasMeter().Limit(), "GasMeter set with limit before setup") + + newCtx, err := antehandler(suite.ctx, tx, false) + suite.Require().Nil(err, "SetUpContextDecorator returned error") + + // Context GasMeter Limit should be set after SetUpContextDecorator runs + suite.Require().Equal(gasLimit, newCtx.GasMeter().Limit(), "GasMeter not set correctly") +} + +func (suite *AnteTestSuite) TestRecoverPanic() { + suite.SetupTest(true) // setup + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() + + // keys and addresses + priv1, _, addr1 := testdata.KeyTestPubAddr() + + // msg and signatures + msg := testdata.NewTestMsg(addr1) + feeAmount := testdata.NewTestFeeAmount() + gasLimit := testdata.NewTestGasLimit() + suite.Require().NoError(suite.txBuilder.SetMsgs(msg)) + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(gasLimit) + + privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{0}, []uint64{0} + tx, err := suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) + + sud := ante.NewSetUpContextDecorator() + antehandler := sdk.ChainAnteDecorators(sud, OutOfGasDecorator{}) + + // Set height to non-zero value for GasMeter to be set + suite.ctx = suite.ctx.WithBlockHeight(1) + + newCtx, err := antehandler(suite.ctx, tx, false) + + suite.Require().NotNil(err, "Did not return error on OutOfGas panic") + + suite.Require().True(sdkerrors.ErrOutOfGas.Is(err), "Returned error is not an out of gas error") + suite.Require().Equal(gasLimit, newCtx.GasMeter().Limit()) + + antehandler = sdk.ChainAnteDecorators(sud, PanicDecorator{}) + suite.Require().Panics(func() { antehandler(suite.ctx, tx, false) }, "Recovered from non-Out-of-Gas panic") // nolint:errcheck +} + +type OutOfGasDecorator struct{} + +// AnteDecorator that will throw OutOfGas panic +func (ogd OutOfGasDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (sdk.Context, error) { + overLimit := ctx.GasMeter().Limit() + 1 + + // Should panic with outofgas error + ctx.GasMeter().ConsumeGas(overLimit, "test panic") + + // not reached + return next(ctx, tx, simulate) +} + +type PanicDecorator struct{} + +func (pd PanicDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (newCtx sdk.Context, err error) { + panic("random error") +} diff --git a/x/auth/middleware/sigverify.go b/x/auth/ante/sigverify.go similarity index 51% rename from x/auth/middleware/sigverify.go rename to x/auth/ante/sigverify.go index 0e13b4142a29..5097478da237 100644 --- a/x/auth/middleware/sigverify.go +++ b/x/auth/ante/sigverify.go @@ -1,9 +1,9 @@ -package middleware +package ante import ( "bytes" - "context" "encoding/base64" + "encoding/hex" "fmt" "github.com/cosmos/cosmos-sdk/crypto/keys/ed25519" @@ -14,11 +14,10 @@ import ( "github.com/cosmos/cosmos-sdk/crypto/types/multisig" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/tx" "github.com/cosmos/cosmos-sdk/types/tx/signing" + "github.com/cosmos/cosmos-sdk/x/auth/migrations/legacytx" authsigning "github.com/cosmos/cosmos-sdk/x/auth/signing" "github.com/cosmos/cosmos-sdk/x/auth/types" - abci "github.com/tendermint/tendermint/abci/types" ) var ( @@ -26,42 +25,44 @@ var ( key = make([]byte, secp256k1.PubKeySize) simSecp256k1Pubkey = &secp256k1.PubKey{Key: key} simSecp256k1Sig [64]byte + + _ authsigning.SigVerifiableTx = (*legacytx.StdTx)(nil) // assert StdTx implements SigVerifiableTx ) +func init() { + // This decodes a valid hex string into a sepc256k1Pubkey for use in transaction simulation + bz, _ := hex.DecodeString("035AD6810A47F073553FF30D2FCC7E0D3B1C0B74B61A1AAA2582344037151E143A") + copy(key, bz) + simSecp256k1Pubkey.Key = key +} + // SignatureVerificationGasConsumer is the type of function that is used to both // consume gas when verifying signatures and also to accept or reject different types of pubkeys // This is where apps can define their own PubKey type SignatureVerificationGasConsumer = func(meter sdk.GasMeter, sig signing.SignatureV2, params types.Params) error -var _ tx.Handler = setPubKeyTxHandler{} - -type setPubKeyTxHandler struct { - ak AccountKeeper - next tx.Handler +// SetPubKeyDecorator sets PubKeys in context for any signer which does not already have pubkey set +// PubKeys must be set in context for all signers before any other sigverify decorators run +// CONTRACT: Tx must implement SigVerifiableTx interface +type SetPubKeyDecorator struct { + ak AccountKeeper } -// SetPubKeyMiddleware sets PubKeys in context for any signer which does not already have pubkey set -// PubKeys must be set in context for all signers before any other sigverify middlewares run -// CONTRACT: Tx must implement SigVerifiableTx interface -func SetPubKeyMiddleware(ak AccountKeeper) tx.Middleware { - return func(txh tx.Handler) tx.Handler { - return setPubKeyTxHandler{ - ak: ak, - next: txh, - } +func NewSetPubKeyDecorator(ak AccountKeeper) SetPubKeyDecorator { + return SetPubKeyDecorator{ + ak: ak, } } -func (spkm setPubKeyTxHandler) setPubKey(ctx context.Context, req tx.Request, simulate bool) error { - sdkCtx := sdk.UnwrapSDKContext(ctx) - sigTx, ok := req.Tx.(authsigning.SigVerifiableTx) +func (spkd SetPubKeyDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (sdk.Context, error) { + sigTx, ok := tx.(authsigning.SigVerifiableTx) if !ok { - return sdkerrors.Wrap(sdkerrors.ErrTxDecode, "invalid tx type") + return ctx, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "invalid tx type") } pubkeys, err := sigTx.GetPubKeys() if err != nil { - return err + return ctx, err } signers := sigTx.GetSigners() @@ -75,13 +76,13 @@ func (spkm setPubKeyTxHandler) setPubKey(ctx context.Context, req tx.Request, si } // Only make check if simulate=false if !simulate && !bytes.Equal(pk.Address(), signers[i]) { - return sdkerrors.Wrapf(sdkerrors.ErrInvalidPubKey, + return ctx, sdkerrors.Wrapf(sdkerrors.ErrInvalidPubKey, "pubKey does not match signer address %s with signer index: %d", signers[i], i) } - acc, err := GetSignerAcc(sdkCtx, spkm.ak, signers[i]) + acc, err := GetSignerAcc(ctx, spkd.ak, signers[i]) if err != nil { - return err + return ctx, err } // account already has pubkey set,no need to reset if acc.GetPubKey() != nil { @@ -89,9 +90,9 @@ func (spkm setPubKeyTxHandler) setPubKey(ctx context.Context, req tx.Request, si } err = acc.SetPubKey(pk) if err != nil { - return sdkerrors.Wrap(sdkerrors.ErrInvalidPubKey, err.Error()) + return ctx, sdkerrors.Wrap(sdkerrors.ErrInvalidPubKey, err.Error()) } - spkm.ak.SetAccount(sdkCtx, acc) + spkd.ak.SetAccount(ctx, acc) } // Also emit the following events, so that txs can be indexed by these @@ -100,7 +101,7 @@ func (spkm setPubKeyTxHandler) setPubKey(ctx context.Context, req tx.Request, si // - concat(address,"/",sequence) (via `tx.acc_seq='cosmos1abc...def/42'`). sigs, err := sigTx.GetSignaturesV2() if err != nil { - return err + return ctx, err } var events sdk.Events @@ -111,7 +112,7 @@ func (spkm setPubKeyTxHandler) setPubKey(ctx context.Context, req tx.Request, si sigBzs, err := signatureDataToBz(sig.Data) if err != nil { - return err + return ctx, err } for _, sigBz := range sigBzs { events = append(events, sdk.NewEvent(sdk.EventTypeTx, @@ -120,210 +121,37 @@ func (spkm setPubKeyTxHandler) setPubKey(ctx context.Context, req tx.Request, si } } - sdkCtx.EventManager().EmitEvents(events) - - return nil -} - -// CheckTx implements tx.Handler.CheckTx. -func (spkm setPubKeyTxHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - if err := spkm.setPubKey(ctx, req, false); err != nil { - return tx.Response{}, tx.ResponseCheckTx{}, err - } + ctx.EventManager().EmitEvents(events) - return spkm.next.CheckTx(ctx, req, checkReq) + return next(ctx, tx, simulate) } -// DeliverTx implements tx.Handler.DeliverTx. -func (spkm setPubKeyTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := spkm.setPubKey(ctx, req, false); err != nil { - return tx.Response{}, err - } - return spkm.next.DeliverTx(ctx, req) -} - -// SimulateTx implements tx.Handler.SimulateTx. -func (spkm setPubKeyTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := spkm.setPubKey(ctx, req, true); err != nil { - return tx.Response{}, err - } - return spkm.next.SimulateTx(ctx, req) -} - -var _ tx.Handler = validateSigCountTxHandler{} - -type validateSigCountTxHandler struct { - ak AccountKeeper - next tx.Handler -} - -// ValidateSigCountMiddleware takes in Params and returns errors if there are too many signatures in the tx for the given params -// otherwise it calls next middleware -// Use this middleware to set parameterized limit on number of signatures in tx +// Consume parameter-defined amount of gas for each signature according to the passed-in SignatureVerificationGasConsumer function +// before calling the next AnteHandler +// CONTRACT: Pubkeys are set in context for all signers before this decorator runs // CONTRACT: Tx must implement SigVerifiableTx interface -func ValidateSigCountMiddleware(ak AccountKeeper) tx.Middleware { - return func(txh tx.Handler) tx.Handler { - return validateSigCountTxHandler{ - ak: ak, - next: txh, - } - } -} - -func (vscd validateSigCountTxHandler) checkSigCount(ctx context.Context, req tx.Request) error { - sdkCtx := sdk.UnwrapSDKContext(ctx) - - sigTx, ok := req.Tx.(authsigning.SigVerifiableTx) - if !ok { - return sdkerrors.Wrap(sdkerrors.ErrTxDecode, "Tx must be a sigTx") - } - - params := vscd.ak.GetParams(sdkCtx) - pubKeys, err := sigTx.GetPubKeys() - if err != nil { - return err - } - - sigCount := 0 - for _, pk := range pubKeys { - sigCount += CountSubKeys(pk) - if uint64(sigCount) > params.TxSigLimit { - return sdkerrors.Wrapf(sdkerrors.ErrTooManySignatures, - "signatures: %d, limit: %d", sigCount, params.TxSigLimit) - } - } - return nil -} - -// CheckTx implements tx.Handler.CheckTx. -func (vscd validateSigCountTxHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - if err := vscd.checkSigCount(ctx, req); err != nil { - return tx.Response{}, tx.ResponseCheckTx{}, err - } - - return vscd.next.CheckTx(ctx, req, checkReq) -} - -// DeliverTx implements tx.Handler.DeliverTx. -func (vscd validateSigCountTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := vscd.checkSigCount(ctx, req); err != nil { - return tx.Response{}, err - } - - return vscd.next.DeliverTx(ctx, req) -} - -// SimulateTx implements tx.Handler.SimulateTx. -func (vscd validateSigCountTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := vscd.checkSigCount(ctx, req); err != nil { - return tx.Response{}, err - } - - return vscd.next.SimulateTx(ctx, req) -} - -// DefaultSigVerificationGasConsumer is the default implementation of SignatureVerificationGasConsumer. It consumes gas -// for signature verification based upon the public key type. The cost is fetched from the given params and is matched -// by the concrete type. -func DefaultSigVerificationGasConsumer( - meter sdk.GasMeter, sig signing.SignatureV2, params types.Params, -) error { - pubkey := sig.PubKey - switch pubkey := pubkey.(type) { - case *ed25519.PubKey: - meter.ConsumeGas(params.SigVerifyCostED25519, "ante verify: ed25519") - return sdkerrors.Wrap(sdkerrors.ErrInvalidPubKey, "ED25519 public keys are unsupported") - - case *secp256k1.PubKey: - meter.ConsumeGas(params.SigVerifyCostSecp256k1, "ante verify: secp256k1") - return nil - - case *secp256r1.PubKey: - meter.ConsumeGas(params.SigVerifyCostSecp256r1(), "ante verify: secp256r1") - return nil - - case multisig.PubKey: - multisignature, ok := sig.Data.(*signing.MultiSignatureData) - if !ok { - return fmt.Errorf("expected %T, got, %T", &signing.MultiSignatureData{}, sig.Data) - } - err := ConsumeMultisignatureVerificationGas(meter, multisignature, pubkey, params, sig.Sequence) - if err != nil { - return err - } - return nil - - default: - return sdkerrors.Wrapf(sdkerrors.ErrInvalidPubKey, "unrecognized public key type: %T", pubkey) - } -} - -// ConsumeMultisignatureVerificationGas consumes gas from a GasMeter for verifying a multisig pubkey signature -func ConsumeMultisignatureVerificationGas( - meter sdk.GasMeter, sig *signing.MultiSignatureData, pubkey multisig.PubKey, - params types.Params, accSeq uint64, -) error { - - size := sig.BitArray.Count() - sigIndex := 0 - - for i := 0; i < size; i++ { - if !sig.BitArray.GetIndex(i) { - continue - } - sigV2 := signing.SignatureV2{ - PubKey: pubkey.GetPubKeys()[i], - Data: sig.Signatures[sigIndex], - Sequence: accSeq, - } - err := DefaultSigVerificationGasConsumer(meter, sigV2, params) - if err != nil { - return err - } - sigIndex++ - } - - return nil -} - -var _ tx.Handler = sigGasConsumeTxHandler{} - -type sigGasConsumeTxHandler struct { +type SigGasConsumeDecorator struct { ak AccountKeeper sigGasConsumer SignatureVerificationGasConsumer - next tx.Handler } -// SigGasConsumeMiddleware consumes parameter-defined amount of gas for each signature according to the passed-in SignatureVerificationGasConsumer function -// before calling the next middleware -// CONTRACT: Pubkeys are set in context for all signers before this middleware runs -// CONTRACT: Tx must implement SigVerifiableTx interface -func SigGasConsumeMiddleware(ak AccountKeeper, sigGasConsumer SignatureVerificationGasConsumer) tx.Middleware { - if sigGasConsumer == nil { - sigGasConsumer = DefaultSigVerificationGasConsumer - } - - return func(h tx.Handler) tx.Handler { - return sigGasConsumeTxHandler{ - ak: ak, - sigGasConsumer: sigGasConsumer, - next: h, - } +func NewSigGasConsumeDecorator(ak AccountKeeper, sigGasConsumer SignatureVerificationGasConsumer) SigGasConsumeDecorator { + return SigGasConsumeDecorator{ + ak: ak, + sigGasConsumer: sigGasConsumer, } } -func (sgcm sigGasConsumeTxHandler) sigGasConsume(ctx context.Context, req tx.Request, simulate bool) error { - sdkCtx := sdk.UnwrapSDKContext(ctx) - - sigTx, ok := req.Tx.(authsigning.SigVerifiableTx) +func (sgcd SigGasConsumeDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (newCtx sdk.Context, err error) { + sigTx, ok := tx.(authsigning.SigVerifiableTx) if !ok { - return sdkerrors.Wrap(sdkerrors.ErrTxDecode, "invalid transaction type") + return ctx, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "invalid transaction type") } - params := sgcm.ak.GetParams(sdkCtx) + params := sgcd.ak.GetParams(ctx) sigs, err := sigTx.GetSignaturesV2() if err != nil { - return err + return ctx, err } // stdSigs contains the sequence number, account number, and signatures. @@ -331,9 +159,9 @@ func (sgcm sigGasConsumeTxHandler) sigGasConsume(ctx context.Context, req tx.Req signerAddrs := sigTx.GetSigners() for i, sig := range sigs { - signerAcc, err := GetSignerAcc(sdkCtx, sgcm.ak, signerAddrs[i]) + signerAcc, err := GetSignerAcc(ctx, sgcd.ak, signerAddrs[i]) if err != nil { - return err + return ctx, err } pubKey := signerAcc.GetPubKey() @@ -353,62 +181,29 @@ func (sgcm sigGasConsumeTxHandler) sigGasConsume(ctx context.Context, req tx.Req Sequence: sig.Sequence, } - err = sgcm.sigGasConsumer(sdkCtx.GasMeter(), sig, params) + err = sgcd.sigGasConsumer(ctx.GasMeter(), sig, params) if err != nil { - return err + return ctx, err } } - return nil -} - -// CheckTx implements tx.Handler.CheckTx. -func (sgcm sigGasConsumeTxHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - if err := sgcm.sigGasConsume(ctx, req, false); err != nil { - return tx.Response{}, tx.ResponseCheckTx{}, err - } - - return sgcm.next.CheckTx(ctx, req, checkReq) -} - -// DeliverTx implements tx.Handler.DeliverTx. -func (sgcm sigGasConsumeTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := sgcm.sigGasConsume(ctx, req, false); err != nil { - return tx.Response{}, err - } - - return sgcm.next.DeliverTx(ctx, req) + return next(ctx, tx, simulate) } -// SimulateTx implements tx.Handler.SimulateTx. -func (sgcm sigGasConsumeTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := sgcm.sigGasConsume(ctx, req, true); err != nil { - return tx.Response{}, err - } - - return sgcm.next.SimulateTx(ctx, req) -} - -var _ tx.Handler = sigVerificationTxHandler{} - -type sigVerificationTxHandler struct { +// Verify all signatures for a tx and return an error if any are invalid. Note, +// the SigVerificationDecorator decorator will not get executed on ReCheck. +// +// CONTRACT: Pubkeys are set in context for all signers before this decorator runs +// CONTRACT: Tx must implement SigVerifiableTx interface +type SigVerificationDecorator struct { ak AccountKeeper signModeHandler authsigning.SignModeHandler - next tx.Handler } -// SigVerificationMiddleware verifies all signatures for a tx and return an error if any are invalid. Note, -// the sigVerificationTxHandler middleware will not get executed on ReCheck. -// -// CONTRACT: Pubkeys are set in context for all signers before this middleware runs -// CONTRACT: Tx must implement SigVerifiableTx interface -func SigVerificationMiddleware(ak AccountKeeper, signModeHandler authsigning.SignModeHandler) tx.Middleware { - return func(h tx.Handler) tx.Handler { - return sigVerificationTxHandler{ - ak: ak, - signModeHandler: signModeHandler, - next: h, - } +func NewSigVerificationDecorator(ak AccountKeeper, signModeHandler authsigning.SignModeHandler) SigVerificationDecorator { + return SigVerificationDecorator{ + ak: ak, + signModeHandler: signModeHandler, } } @@ -416,7 +211,7 @@ func SigVerificationMiddleware(ak AccountKeeper, signModeHandler authsigning.Sig // signers are using SIGN_MODE_LEGACY_AMINO_JSON. If this is the case // then the corresponding SignatureV2 struct will not have account sequence // explicitly set, and we should skip the explicit verification of sig.Sequence -// in the SigVerificationMiddleware's middleware function. +// in the SigVerificationDecorator's AnteHandler function. func OnlyLegacyAminoSigners(sigData signing.SignatureData) bool { switch v := sigData.(type) { case *signing.SingleSignatureData: @@ -433,69 +228,65 @@ func OnlyLegacyAminoSigners(sigData signing.SignatureData) bool { } } -func (svd sigVerificationTxHandler) sigVerify(ctx context.Context, req tx.Request, isReCheckTx, simulate bool) error { - sdkCtx := sdk.UnwrapSDKContext(ctx) +func (svd SigVerificationDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (newCtx sdk.Context, err error) { // no need to verify signatures on recheck tx - if isReCheckTx { - return nil + if ctx.IsReCheckTx() { + return next(ctx, tx, simulate) } - sigTx, ok := req.Tx.(authsigning.SigVerifiableTx) + sigTx, ok := tx.(authsigning.SigVerifiableTx) if !ok { - return sdkerrors.Wrap(sdkerrors.ErrTxDecode, "invalid transaction type") + return ctx, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "invalid transaction type") } // stdSigs contains the sequence number, account number, and signatures. // When simulating, this would just be a 0-length slice. sigs, err := sigTx.GetSignaturesV2() if err != nil { - return err + return ctx, err } signerAddrs := sigTx.GetSigners() // check that signer length and signature length are the same if len(sigs) != len(signerAddrs) { - return sdkerrors.Wrapf(sdkerrors.ErrUnauthorized, "invalid number of signer; expected: %d, got %d", len(signerAddrs), len(sigs)) + return ctx, sdkerrors.Wrapf(sdkerrors.ErrUnauthorized, "invalid number of signer; expected: %d, got %d", len(signerAddrs), len(sigs)) } for i, sig := range sigs { - acc, err := GetSignerAcc(sdkCtx, svd.ak, signerAddrs[i]) + acc, err := GetSignerAcc(ctx, svd.ak, signerAddrs[i]) if err != nil { - return err + return ctx, err } // retrieve pubkey pubKey := acc.GetPubKey() if !simulate && pubKey == nil { - return sdkerrors.Wrap(sdkerrors.ErrInvalidPubKey, "pubkey on account is not set") + return ctx, sdkerrors.Wrap(sdkerrors.ErrInvalidPubKey, "pubkey on account is not set") } // Check account sequence number. if sig.Sequence != acc.GetSequence() { - return sdkerrors.Wrapf( + return ctx, sdkerrors.Wrapf( sdkerrors.ErrWrongSequence, "account sequence mismatch, expected %d, got %d", acc.GetSequence(), sig.Sequence, ) } // retrieve signer data - genesis := sdkCtx.BlockHeight() == 0 - chainID := sdkCtx.ChainID() + genesis := ctx.BlockHeight() == 0 + chainID := ctx.ChainID() var accNum uint64 if !genesis { accNum = acc.GetAccountNumber() } - signerData := authsigning.SignerData{ - Address: signerAddrs[i].String(), ChainID: chainID, AccountNumber: accNum, Sequence: acc.GetSequence(), - PubKey: pubKey, } if !simulate { - err := authsigning.VerifySignature(pubKey, signerData, sig.Data, svd.signModeHandler, req.Tx) + err := authsigning.VerifySignature(pubKey, signerData, sig.Data, svd.signModeHandler, tx) if err != nil { var errMsg string if OnlyLegacyAminoSigners(sig.Data) { @@ -505,112 +296,153 @@ func (svd sigVerificationTxHandler) sigVerify(ctx context.Context, req tx.Reques } else { errMsg = fmt.Sprintf("signature verification failed; please verify account number (%d) and chain-id (%s)", accNum, chainID) } - return sdkerrors.Wrap(sdkerrors.ErrUnauthorized, errMsg) + return ctx, sdkerrors.Wrap(sdkerrors.ErrUnauthorized, errMsg) } } } - return nil -} - -// CheckTx implements tx.Handler.CheckTx. -func (svd sigVerificationTxHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - if err := svd.sigVerify(ctx, req, checkReq.Type == abci.CheckTxType_Recheck, false); err != nil { - return tx.Response{}, tx.ResponseCheckTx{}, err - } - - return svd.next.CheckTx(ctx, req, checkReq) -} - -// DeliverTx implements tx.Handler.DeliverTx. -func (svd sigVerificationTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := svd.sigVerify(ctx, req, false, false); err != nil { - return tx.Response{}, err - } - - return svd.next.DeliverTx(ctx, req) -} - -// SimulateTx implements tx.Handler.SimulateTx. -func (svd sigVerificationTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := svd.sigVerify(ctx, req, false, true); err != nil { - return tx.Response{}, err - } - - return svd.next.SimulateTx(ctx, req) -} - -var _ tx.Handler = incrementSequenceTxHandler{} - -type incrementSequenceTxHandler struct { - ak AccountKeeper - next tx.Handler + return next(ctx, tx, simulate) } -// IncrementSequenceMiddleware handles incrementing sequences of all signers. -// Use the incrementSequenceTxHandler middleware to prevent replay attacks. Note, -// there is no need to execute incrementSequenceTxHandler on RecheckTX since +// IncrementSequenceDecorator handles incrementing sequences of all signers. +// Use the IncrementSequenceDecorator decorator to prevent replay attacks. Note, +// there is no need to execute IncrementSequenceDecorator on RecheckTX since // CheckTx would already bump the sequence number. // // NOTE: Since CheckTx and DeliverTx state are managed separately, subsequent and // sequential txs orginating from the same account cannot be handled correctly in // a reliable way unless sequence numbers are managed and tracked manually by a // client. It is recommended to instead use multiple messages in a tx. -func IncrementSequenceMiddleware(ak AccountKeeper) tx.Middleware { - return func(h tx.Handler) tx.Handler { - return incrementSequenceTxHandler{ - ak: ak, - next: h, - } +type IncrementSequenceDecorator struct { + ak AccountKeeper +} + +func NewIncrementSequenceDecorator(ak AccountKeeper) IncrementSequenceDecorator { + return IncrementSequenceDecorator{ + ak: ak, } } -func (isd incrementSequenceTxHandler) incrementSeq(ctx context.Context, req tx.Request) error { - sdkCtx := sdk.UnwrapSDKContext(ctx) - sigTx, ok := req.Tx.(authsigning.SigVerifiableTx) +func (isd IncrementSequenceDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (sdk.Context, error) { + sigTx, ok := tx.(authsigning.SigVerifiableTx) if !ok { - return sdkerrors.Wrap(sdkerrors.ErrTxDecode, "invalid transaction type") + return ctx, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "invalid transaction type") } // increment sequence of all signers for _, addr := range sigTx.GetSigners() { - acc := isd.ak.GetAccount(sdkCtx, addr) + acc := isd.ak.GetAccount(ctx, addr) if err := acc.SetSequence(acc.GetSequence() + 1); err != nil { panic(err) } - isd.ak.SetAccount(sdkCtx, acc) + isd.ak.SetAccount(ctx, acc) } - return nil + return next(ctx, tx, simulate) } -// CheckTx implements tx.Handler.CheckTx. -func (isd incrementSequenceTxHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - if err := isd.incrementSeq(ctx, req); err != nil { - return tx.Response{}, tx.ResponseCheckTx{}, err - } +// ValidateSigCountDecorator takes in Params and returns errors if there are too many signatures in the tx for the given params +// otherwise it calls next AnteHandler +// Use this decorator to set parameterized limit on number of signatures in tx +// CONTRACT: Tx must implement SigVerifiableTx interface +type ValidateSigCountDecorator struct { + ak AccountKeeper +} - return isd.next.CheckTx(ctx, req, checkReq) +func NewValidateSigCountDecorator(ak AccountKeeper) ValidateSigCountDecorator { + return ValidateSigCountDecorator{ + ak: ak, + } } -// DeliverTx implements tx.Handler.DeliverTx. -func (isd incrementSequenceTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := isd.incrementSeq(ctx, req); err != nil { - return tx.Response{}, err +func (vscd ValidateSigCountDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (sdk.Context, error) { + sigTx, ok := tx.(authsigning.SigVerifiableTx) + if !ok { + return ctx, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "Tx must be a sigTx") + } + + params := vscd.ak.GetParams(ctx) + pubKeys, err := sigTx.GetPubKeys() + if err != nil { + return ctx, err + } + + sigCount := 0 + for _, pk := range pubKeys { + sigCount += CountSubKeys(pk) + if uint64(sigCount) > params.TxSigLimit { + return ctx, sdkerrors.Wrapf(sdkerrors.ErrTooManySignatures, + "signatures: %d, limit: %d", sigCount, params.TxSigLimit) + } } - return isd.next.DeliverTx(ctx, req) + return next(ctx, tx, simulate) +} + +// DefaultSigVerificationGasConsumer is the default implementation of SignatureVerificationGasConsumer. It consumes gas +// for signature verification based upon the public key type. The cost is fetched from the given params and is matched +// by the concrete type. +func DefaultSigVerificationGasConsumer( + meter sdk.GasMeter, sig signing.SignatureV2, params types.Params, +) error { + pubkey := sig.PubKey + switch pubkey := pubkey.(type) { + case *ed25519.PubKey: + meter.ConsumeGas(params.SigVerifyCostED25519, "ante verify: ed25519") + return sdkerrors.Wrap(sdkerrors.ErrInvalidPubKey, "ED25519 public keys are unsupported") + + case *secp256k1.PubKey: + meter.ConsumeGas(params.SigVerifyCostSecp256k1, "ante verify: secp256k1") + return nil + + case *secp256r1.PubKey: + meter.ConsumeGas(params.SigVerifyCostSecp256r1(), "ante verify: secp256r1") + return nil + + case multisig.PubKey: + multisignature, ok := sig.Data.(*signing.MultiSignatureData) + if !ok { + return fmt.Errorf("expected %T, got, %T", &signing.MultiSignatureData{}, sig.Data) + } + err := ConsumeMultisignatureVerificationGas(meter, multisignature, pubkey, params, sig.Sequence) + if err != nil { + return err + } + return nil + + default: + return sdkerrors.Wrapf(sdkerrors.ErrInvalidPubKey, "unrecognized public key type: %T", pubkey) + } } -// SimulateTx implements tx.Handler.SimulateTx. -func (isd incrementSequenceTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := isd.incrementSeq(ctx, req); err != nil { - return tx.Response{}, err +// ConsumeMultisignatureVerificationGas consumes gas from a GasMeter for verifying a multisig pubkey signature +func ConsumeMultisignatureVerificationGas( + meter sdk.GasMeter, sig *signing.MultiSignatureData, pubkey multisig.PubKey, + params types.Params, accSeq uint64, +) error { + + size := sig.BitArray.Count() + sigIndex := 0 + + for i := 0; i < size; i++ { + if !sig.BitArray.GetIndex(i) { + continue + } + sigV2 := signing.SignatureV2{ + PubKey: pubkey.GetPubKeys()[i], + Data: sig.Signatures[sigIndex], + Sequence: accSeq, + } + err := DefaultSigVerificationGasConsumer(meter, sigV2, params) + if err != nil { + return err + } + sigIndex++ } - return isd.next.SimulateTx(ctx, req) + return nil } // GetSignerAcc returns an account for a given address that is expected to sign diff --git a/x/auth/middleware/sigverify_benchmark_test.go b/x/auth/ante/sigverify_benchmark_test.go similarity index 89% rename from x/auth/middleware/sigverify_benchmark_test.go rename to x/auth/ante/sigverify_benchmark_test.go index dc635985170b..56e596fa6b55 100644 --- a/x/auth/middleware/sigverify_benchmark_test.go +++ b/x/auth/ante/sigverify_benchmark_test.go @@ -1,4 +1,4 @@ -package middleware_test +package ante_test import ( "testing" @@ -10,7 +10,7 @@ import ( "github.com/cosmos/cosmos-sdk/crypto/keys/secp256r1" ) -// This benchmark is used to asses the middleware.Secp256k1ToR1GasFactor value +// This benchmark is used to asses the ante.Secp256k1ToR1GasFactor value func BenchmarkSig(b *testing.B) { require := require.New(b) msg := tmcrypto.CRandBytes(1000) diff --git a/x/auth/middleware/sigverify_test.go b/x/auth/ante/sigverify_test.go similarity index 52% rename from x/auth/middleware/sigverify_test.go rename to x/auth/ante/sigverify_test.go index fd4fc5622041..0ddd2d226e0f 100644 --- a/x/auth/middleware/sigverify_test.go +++ b/x/auth/ante/sigverify_test.go @@ -1,10 +1,10 @@ -package middleware_test +package ante_test import ( "fmt" "github.com/cosmos/cosmos-sdk/client" - "github.com/cosmos/cosmos-sdk/codec/legacy" + "github.com/cosmos/cosmos-sdk/codec" "github.com/cosmos/cosmos-sdk/crypto/keys/ed25519" kmultisig "github.com/cosmos/cosmos-sdk/crypto/keys/multisig" "github.com/cosmos/cosmos-sdk/crypto/keys/secp256k1" @@ -14,22 +14,16 @@ import ( "github.com/cosmos/cosmos-sdk/simapp" "github.com/cosmos/cosmos-sdk/testutil/testdata" sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/types/tx" "github.com/cosmos/cosmos-sdk/types/tx/signing" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" + "github.com/cosmos/cosmos-sdk/x/auth/ante" "github.com/cosmos/cosmos-sdk/x/auth/migrations/legacytx" "github.com/cosmos/cosmos-sdk/x/auth/types" - abci "github.com/tendermint/tendermint/abci/types" ) -func (s *MWTestSuite) TestSetPubKey() { - ctx := s.SetupTest(true) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() - require := s.Require() - txHandler := middleware.ComposeMiddlewares( - noopTxHandler, - middleware.SetPubKeyMiddleware(s.app.AccountKeeper), - ) +func (suite *AnteTestSuite) TestSetPubKey() { + suite.SetupTest(true) // setup + require := suite.Require() + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() // keys and addresses priv1, pub1, addr1 := testdata.KeyTestPubAddr() @@ -42,37 +36,35 @@ func (s *MWTestSuite) TestSetPubKey() { msgs := make([]sdk.Msg, len(addrs)) // set accounts and create msg for each address for i, addr := range addrs { - acc := s.app.AccountKeeper.NewAccountWithAddress(ctx, addr) + acc := suite.app.AccountKeeper.NewAccountWithAddress(suite.ctx, addr) require.NoError(acc.SetAccountNumber(uint64(i))) - s.app.AccountKeeper.SetAccount(ctx, acc) + suite.app.AccountKeeper.SetAccount(suite.ctx, acc) msgs[i] = testdata.NewTestMsg(addr) } - require.NoError(txBuilder.SetMsgs(msgs...)) - txBuilder.SetFeeAmount(testdata.NewTestFeeAmount()) - txBuilder.SetGasLimit(testdata.NewTestGasLimit()) + require.NoError(suite.txBuilder.SetMsgs(msgs...)) + suite.txBuilder.SetFeeAmount(testdata.NewTestFeeAmount()) + suite.txBuilder.SetGasLimit(testdata.NewTestGasLimit()) privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1, priv2, priv3}, []uint64{0, 1, 2}, []uint64{0, 0, 0} - testTx, _, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) + tx, err := suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) require.NoError(err) - // DeliverTx - _, err = txHandler.DeliverTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}) + spkd := ante.NewSetPubKeyDecorator(suite.app.AccountKeeper) + antehandler := sdk.ChainAnteDecorators(spkd) + + ctx, err := antehandler(suite.ctx, tx, false) require.NoError(err) - // Require that all accounts have pubkey set after middleware runs + // Require that all accounts have pubkey set after Decorator runs for i, addr := range addrs { - pk, err := s.app.AccountKeeper.GetPubKey(ctx, addr) + pk, err := suite.app.AccountKeeper.GetPubKey(ctx, addr) require.NoError(err, "Error on retrieving pubkey from account") require.True(pubs[i].Equals(pk), "Wrong Pubkey retrieved from AccountKeeper, idx=%d\nexpected=%s\n got=%s", i, pubs[i], pk) } - - // SimulateTx - _, err = txHandler.SimulateTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}) - require.NoError(err) } -func (s *MWTestSuite) TestConsumeSignatureVerificationGas() { +func (suite *AnteTestSuite) TestConsumeSignatureVerificationGas() { params := types.DefaultParams() msg := []byte{1, 2, 3, 4} cdc := simapp.MakeTestEncodingConfig().Amino @@ -86,9 +78,9 @@ func (s *MWTestSuite) TestConsumeSignatureVerificationGas() { for i := 0; i < len(pkSet1); i++ { stdSig := legacytx.StdSignature{PubKey: pkSet1[i], Signature: sigSet1[i]} sigV2, err := legacytx.StdSignatureToSignatureV2(cdc, stdSig) - s.Require().NoError(err) + suite.Require().NoError(err) err = multisig.AddSignatureV2(multisignature1, sigV2, pkSet1) - s.Require().NoError(err) + suite.Require().NoError(err) } type args struct { @@ -115,30 +107,23 @@ func (s *MWTestSuite) TestConsumeSignatureVerificationGas() { Data: tt.args.sig, Sequence: 0, // Arbitrary account sequence } - err := middleware.DefaultSigVerificationGasConsumer(tt.args.meter, sigV2, tt.args.params) + err := ante.DefaultSigVerificationGasConsumer(tt.args.meter, sigV2, tt.args.params) if tt.shouldErr { - s.Require().NotNil(err) + suite.Require().NotNil(err) } else { - s.Require().Nil(err) - s.Require().Equal(tt.gasConsumed, tt.args.meter.GasConsumed(), fmt.Sprintf("%d != %d", tt.gasConsumed, tt.args.meter.GasConsumed())) + suite.Require().Nil(err) + suite.Require().Equal(tt.gasConsumed, tt.args.meter.GasConsumed(), fmt.Sprintf("%d != %d", tt.gasConsumed, tt.args.meter.GasConsumed())) } } } -func (s *MWTestSuite) TestSigVerification() { - ctx := s.SetupTest(true) // setup +func (suite *AnteTestSuite) TestSigVerification() { + suite.SetupTest(true) // setup + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() // make block height non-zero to ensure account numbers part of signBytes - ctx = ctx.WithBlockHeight(1) - txHandler := middleware.ComposeMiddlewares( - noopTxHandler, - middleware.SetPubKeyMiddleware(s.app.AccountKeeper), - middleware.SigVerificationMiddleware( - s.app.AccountKeeper, - s.clientCtx.TxConfig.SignModeHandler(), - ), - ) + suite.ctx = suite.ctx.WithBlockHeight(1) // keys and addresses priv1, _, addr1 := testdata.KeyTestPubAddr() @@ -150,15 +135,19 @@ func (s *MWTestSuite) TestSigVerification() { msgs := make([]sdk.Msg, len(addrs)) // set accounts and create msg for each address for i, addr := range addrs { - acc := s.app.AccountKeeper.NewAccountWithAddress(ctx, addr) - s.Require().NoError(acc.SetAccountNumber(uint64(i))) - s.app.AccountKeeper.SetAccount(ctx, acc) + acc := suite.app.AccountKeeper.NewAccountWithAddress(suite.ctx, addr) + suite.Require().NoError(acc.SetAccountNumber(uint64(i))) + suite.app.AccountKeeper.SetAccount(suite.ctx, acc) msgs[i] = testdata.NewTestMsg(addr) } feeAmount := testdata.NewTestFeeAmount() gasLimit := testdata.NewTestGasLimit() + spkd := ante.NewSetPubKeyDecorator(suite.app.AccountKeeper) + svd := ante.NewSigVerificationDecorator(suite.app.AccountKeeper, suite.clientCtx.TxConfig.SignModeHandler()) + antehandler := sdk.ChainAnteDecorators(spkd, svd) + type testCase struct { name string privs []cryptotypes.PrivKey @@ -177,25 +166,21 @@ func (s *MWTestSuite) TestSigVerification() { {"no err on recheck", []cryptotypes.PrivKey{}, []uint64{}, []uint64{}, true, false}, } for i, tc := range testCases { - ctx = ctx.WithIsReCheckTx(tc.recheck) - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() // Create new txBuilder for each test + suite.ctx = suite.ctx.WithIsReCheckTx(tc.recheck) + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() // Create new txBuilder for each test - s.Require().NoError(txBuilder.SetMsgs(msgs...)) - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(gasLimit) + suite.Require().NoError(suite.txBuilder.SetMsgs(msgs...)) + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(gasLimit) - testTx, _, err := s.createTestTx(txBuilder, tc.privs, tc.accNums, tc.accSeqs, ctx.ChainID()) - s.Require().NoError(err) + tx, err := suite.CreateTestTx(tc.privs, tc.accNums, tc.accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) - if tc.recheck { - _, _, err = txHandler.CheckTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}, tx.RequestCheckTx{Type: abci.CheckTxType_Recheck}) - } else { - _, _, err = txHandler.CheckTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}, tx.RequestCheckTx{}) - } + _, err = antehandler(suite.ctx, tx, false) if tc.shouldErr { - s.Require().NotNil(err, "TestCase %d: %s did not error as expected", i, tc.name) + suite.Require().NotNil(err, "TestCase %d: %s did not error as expected", i, tc.name) } else { - s.Require().Nil(err, "TestCase %d: %s errored unexpectedly. Err: %v", i, tc.name, err) + suite.Require().Nil(err, "TestCase %d: %s errored unexpectedly. Err: %v", i, tc.name, err) } } } @@ -206,22 +191,35 @@ func (s *MWTestSuite) TestSigVerification() { // this, since it'll be handled by the test matrix. // In the meantime, we want to make double-sure amino compatibility works. // ref: https://github.com/cosmos/cosmos-sdk/issues/7229 -func (s *MWTestSuite) TestSigVerification_ExplicitAmino() { - ctx := s.SetupTest(true) - ctx = ctx.WithBlockHeight(1) +func (suite *AnteTestSuite) TestSigVerification_ExplicitAmino() { + suite.app, suite.ctx = createTestApp(true) + suite.ctx = suite.ctx.WithBlockHeight(1) // Set up TxConfig. - aminoCdc := legacy.Cdc - aminoCdc.RegisterConcrete(&testdata.TestMsg{}, "testdata.TestMsg", nil) - + aminoCdc := codec.NewLegacyAmino() // We're using TestMsg amino encoding in some tests, so register it here. txConfig := legacytx.StdTxConfig{Cdc: aminoCdc} - s.clientCtx = client.Context{}. + suite.clientCtx = client.Context{}. WithTxConfig(txConfig) + anteHandler, err := ante.NewAnteHandler( + ante.HandlerOptions{ + AccountKeeper: suite.app.AccountKeeper, + BankKeeper: suite.app.BankKeeper, + FeegrantKeeper: suite.app.FeeGrantKeeper, + SignModeHandler: txConfig.SignModeHandler(), + SigGasConsumer: ante.DefaultSigVerificationGasConsumer, + }, + ) + + suite.Require().NoError(err) + suite.anteHandler = anteHandler + + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() + // make block height non-zero to ensure account numbers part of signBytes - ctx = ctx.WithBlockHeight(1) + suite.ctx = suite.ctx.WithBlockHeight(1) // keys and addresses priv1, _, addr1 := testdata.KeyTestPubAddr() @@ -233,23 +231,18 @@ func (s *MWTestSuite) TestSigVerification_ExplicitAmino() { msgs := make([]sdk.Msg, len(addrs)) // set accounts and create msg for each address for i, addr := range addrs { - acc := s.app.AccountKeeper.NewAccountWithAddress(ctx, addr) - s.Require().NoError(acc.SetAccountNumber(uint64(i))) - s.app.AccountKeeper.SetAccount(ctx, acc) + acc := suite.app.AccountKeeper.NewAccountWithAddress(suite.ctx, addr) + suite.Require().NoError(acc.SetAccountNumber(uint64(i))) + suite.app.AccountKeeper.SetAccount(suite.ctx, acc) msgs[i] = testdata.NewTestMsg(addr) } feeAmount := testdata.NewTestFeeAmount() gasLimit := testdata.NewTestGasLimit() - txHandler := middleware.ComposeMiddlewares( - noopTxHandler, - middleware.SetPubKeyMiddleware(s.app.AccountKeeper), - middleware.SigVerificationMiddleware( - s.app.AccountKeeper, - s.clientCtx.TxConfig.SignModeHandler(), - ), - ) + spkd := ante.NewSetPubKeyDecorator(suite.app.AccountKeeper) + svd := ante.NewSigVerificationDecorator(suite.app.AccountKeeper, suite.clientCtx.TxConfig.SignModeHandler()) + antehandler := sdk.ChainAnteDecorators(spkd, svd) type testCase struct { name string @@ -259,7 +252,6 @@ func (s *MWTestSuite) TestSigVerification_ExplicitAmino() { recheck bool shouldErr bool } - testCases := []testCase{ {"no signers", []cryptotypes.PrivKey{}, []uint64{}, []uint64{}, false, true}, {"not enough signers", []cryptotypes.PrivKey{priv1, priv2}, []uint64{0, 1}, []uint64{0, 0}, false, true}, @@ -269,32 +261,27 @@ func (s *MWTestSuite) TestSigVerification_ExplicitAmino() { {"valid tx", []cryptotypes.PrivKey{priv1, priv2, priv3}, []uint64{0, 1, 2}, []uint64{0, 0, 0}, false, false}, {"no err on recheck", []cryptotypes.PrivKey{}, []uint64{}, []uint64{}, true, false}, } - for i, tc := range testCases { - ctx = ctx.WithIsReCheckTx(tc.recheck) - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() // Create new txBuilder for each test + suite.ctx = suite.ctx.WithIsReCheckTx(tc.recheck) + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() // Create new txBuilder for each test - s.Require().NoError(txBuilder.SetMsgs(msgs...)) - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(gasLimit) + suite.Require().NoError(suite.txBuilder.SetMsgs(msgs...)) + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(gasLimit) - testTx, _, err := s.createTestTx(txBuilder, tc.privs, tc.accNums, tc.accSeqs, ctx.ChainID()) - s.Require().NoError(err) + tx, err := suite.CreateTestTx(tc.privs, tc.accNums, tc.accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) - if tc.recheck { - _, _, err = txHandler.CheckTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}, tx.RequestCheckTx{Type: abci.CheckTxType_Recheck}) - } else { - _, _, err = txHandler.CheckTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}, tx.RequestCheckTx{}) - } + _, err = antehandler(suite.ctx, tx, false) if tc.shouldErr { - s.Require().NotNil(err, "TestCase %d: %s did not error as expected", i, tc.name) + suite.Require().NotNil(err, "TestCase %d: %s did not error as expected", i, tc.name) } else { - s.Require().Nil(err, "TestCase %d: %s errored unexpectedly. Err: %v", i, tc.name, err) + suite.Require().Nil(err, "TestCase %d: %s errored unexpectedly. Err: %v", i, tc.name, err) } } } -func (s *MWTestSuite) TestSigIntegration() { +func (suite *AnteTestSuite) TestSigIntegration() { // generate private keys privs := []cryptotypes.PrivKey{ secp256k1.GenPrivKey(), @@ -304,23 +291,23 @@ func (s *MWTestSuite) TestSigIntegration() { params := types.DefaultParams() initialSigCost := params.SigVerifyCostSecp256k1 - initialCost, err := s.runSigMiddlewares(params, false, privs...) - s.Require().Nil(err) + initialCost, err := suite.runSigDecorators(params, false, privs...) + suite.Require().Nil(err) params.SigVerifyCostSecp256k1 *= 2 - doubleCost, err := s.runSigMiddlewares(params, false, privs...) - s.Require().Nil(err) + doubleCost, err := suite.runSigDecorators(params, false, privs...) + suite.Require().Nil(err) - s.Require().Equal(initialSigCost*uint64(len(privs)), doubleCost-initialCost) + suite.Require().Equal(initialSigCost*uint64(len(privs)), doubleCost-initialCost) } -func (s *MWTestSuite) runSigMiddlewares(params types.Params, _ bool, privs ...cryptotypes.PrivKey) (sdk.Gas, error) { - ctx := s.SetupTest(true) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() +func (suite *AnteTestSuite) runSigDecorators(params types.Params, _ bool, privs ...cryptotypes.PrivKey) (sdk.Gas, error) { + suite.SetupTest(true) // setup + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() // Make block-height non-zero to include accNum in SignBytes - ctx = ctx.WithBlockHeight(1) - s.app.AccountKeeper.SetParams(ctx, params) + suite.ctx = suite.ctx.WithBlockHeight(1) + suite.app.AccountKeeper.SetParams(suite.ctx, params) msgs := make([]sdk.Msg, len(privs)) accNums := make([]uint64, len(privs)) @@ -328,89 +315,76 @@ func (s *MWTestSuite) runSigMiddlewares(params types.Params, _ bool, privs ...cr // set accounts and create msg for each address for i, priv := range privs { addr := sdk.AccAddress(priv.PubKey().Address()) - acc := s.app.AccountKeeper.NewAccountWithAddress(ctx, addr) - s.Require().NoError(acc.SetAccountNumber(uint64(i))) - s.app.AccountKeeper.SetAccount(ctx, acc) + acc := suite.app.AccountKeeper.NewAccountWithAddress(suite.ctx, addr) + suite.Require().NoError(acc.SetAccountNumber(uint64(i))) + suite.app.AccountKeeper.SetAccount(suite.ctx, acc) msgs[i] = testdata.NewTestMsg(addr) accNums[i] = uint64(i) accSeqs[i] = uint64(0) } - s.Require().NoError(txBuilder.SetMsgs(msgs...)) + suite.Require().NoError(suite.txBuilder.SetMsgs(msgs...)) feeAmount := testdata.NewTestFeeAmount() gasLimit := testdata.NewTestGasLimit() - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(gasLimit) - - testTx, _, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) - - txHandler := middleware.ComposeMiddlewares( - noopTxHandler, - middleware.SetPubKeyMiddleware(s.app.AccountKeeper), - middleware.SigGasConsumeMiddleware(s.app.AccountKeeper, middleware.DefaultSigVerificationGasConsumer), - middleware.SigVerificationMiddleware( - s.app.AccountKeeper, - s.clientCtx.TxConfig.SignModeHandler(), - ), - ) + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(gasLimit) - // Determine gas consumption of txhandler with default params - before := ctx.GasMeter().GasConsumed() - _, err = txHandler.DeliverTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}) + tx, err := suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) + + spkd := ante.NewSetPubKeyDecorator(suite.app.AccountKeeper) + svgc := ante.NewSigGasConsumeDecorator(suite.app.AccountKeeper, ante.DefaultSigVerificationGasConsumer) + svd := ante.NewSigVerificationDecorator(suite.app.AccountKeeper, suite.clientCtx.TxConfig.SignModeHandler()) + antehandler := sdk.ChainAnteDecorators(spkd, svgc, svd) + + // Determine gas consumption of antehandler with default params + before := suite.ctx.GasMeter().GasConsumed() + ctx, err := antehandler(suite.ctx, tx, false) after := ctx.GasMeter().GasConsumed() return after - before, err } -func (s *MWTestSuite) TestIncrementSequenceMiddleware() { - ctx := s.SetupTest(true) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() +func (suite *AnteTestSuite) TestIncrementSequenceDecorator() { + suite.SetupTest(true) // setup + suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() priv, _, addr := testdata.KeyTestPubAddr() - acc := s.app.AccountKeeper.NewAccountWithAddress(ctx, addr) - s.Require().NoError(acc.SetAccountNumber(uint64(50))) - s.app.AccountKeeper.SetAccount(ctx, acc) + acc := suite.app.AccountKeeper.NewAccountWithAddress(suite.ctx, addr) + suite.Require().NoError(acc.SetAccountNumber(uint64(50))) + suite.app.AccountKeeper.SetAccount(suite.ctx, acc) msgs := []sdk.Msg{testdata.NewTestMsg(addr)} - s.Require().NoError(txBuilder.SetMsgs(msgs...)) + suite.Require().NoError(suite.txBuilder.SetMsgs(msgs...)) privs := []cryptotypes.PrivKey{priv} - accNums := []uint64{s.app.AccountKeeper.GetAccount(ctx, addr).GetAccountNumber()} - accSeqs := []uint64{s.app.AccountKeeper.GetAccount(ctx, addr).GetSequence()} + accNums := []uint64{suite.app.AccountKeeper.GetAccount(suite.ctx, addr).GetAccountNumber()} + accSeqs := []uint64{suite.app.AccountKeeper.GetAccount(suite.ctx, addr).GetSequence()} feeAmount := testdata.NewTestFeeAmount() gasLimit := testdata.NewTestGasLimit() - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(gasLimit) + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(gasLimit) - testTx, _, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) + tx, err := suite.CreateTestTx(privs, accNums, accSeqs, suite.ctx.ChainID()) + suite.Require().NoError(err) - txHandler := middleware.ComposeMiddlewares( - noopTxHandler, - middleware.IncrementSequenceMiddleware(s.app.AccountKeeper), - ) + isd := ante.NewIncrementSequenceDecorator(suite.app.AccountKeeper) + antehandler := sdk.ChainAnteDecorators(isd) testCases := []struct { ctx sdk.Context simulate bool expectedSeq uint64 }{ - {ctx.WithIsReCheckTx(true), false, 1}, - {ctx.WithIsCheckTx(true).WithIsReCheckTx(false), false, 2}, - {ctx.WithIsReCheckTx(true), false, 3}, - {ctx.WithIsReCheckTx(true), false, 4}, - {ctx.WithIsReCheckTx(true), true, 5}, + {suite.ctx.WithIsReCheckTx(true), false, 1}, + {suite.ctx.WithIsCheckTx(true).WithIsReCheckTx(false), false, 2}, + {suite.ctx.WithIsReCheckTx(true), false, 3}, + {suite.ctx.WithIsReCheckTx(true), false, 4}, + {suite.ctx.WithIsReCheckTx(true), true, 5}, } for i, tc := range testCases { - var err error - if tc.simulate { - _, err = txHandler.SimulateTx(sdk.WrapSDKContext(tc.ctx), tx.Request{Tx: testTx}) - } else { - _, err = txHandler.DeliverTx(sdk.WrapSDKContext(tc.ctx), tx.Request{Tx: testTx}) - } - - s.Require().NoError(err, "unexpected error; tc #%d, %v", i, tc) - s.Require().Equal(tc.expectedSeq, s.app.AccountKeeper.GetAccount(ctx, addr).GetSequence()) + _, err := antehandler(tc.ctx, tx, tc.simulate) + suite.Require().NoError(err, "unexpected error; tc #%d, %v", i, tc) + suite.Require().Equal(tc.expectedSeq, suite.app.AccountKeeper.GetAccount(suite.ctx, addr).GetSequence()) } } diff --git a/x/auth/ante/testutil_test.go b/x/auth/ante/testutil_test.go new file mode 100644 index 000000000000..74216420eb14 --- /dev/null +++ b/x/auth/ante/testutil_test.go @@ -0,0 +1,200 @@ +package ante_test + +import ( + "errors" + "fmt" + "testing" + + minttypes "github.com/cosmos/cosmos-sdk/x/mint/types" + + "github.com/stretchr/testify/suite" + tmproto "github.com/tendermint/tendermint/proto/tendermint/types" + + "github.com/cosmos/cosmos-sdk/client" + "github.com/cosmos/cosmos-sdk/client/tx" + cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" + "github.com/cosmos/cosmos-sdk/simapp" + "github.com/cosmos/cosmos-sdk/testutil/testdata" + sdk "github.com/cosmos/cosmos-sdk/types" + "github.com/cosmos/cosmos-sdk/types/tx/signing" + "github.com/cosmos/cosmos-sdk/x/auth/ante" + xauthsigning "github.com/cosmos/cosmos-sdk/x/auth/signing" + "github.com/cosmos/cosmos-sdk/x/auth/types" + authtypes "github.com/cosmos/cosmos-sdk/x/auth/types" +) + +// TestAccount represents an account used in the tests in x/auth/ante. +type TestAccount struct { + acc types.AccountI + priv cryptotypes.PrivKey +} + +// AnteTestSuite is a test suite to be used with ante handler tests. +type AnteTestSuite struct { + suite.Suite + + app *simapp.SimApp + anteHandler sdk.AnteHandler + ctx sdk.Context + clientCtx client.Context + txBuilder client.TxBuilder +} + +// returns context and app with params set on account keeper +func createTestApp(isCheckTx bool) (*simapp.SimApp, sdk.Context) { + app := simapp.Setup(isCheckTx) + ctx := app.BaseApp.NewContext(isCheckTx, tmproto.Header{}) + app.AccountKeeper.SetParams(ctx, authtypes.DefaultParams()) + + return app, ctx +} + +// SetupTest setups a new test, with new app, context, and anteHandler. +func (suite *AnteTestSuite) SetupTest(isCheckTx bool) { + suite.app, suite.ctx = createTestApp(isCheckTx) + suite.ctx = suite.ctx.WithBlockHeight(1) + + // Set up TxConfig. + encodingConfig := simapp.MakeTestEncodingConfig() + // We're using TestMsg encoding in some tests, so register it here. + encodingConfig.Amino.RegisterConcrete(&testdata.TestMsg{}, "testdata.TestMsg", nil) + testdata.RegisterInterfaces(encodingConfig.InterfaceRegistry) + + suite.clientCtx = client.Context{}. + WithTxConfig(encodingConfig.TxConfig) + + anteHandler, err := ante.NewAnteHandler( + ante.HandlerOptions{ + AccountKeeper: suite.app.AccountKeeper, + BankKeeper: suite.app.BankKeeper, + FeegrantKeeper: suite.app.FeeGrantKeeper, + SignModeHandler: encodingConfig.TxConfig.SignModeHandler(), + SigGasConsumer: ante.DefaultSigVerificationGasConsumer, + }, + ) + + suite.Require().NoError(err) + suite.anteHandler = anteHandler +} + +// CreateTestAccounts creates `numAccs` accounts, and return all relevant +// information about them including their private keys. +func (suite *AnteTestSuite) CreateTestAccounts(numAccs int) []TestAccount { + var accounts []TestAccount + + for i := 0; i < numAccs; i++ { + priv, _, addr := testdata.KeyTestPubAddr() + acc := suite.app.AccountKeeper.NewAccountWithAddress(suite.ctx, addr) + err := acc.SetAccountNumber(uint64(i)) + suite.Require().NoError(err) + suite.app.AccountKeeper.SetAccount(suite.ctx, acc) + someCoins := sdk.Coins{ + sdk.NewInt64Coin("atom", 10000000), + } + err = suite.app.BankKeeper.MintCoins(suite.ctx, minttypes.ModuleName, someCoins) + suite.Require().NoError(err) + + err = suite.app.BankKeeper.SendCoinsFromModuleToAccount(suite.ctx, minttypes.ModuleName, addr, someCoins) + suite.Require().NoError(err) + + accounts = append(accounts, TestAccount{acc, priv}) + } + + return accounts +} + +// CreateTestTx is a helper function to create a tx given multiple inputs. +func (suite *AnteTestSuite) CreateTestTx(privs []cryptotypes.PrivKey, accNums []uint64, accSeqs []uint64, chainID string) (xauthsigning.Tx, error) { + // First round: we gather all the signer infos. We use the "set empty + // signature" hack to do that. + var sigsV2 []signing.SignatureV2 + for i, priv := range privs { + sigV2 := signing.SignatureV2{ + PubKey: priv.PubKey(), + Data: &signing.SingleSignatureData{ + SignMode: suite.clientCtx.TxConfig.SignModeHandler().DefaultMode(), + Signature: nil, + }, + Sequence: accSeqs[i], + } + + sigsV2 = append(sigsV2, sigV2) + } + err := suite.txBuilder.SetSignatures(sigsV2...) + if err != nil { + return nil, err + } + + // Second round: all signer infos are set, so each signer can sign. + sigsV2 = []signing.SignatureV2{} + for i, priv := range privs { + signerData := xauthsigning.SignerData{ + ChainID: chainID, + AccountNumber: accNums[i], + Sequence: accSeqs[i], + } + sigV2, err := tx.SignWithPrivKey( + suite.clientCtx.TxConfig.SignModeHandler().DefaultMode(), signerData, + suite.txBuilder, priv, suite.clientCtx.TxConfig, accSeqs[i]) + if err != nil { + return nil, err + } + + sigsV2 = append(sigsV2, sigV2) + } + err = suite.txBuilder.SetSignatures(sigsV2...) + if err != nil { + return nil, err + } + + return suite.txBuilder.GetTx(), nil +} + +// TestCase represents a test case used in test tables. +type TestCase struct { + desc string + malleate func() + simulate bool + expPass bool + expErr error +} + +// CreateTestTx is a helper function to create a tx given multiple inputs. +func (suite *AnteTestSuite) RunTestCase(privs []cryptotypes.PrivKey, msgs []sdk.Msg, feeAmount sdk.Coins, gasLimit uint64, accNums, accSeqs []uint64, chainID string, tc TestCase) { + suite.Run(fmt.Sprintf("Case %s", tc.desc), func() { + suite.Require().NoError(suite.txBuilder.SetMsgs(msgs...)) + suite.txBuilder.SetFeeAmount(feeAmount) + suite.txBuilder.SetGasLimit(gasLimit) + + // Theoretically speaking, ante handler unit tests should only test + // ante handlers, but here we sometimes also test the tx creation + // process. + tx, txErr := suite.CreateTestTx(privs, accNums, accSeqs, chainID) + newCtx, anteErr := suite.anteHandler(suite.ctx, tx, tc.simulate) + + if tc.expPass { + suite.Require().NoError(txErr) + suite.Require().NoError(anteErr) + suite.Require().NotNil(newCtx) + + suite.ctx = newCtx + } else { + switch { + case txErr != nil: + suite.Require().Error(txErr) + suite.Require().True(errors.Is(txErr, tc.expErr)) + + case anteErr != nil: + suite.Require().Error(anteErr) + suite.Require().True(errors.Is(anteErr, tc.expErr)) + + default: + suite.Fail("expected one of txErr,anteErr to be an error") + } + } + }) +} + +func TestAnteTestSuite(t *testing.T) { + suite.Run(t, new(AnteTestSuite)) +} diff --git a/x/auth/middleware/basic.go b/x/auth/middleware/basic.go deleted file mode 100644 index 429c9a39537f..000000000000 --- a/x/auth/middleware/basic.go +++ /dev/null @@ -1,359 +0,0 @@ -package middleware - -import ( - "context" - - "github.com/cosmos/cosmos-sdk/codec/legacy" - "github.com/cosmos/cosmos-sdk/crypto/keys/multisig" - cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" - sdk "github.com/cosmos/cosmos-sdk/types" - sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/types/tx/signing" - "github.com/cosmos/cosmos-sdk/x/auth/migrations/legacytx" - authsigning "github.com/cosmos/cosmos-sdk/x/auth/signing" - abci "github.com/tendermint/tendermint/abci/types" -) - -type validateBasicTxHandler struct { - next tx.Handler -} - -// ValidateBasicMiddleware will call tx.ValidateBasic, msg.ValidateBasic(for each msg inside tx) -// and return any non-nil error. -// If ValidateBasic passes, middleware calls next middleware in chain. Note, -// validateBasicTxHandler will not get executed on ReCheckTx since it -// is not dependent on application state. -func ValidateBasicMiddleware(txh tx.Handler) tx.Handler { - return validateBasicTxHandler{ - next: txh, - } -} - -var _ tx.Handler = validateBasicTxHandler{} - -// validateBasicTxMsgs executes basic validator calls for messages. -func validateBasicTxMsgs(msgs []sdk.Msg) error { - if len(msgs) == 0 { - return sdkerrors.Wrap(sdkerrors.ErrInvalidRequest, "must contain at least one message") - } - - for _, msg := range msgs { - err := msg.ValidateBasic() - if err != nil { - return err - } - } - - return nil -} - -// CheckTx implements tx.Handler.CheckTx. -func (txh validateBasicTxHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - // no need to validate basic on recheck tx, call next middleware - if checkReq.Type == abci.CheckTxType_Recheck { - return txh.next.CheckTx(ctx, req, checkReq) - } - - if err := validateBasicTxMsgs(req.Tx.GetMsgs()); err != nil { - return tx.Response{}, tx.ResponseCheckTx{}, err - } - - if err := req.Tx.ValidateBasic(); err != nil { - return tx.Response{}, tx.ResponseCheckTx{}, err - } - - return txh.next.CheckTx(ctx, req, checkReq) -} - -// DeliverTx implements tx.Handler.DeliverTx. -func (txh validateBasicTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := req.Tx.ValidateBasic(); err != nil { - return tx.Response{}, err - } - - if err := validateBasicTxMsgs(req.Tx.GetMsgs()); err != nil { - return tx.Response{}, err - } - - return txh.next.DeliverTx(ctx, req) -} - -// SimulateTx implements tx.Handler.SimulateTx. -func (txh validateBasicTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := req.Tx.ValidateBasic(); err != nil { - return tx.Response{}, err - } - - if err := validateBasicTxMsgs(req.Tx.GetMsgs()); err != nil { - return tx.Response{}, err - } - - return txh.next.SimulateTx(ctx, req) -} - -var _ tx.Handler = txTimeoutHeightTxHandler{} - -type txTimeoutHeightTxHandler struct { - next tx.Handler -} - -// TxTimeoutHeightMiddleware defines a middleware that checks for a -// tx height timeout. -func TxTimeoutHeightMiddleware(txh tx.Handler) tx.Handler { - return txTimeoutHeightTxHandler{ - next: txh, - } -} - -func checkTimeout(ctx context.Context, tx sdk.Tx) error { - sdkCtx := sdk.UnwrapSDKContext(ctx) - timeoutTx, ok := tx.(sdk.TxWithTimeoutHeight) - if !ok { - return sdkerrors.Wrap(sdkerrors.ErrTxDecode, "expected tx to implement TxWithTimeoutHeight") - } - - timeoutHeight := timeoutTx.GetTimeoutHeight() - if timeoutHeight > 0 && uint64(sdkCtx.BlockHeight()) > timeoutHeight { - return sdkerrors.Wrapf( - sdkerrors.ErrTxTimeoutHeight, "block height: %d, timeout height: %d", sdkCtx.BlockHeight(), timeoutHeight, - ) - } - - return nil -} - -// CheckTx implements tx.Handler.CheckTx. -func (txh txTimeoutHeightTxHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - if err := checkTimeout(ctx, req.Tx); err != nil { - return tx.Response{}, tx.ResponseCheckTx{}, err - } - - return txh.next.CheckTx(ctx, req, checkReq) -} - -// DeliverTx implements tx.Handler.DeliverTx. -func (txh txTimeoutHeightTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := checkTimeout(ctx, req.Tx); err != nil { - return tx.Response{}, err - } - - return txh.next.DeliverTx(ctx, req) -} - -// SimulateTx implements tx.Handler.SimulateTx. -func (txh txTimeoutHeightTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := checkTimeout(ctx, req.Tx); err != nil { - return tx.Response{}, err - } - - return txh.next.SimulateTx(ctx, req) -} - -type validateMemoTxHandler struct { - ak AccountKeeper - next tx.Handler -} - -// ValidateMemoMiddleware will validate memo given the parameters passed in -// If memo is too large middleware returns with error, otherwise call next middleware -// CONTRACT: Tx must implement TxWithMemo interface -func ValidateMemoMiddleware(ak AccountKeeper) tx.Middleware { - return func(txHandler tx.Handler) tx.Handler { - return validateMemoTxHandler{ - ak: ak, - next: txHandler, - } - } -} - -var _ tx.Handler = validateMemoTxHandler{} - -func (vmm validateMemoTxHandler) checkForValidMemo(ctx context.Context, tx sdk.Tx) error { - sdkCtx := sdk.UnwrapSDKContext(ctx) - memoTx, ok := tx.(sdk.TxWithMemo) - if !ok { - return sdkerrors.Wrap(sdkerrors.ErrTxDecode, "invalid transaction type") - } - - params := vmm.ak.GetParams(sdkCtx) - - memoLength := len(memoTx.GetMemo()) - if uint64(memoLength) > params.MaxMemoCharacters { - return sdkerrors.Wrapf(sdkerrors.ErrMemoTooLarge, - "maximum number of characters is %d but received %d characters", - params.MaxMemoCharacters, memoLength, - ) - } - - return nil -} - -// CheckTx implements tx.Handler.CheckTx method. -func (vmm validateMemoTxHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - if err := vmm.checkForValidMemo(ctx, req.Tx); err != nil { - return tx.Response{}, tx.ResponseCheckTx{}, err - } - - return vmm.next.CheckTx(ctx, req, checkReq) -} - -// DeliverTx implements tx.Handler.DeliverTx method. -func (vmm validateMemoTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := vmm.checkForValidMemo(ctx, req.Tx); err != nil { - return tx.Response{}, err - } - - return vmm.next.DeliverTx(ctx, req) -} - -// SimulateTx implements tx.Handler.SimulateTx method. -func (vmm validateMemoTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := vmm.checkForValidMemo(ctx, req.Tx); err != nil { - return tx.Response{}, err - } - - return vmm.next.SimulateTx(ctx, req) -} - -var _ tx.Handler = consumeTxSizeGasTxHandler{} - -type consumeTxSizeGasTxHandler struct { - ak AccountKeeper - next tx.Handler -} - -// ConsumeTxSizeGasMiddleware will take in parameters and consume gas proportional -// to the size of tx before calling next middleware. Note, the gas costs will be -// slightly over estimated due to the fact that any given signing account may need -// to be retrieved from state. -// -// CONTRACT: If simulate=true, then signatures must either be completely filled -// in or empty. -// CONTRACT: To use this middleware, signatures of transaction must be represented -// as legacytx.StdSignature otherwise simulate mode will incorrectly estimate gas cost. -func ConsumeTxSizeGasMiddleware(ak AccountKeeper) tx.Middleware { - return func(txHandler tx.Handler) tx.Handler { - return consumeTxSizeGasTxHandler{ - ak: ak, - next: txHandler, - } - } -} - -func (cgts consumeTxSizeGasTxHandler) simulateSigGasCost(ctx context.Context, tx sdk.Tx) error { - sdkCtx := sdk.UnwrapSDKContext(ctx) - params := cgts.ak.GetParams(sdkCtx) - - sigTx, ok := tx.(authsigning.SigVerifiableTx) - if !ok { - return sdkerrors.Wrap(sdkerrors.ErrTxDecode, "invalid tx type") - } - - // in simulate mode, each element should be a nil signature - sigs, err := sigTx.GetSignaturesV2() - if err != nil { - return err - } - n := len(sigs) - - for i, signer := range sigTx.GetSigners() { - // if signature is already filled in, no need to simulate gas cost - if i < n && !isIncompleteSignature(sigs[i].Data) { - continue - } - - var pubkey cryptotypes.PubKey - - acc := cgts.ak.GetAccount(sdkCtx, signer) - - // use placeholder simSecp256k1Pubkey if sig is nil - if acc == nil || acc.GetPubKey() == nil { - pubkey = simSecp256k1Pubkey - } else { - pubkey = acc.GetPubKey() - } - - // use stdsignature to mock the size of a full signature - simSig := legacytx.StdSignature{ //nolint:staticcheck // this will be removed when proto is ready - Signature: simSecp256k1Sig[:], - PubKey: pubkey, - } - - sigBz := legacy.Cdc.MustMarshal(simSig) - cost := sdk.Gas(len(sigBz) + 6) - - // If the pubkey is a multi-signature pubkey, then we estimate for the maximum - // number of signers. - if _, ok := pubkey.(*multisig.LegacyAminoPubKey); ok { - cost *= params.TxSigLimit - } - - sdkCtx.GasMeter().ConsumeGas(params.TxSizeCostPerByte*cost, "txSize") - } - - return nil -} - -//nolint:unparam -func (cgts consumeTxSizeGasTxHandler) consumeTxSizeGas(ctx context.Context, _ sdk.Tx, txBytes []byte) error { - sdkCtx := sdk.UnwrapSDKContext(ctx) - params := cgts.ak.GetParams(sdkCtx) - sdkCtx.GasMeter().ConsumeGas(params.TxSizeCostPerByte*sdk.Gas(len(txBytes)), "txSize") - - return nil -} - -// CheckTx implements tx.Handler.CheckTx. -func (cgts consumeTxSizeGasTxHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - if err := cgts.consumeTxSizeGas(ctx, req.Tx, req.TxBytes); err != nil { - return tx.Response{}, tx.ResponseCheckTx{}, err - } - - return cgts.next.CheckTx(ctx, req, checkReq) -} - -// DeliverTx implements tx.Handler.DeliverTx. -func (cgts consumeTxSizeGasTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := cgts.consumeTxSizeGas(ctx, req.Tx, req.TxBytes); err != nil { - return tx.Response{}, err - } - - return cgts.next.DeliverTx(ctx, req) -} - -// SimulateTx implements tx.Handler.SimulateTx. -func (cgts consumeTxSizeGasTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := cgts.consumeTxSizeGas(ctx, req.Tx, req.TxBytes); err != nil { - return tx.Response{}, err - } - - if err := cgts.simulateSigGasCost(ctx, req.Tx); err != nil { - return tx.Response{}, err - } - - return cgts.next.SimulateTx(ctx, req) -} - -// isIncompleteSignature tests whether SignatureData is fully filled in for simulation purposes -func isIncompleteSignature(data signing.SignatureData) bool { - if data == nil { - return true - } - - switch data := data.(type) { - case *signing.SingleSignatureData: - return len(data.Signature) == 0 - case *signing.MultiSignatureData: - if len(data.Signatures) == 0 { - return true - } - for _, s := range data.Signatures { - if isIncompleteSignature(s) { - return true - } - } - } - - return false -} diff --git a/x/auth/middleware/basic_test.go b/x/auth/middleware/basic_test.go deleted file mode 100644 index 7bc583ed8d75..000000000000 --- a/x/auth/middleware/basic_test.go +++ /dev/null @@ -1,252 +0,0 @@ -package middleware_test - -import ( - "strings" - - cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" - "github.com/cosmos/cosmos-sdk/crypto/types/multisig" - "github.com/cosmos/cosmos-sdk/testutil/testdata" - sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/types/tx/signing" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" -) - -func (s *MWTestSuite) TestValidateBasic() { - ctx := s.SetupTest(true) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() - - txHandler := middleware.ComposeMiddlewares(noopTxHandler, middleware.ValidateBasicMiddleware) - - // keys and addresses - priv1, _, addr1 := testdata.KeyTestPubAddr() - - // msg and signatures - msg := testdata.NewTestMsg(addr1) - feeAmount := testdata.NewTestFeeAmount() - gasLimit := testdata.NewTestGasLimit() - s.Require().NoError(txBuilder.SetMsgs(msg)) - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(gasLimit) - - privs, accNums, accSeqs := []cryptotypes.PrivKey{}, []uint64{}, []uint64{} - invalidTx, _, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) - - // DeliverTx - _, err = txHandler.DeliverTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: invalidTx}) - s.Require().NotNil(err, "Did not error on invalid tx") - - // SimulateTx - _, err = txHandler.SimulateTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: invalidTx}) - s.Require().NotNil(err, "Did not error on invalid tx") - - privs, accNums, accSeqs = []cryptotypes.PrivKey{priv1}, []uint64{0}, []uint64{0} - validTx, _, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) - - // DeliverTx - _, err = txHandler.DeliverTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: validTx}) - s.Require().Nil(err, "ValidateBasicMiddleware returned error on valid tx. err: %v", err) - - // SimulateTx - _, err = txHandler.SimulateTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: validTx}) - s.Require().Nil(err, "ValidateBasicMiddleware returned error on valid tx. err: %v", err) - - // test middleware skips on recheck - ctx = ctx.WithIsReCheckTx(true) - - // middleware should skip processing invalidTx on recheck and thus return nil-error - // DeliverTx - _, err = txHandler.DeliverTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: invalidTx}) - s.Require().Nil(err, "ValidateBasicMiddleware ran on ReCheck") - - // SimulateTx - _, err = txHandler.SimulateTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: invalidTx}) - s.Require().Nil(err, "ValidateBasicMiddleware ran on ReCheck") -} - -func (s *MWTestSuite) TestValidateMemo() { - ctx := s.SetupTest(true) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() - txHandler := middleware.ComposeMiddlewares(noopTxHandler, middleware.ValidateMemoMiddleware(s.app.AccountKeeper)) - - // keys and addresses - priv1, _, addr1 := testdata.KeyTestPubAddr() - - // msg and signatures - msg := testdata.NewTestMsg(addr1) - feeAmount := testdata.NewTestFeeAmount() - gasLimit := testdata.NewTestGasLimit() - s.Require().NoError(txBuilder.SetMsgs(msg)) - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(gasLimit) - - privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{0}, []uint64{0} - txBuilder.SetMemo(strings.Repeat("01234567890", 500)) - invalidTx, _, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) - - // require that long memos get rejected - // DeliverTx - _, err = txHandler.DeliverTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: invalidTx}) - s.Require().NotNil(err, "Did not error on tx with high memo") - - // SimulateTx - _, err = txHandler.SimulateTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: invalidTx}) - s.Require().NotNil(err, "Did not error on tx with high memo") - - txBuilder.SetMemo(strings.Repeat("01234567890", 10)) - validTx, _, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) - - // require small memos pass ValidateMemo middleware - // DeliverTx - _, err = txHandler.DeliverTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: validTx}) - s.Require().Nil(err, "ValidateBasicMiddleware returned error on valid tx. err: %v", err) - - // SimulateTx - _, err = txHandler.SimulateTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: validTx}) - s.Require().Nil(err, "ValidateBasicMiddleware returned error on valid tx. err: %v", err) -} - -func (s *MWTestSuite) TestConsumeGasForTxSize() { - ctx := s.SetupTest(true) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() - - txHandler := middleware.ComposeMiddlewares(noopTxHandler, middleware.ConsumeTxSizeGasMiddleware(s.app.AccountKeeper)) - - // keys and addresses - priv1, _, addr1 := testdata.KeyTestPubAddr() - - // msg and signatures - msg := testdata.NewTestMsg(addr1) - feeAmount := testdata.NewTestFeeAmount() - gasLimit := testdata.NewTestGasLimit() - - testCases := []struct { - name string - sigV2 signing.SignatureV2 - }{ - {"SingleSignatureData", signing.SignatureV2{PubKey: priv1.PubKey()}}, - {"MultiSignatureData", signing.SignatureV2{PubKey: priv1.PubKey(), Data: multisig.NewMultisig(2)}}, - } - - for _, tc := range testCases { - s.Run(tc.name, func() { - txBuilder = s.clientCtx.TxConfig.NewTxBuilder() - s.Require().NoError(txBuilder.SetMsgs(msg)) - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(gasLimit) - txBuilder.SetMemo(strings.Repeat("01234567890", 10)) - - privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{0}, []uint64{0} - testTx, _, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) - - txBytes, err := s.clientCtx.TxConfig.TxJSONEncoder()(testTx) - s.Require().Nil(err, "Cannot marshal tx: %v", err) - - params := s.app.AccountKeeper.GetParams(ctx) - expectedGas := sdk.Gas(len(txBytes)) * params.TxSizeCostPerByte - - // Set ctx with TxBytes manually - ctx = ctx.WithTxBytes(txBytes) - - // track how much gas is necessary to retrieve parameters - beforeGas := ctx.GasMeter().GasConsumed() - s.app.AccountKeeper.GetParams(ctx) - afterGas := ctx.GasMeter().GasConsumed() - expectedGas += afterGas - beforeGas - - beforeGas = ctx.GasMeter().GasConsumed() - _, err = txHandler.DeliverTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx, TxBytes: txBytes}) - - s.Require().Nil(err, "ConsumeTxSizeGasMiddleware returned error: %v", err) - - // require that middleware consumes expected amount of gas - consumedGas := ctx.GasMeter().GasConsumed() - beforeGas - s.Require().Equal(expectedGas, consumedGas, "Middleware did not consume the correct amount of gas") - - // simulation must not underestimate gas of this middleware even with nil signatures - txBuilder, err := s.clientCtx.TxConfig.WrapTxBuilder(testTx) - s.Require().NoError(err) - s.Require().NoError(txBuilder.SetSignatures(tc.sigV2)) - testTx = txBuilder.GetTx() - - simTxBytes, err := s.clientCtx.TxConfig.TxJSONEncoder()(testTx) - s.Require().Nil(err, "Cannot marshal tx: %v", err) - // require that simulated tx is smaller than tx with signatures - s.Require().True(len(simTxBytes) < len(txBytes), "simulated tx still has signatures") - - // Set s.ctx with smaller simulated TxBytes manually - ctx = ctx.WithTxBytes(simTxBytes) - - beforeSimGas := ctx.GasMeter().GasConsumed() - - // run txhandler in simulate mode - _, err = txHandler.SimulateTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx, TxBytes: simTxBytes}) - consumedSimGas := ctx.GasMeter().GasConsumed() - beforeSimGas - - // require that txhandler passes and does not underestimate middleware cost - s.Require().Nil(err, "ConsumeTxSizeGasMiddleware returned error: %v", err) - s.Require().True(consumedSimGas >= expectedGas, "Simulate mode underestimates gas on Middleware. Simulated cost: %d, expected cost: %d", consumedSimGas, expectedGas) - }) - } -} - -func (s *MWTestSuite) TestTxHeightTimeoutMiddleware() { - ctx := s.SetupTest(true) - - txHandler := middleware.ComposeMiddlewares(noopTxHandler, middleware.TxTimeoutHeightMiddleware) - - // keys and addresses - priv1, _, addr1 := testdata.KeyTestPubAddr() - - // msg and signatures - msg := testdata.NewTestMsg(addr1) - feeAmount := testdata.NewTestFeeAmount() - gasLimit := testdata.NewTestGasLimit() - - testCases := []struct { - name string - timeout uint64 - height int64 - expectErr bool - }{ - {"default value", 0, 10, false}, - {"no timeout (greater height)", 15, 10, false}, - {"no timeout (same height)", 10, 10, false}, - {"timeout (smaller height)", 9, 10, true}, - } - - for _, tc := range testCases { - tc := tc - - s.Run(tc.name, func() { - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() - - s.Require().NoError(txBuilder.SetMsgs(msg)) - - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(gasLimit) - txBuilder.SetMemo(strings.Repeat("01234567890", 10)) - txBuilder.SetTimeoutHeight(tc.timeout) - - privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{0}, []uint64{0} - testTx, _, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) - - ctx := ctx.WithBlockHeight(tc.height) - - // DeliverTx - _, err = txHandler.DeliverTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}) - s.Require().Equal(tc.expectErr, err != nil, err) - - // SimulateTx - _, err = txHandler.SimulateTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}) - s.Require().Equal(tc.expectErr, err != nil, err) - - }) - } -} diff --git a/x/auth/middleware/block_gas.go b/x/auth/middleware/block_gas.go deleted file mode 100644 index bfd67f92ecbe..000000000000 --- a/x/auth/middleware/block_gas.go +++ /dev/null @@ -1,53 +0,0 @@ -package middleware - -import ( - "context" - - sdk "github.com/cosmos/cosmos-sdk/types" - sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/tx" -) - -type consumeBlockGasHandler struct { - next tx.Handler -} - -// ConsumeBlockGasMiddleware check and consume block gas meter. -func ConsumeBlockGasMiddleware(txh tx.Handler) tx.Handler { - return consumeBlockGasHandler{next: txh} -} - -var _ tx.Handler = consumeBlockGasHandler{} - -// CheckTx implements tx.Handler.CheckTx method. -func (cbgh consumeBlockGasHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (res tx.Response, resCheckTx tx.ResponseCheckTx, err error) { - return cbgh.next.CheckTx(ctx, req, checkReq) -} - -// DeliverTx implements tx.Handler.DeliverTx method. -// Consume block gas meter, panic when block gas meter exceeded, -// the panic should be caught by `RecoveryTxMiddleware`. -func (cbgh consumeBlockGasHandler) DeliverTx(ctx context.Context, req tx.Request) (res tx.Response, err error) { - sdkCtx := sdk.UnwrapSDKContext(ctx) - // only run the tx if there is block gas remaining - if sdkCtx.BlockGasMeter().IsOutOfGas() { - err = sdkerrors.Wrap(sdkerrors.ErrOutOfGas, "no block gas left to run tx") - return - } - - // If BlockGasMeter() panics it will be caught by the `RecoveryTxMiddleware` and will - // return an error - in any case BlockGasMeter will consume gas past the limit. - defer func() { - sdkCtx.BlockGasMeter().ConsumeGas( - sdkCtx.GasMeter().GasConsumedToLimit(), "block gas meter", - ) - - }() - - return cbgh.next.DeliverTx(ctx, req) -} - -// SimulateTx implements tx.Handler.SimulateTx method. -func (cbgh consumeBlockGasHandler) SimulateTx(ctx context.Context, req tx.Request) (res tx.Response, err error) { - return cbgh.next.SimulateTx(ctx, req) -} diff --git a/x/auth/middleware/branch_store.go b/x/auth/middleware/branch_store.go deleted file mode 100644 index 236d288c122b..000000000000 --- a/x/auth/middleware/branch_store.go +++ /dev/null @@ -1,70 +0,0 @@ -package middleware - -import ( - "context" - - sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/types/tx" - tmtypes "github.com/tendermint/tendermint/types" -) - -type branchStoreHandler struct { - next tx.Handler -} - -// WithBranchedStore creates a new MultiStore branch and commits the store if the downstream -// returned no error. It cancels writes from the failed transactions. -func WithBranchedStore(txh tx.Handler) tx.Handler { - return branchStoreHandler{next: txh} -} - -// CheckTx implements tx.Handler.CheckTx method. -// Do nothing during CheckTx. -func (sh branchStoreHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - return sh.next.CheckTx(ctx, req, checkReq) -} - -// DeliverTx implements tx.Handler.DeliverTx method. -func (sh branchStoreHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - return branchAndRun(ctx, req, sh.next.DeliverTx) -} - -// SimulateTx implements tx.Handler.SimulateTx method. -func (sh branchStoreHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - return branchAndRun(ctx, req, sh.next.SimulateTx) -} - -type nextFn func(ctx context.Context, req tx.Request) (tx.Response, error) - -// branchAndRun creates a new Context based on the existing Context with a MultiStore branch -// in case message processing fails. -func branchAndRun(ctx context.Context, req tx.Request, fn nextFn) (tx.Response, error) { - sdkCtx := sdk.UnwrapSDKContext(ctx) - runMsgCtx, branchedStore := branchStore(sdkCtx, tmtypes.Tx(req.TxBytes)) - - rsp, err := fn(sdk.WrapSDKContext(runMsgCtx), req) - if err == nil { - // commit storage iff no error - branchedStore.Write() - } - - return rsp, err -} - -// branchStore returns a new context based off of the provided context with -// a branched multi-store. -func branchStore(sdkCtx sdk.Context, tx tmtypes.Tx) (sdk.Context, sdk.CacheMultiStore) { - ms := sdkCtx.MultiStore() - msCache := ms.CacheMultiStore() - if msCache.TracingEnabled() { - msCache = msCache.SetTracingContext( - sdk.TraceContext( - map[string]interface{}{ - "txHash": tx.Hash(), - }, - ), - ).(sdk.CacheMultiStore) - } - - return sdkCtx.WithMultiStore(msCache), msCache -} diff --git a/x/auth/middleware/branch_store_test.go b/x/auth/middleware/branch_store_test.go deleted file mode 100644 index 80e116fb2a82..000000000000 --- a/x/auth/middleware/branch_store_test.go +++ /dev/null @@ -1,129 +0,0 @@ -package middleware_test - -import ( - "context" - "fmt" - "math" - - cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" - "github.com/cosmos/cosmos-sdk/simapp" - "github.com/cosmos/cosmos-sdk/testutil/testdata" - sdk "github.com/cosmos/cosmos-sdk/types" - sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" - minttypes "github.com/cosmos/cosmos-sdk/x/mint/types" -) - -var blockMaxGas = uint64(simapp.DefaultConsensusParams.Block.MaxGas) - -func (s *MWTestSuite) TestBranchStore() { - testcases := []struct { - name string - gasToConsume uint64 // gas to consume in the msg execution - panicTx bool // panic explicitly in tx execution - expErr bool - }{ - {"less than block gas meter", 10, false, false}, - {"more than block gas meter", blockMaxGas, false, true}, - {"more than block gas meter", uint64(float64(blockMaxGas) * 1.2), false, true}, - {"consume MaxUint64", math.MaxUint64, false, true}, - {"consume block gas when paniced", 10, true, true}, - } - - for _, tc := range testcases { - s.Run(tc.name, func() { - ctx := s.SetupTest(true).WithBlockGasMeter(sdk.NewGasMeter(blockMaxGas)) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() - - // tx fee - feeCoin := sdk.NewCoin("atom", sdk.NewInt(150)) - feeAmount := sdk.NewCoins(feeCoin) - - // test account and fund - priv1, _, addr1 := testdata.KeyTestPubAddr() - err := s.app.BankKeeper.MintCoins(ctx, minttypes.ModuleName, feeAmount) - s.Require().NoError(err) - err = s.app.BankKeeper.SendCoinsFromModuleToAccount(ctx, minttypes.ModuleName, addr1, feeAmount) - s.Require().NoError(err) - s.Require().Equal(feeCoin.Amount, s.app.BankKeeper.GetBalance(ctx, addr1, feeCoin.Denom).Amount) - seq, _ := s.app.AccountKeeper.GetSequence(ctx, addr1) - s.Require().Equal(uint64(0), seq) - - // testMsgTxHandler is a test txHandler that handles one single TestMsg, - // consumes the given `tc.gasToConsume`, and sets the bank store "ok" key to "ok". - var testMsgTxHandler = customTxHandler{func(ctx context.Context, req tx.Request) (tx.Response, error) { - msg, ok := req.Tx.GetMsgs()[0].(*testdata.TestMsg) - if !ok { - return tx.Response{}, fmt.Errorf("Wrong Msg type, expected %T, got %T", (*testdata.TestMsg)(nil), msg) - } - - sdkCtx := sdk.UnwrapSDKContext(ctx) - sdkCtx.KVStore(s.app.GetKey("bank")).Set([]byte("ok"), []byte("ok")) - sdkCtx.GasMeter().ConsumeGas(tc.gasToConsume, "TestMsg") - if tc.panicTx { - panic("panic in tx execution") - } - return tx.Response{}, nil - }} - - txHandler := middleware.ComposeMiddlewares( - testMsgTxHandler, - middleware.NewTxDecoderMiddleware(s.clientCtx.TxConfig.TxDecoder()), - middleware.GasTxMiddleware, - middleware.RecoveryTxMiddleware, - middleware.DeductFeeMiddleware(s.app.AccountKeeper, s.app.BankKeeper, s.app.FeeGrantKeeper, nil), - middleware.IncrementSequenceMiddleware(s.app.AccountKeeper), - middleware.WithBranchedStore, - middleware.ConsumeBlockGasMiddleware, - ) - - // msg and signatures - msg := testdata.NewTestMsg(addr1) - var gasLimit uint64 = math.MaxUint64 // no limit on sdk.GasMeter - s.Require().NoError(txBuilder.SetMsgs(msg)) - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(gasLimit) - - privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{0}, []uint64{0} - testTx, _, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) - - _, err = txHandler.DeliverTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}) - - bankStore := ctx.KVStore(s.app.GetKey("bank")) - okValue := bankStore.Get([]byte("ok")) - - if tc.expErr { - s.Require().Error(err) - if tc.panicTx { - s.Require().True(sdkerrors.IsOf(err, sdkerrors.ErrPanic)) - } else { - s.Require().True(sdkerrors.IsOf(err, sdkerrors.ErrOutOfGas)) - } - s.Require().Empty(okValue) - } else { - s.Require().NoError(err) - s.Require().Equal([]byte("ok"), okValue) - } - // block gas is always consumed - baseGas := uint64(24564) // baseGas is the gas consumed by middlewares - expGasConsumed := addUint64Saturating(tc.gasToConsume, baseGas) - s.Require().Equal(expGasConsumed, ctx.BlockGasMeter().GasConsumed()) - // tx fee is always deducted - s.Require().Equal(int64(0), s.app.BankKeeper.GetBalance(ctx, addr1, feeCoin.Denom).Amount.Int64()) - // sender's sequence is always increased - seq, err = s.app.AccountKeeper.GetSequence(ctx, addr1) - s.Require().NoError(err) - s.Require().Equal(uint64(1), seq) - }) - } -} - -func addUint64Saturating(a, b uint64) uint64 { - if math.MaxUint64-a < b { - return math.MaxUint64 - } - - return a + b -} diff --git a/x/auth/middleware/ext.go b/x/auth/middleware/ext.go deleted file mode 100644 index 5159e3c5f1f8..000000000000 --- a/x/auth/middleware/ext.go +++ /dev/null @@ -1,86 +0,0 @@ -package middleware - -import ( - "context" - - codectypes "github.com/cosmos/cosmos-sdk/codec/types" - sdk "github.com/cosmos/cosmos-sdk/types" - sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/tx" -) - -type HasExtensionOptionsTx interface { - GetExtensionOptions() []*codectypes.Any - GetNonCriticalExtensionOptions() []*codectypes.Any -} - -// ExtensionOptionChecker is a function that returns true if the extension option is accepted. -type ExtensionOptionChecker func(*codectypes.Any) bool - -// rejectExtensionOption is the default extension check that reject all tx -// extensions. -func rejectExtensionOption(*codectypes.Any) bool { - return false -} - -type rejectExtensionOptionsTxHandler struct { - next tx.Handler - checker ExtensionOptionChecker -} - -// NewExtensionOptionsMiddleware creates a new middleware that rejects all extension -// options which can optionally be included in protobuf transactions that don't pass the checker. -// Users that need extension options should pass a custom checker that returns true for the -// needed extension options. -func NewExtensionOptionsMiddleware(checker ExtensionOptionChecker) tx.Middleware { - if checker == nil { - checker = rejectExtensionOption - } - return func(txh tx.Handler) tx.Handler { - return rejectExtensionOptionsTxHandler{ - next: txh, - checker: checker, - } - } -} - -var _ tx.Handler = rejectExtensionOptionsTxHandler{} - -func checkExtOpts(tx sdk.Tx, checker ExtensionOptionChecker) error { - if hasExtOptsTx, ok := tx.(HasExtensionOptionsTx); ok { - for _, opt := range hasExtOptsTx.GetExtensionOptions() { - if !checker(opt) { - return sdkerrors.ErrUnknownExtensionOptions - } - } - } - - return nil -} - -// CheckTx implements tx.Handler.CheckTx. -func (txh rejectExtensionOptionsTxHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - if err := checkExtOpts(req.Tx, txh.checker); err != nil { - return tx.Response{}, tx.ResponseCheckTx{}, err - } - - return txh.next.CheckTx(ctx, req, checkReq) -} - -// DeliverTx implements tx.Handler.DeliverTx. -func (txh rejectExtensionOptionsTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := checkExtOpts(req.Tx, txh.checker); err != nil { - return tx.Response{}, err - } - - return txh.next.DeliverTx(ctx, req) -} - -// SimulateTx implements tx.Handler.SimulateTx method. -func (txh rejectExtensionOptionsTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - if err := checkExtOpts(req.Tx, txh.checker); err != nil { - return tx.Response{}, err - } - - return txh.next.SimulateTx(ctx, req) -} diff --git a/x/auth/middleware/ext_test.go b/x/auth/middleware/ext_test.go deleted file mode 100644 index 27c294794cfb..000000000000 --- a/x/auth/middleware/ext_test.go +++ /dev/null @@ -1,54 +0,0 @@ -package middleware_test - -import ( - "github.com/cosmos/cosmos-sdk/codec/types" - codectypes "github.com/cosmos/cosmos-sdk/codec/types" - "github.com/cosmos/cosmos-sdk/testutil/testdata" - sdk "github.com/cosmos/cosmos-sdk/types" - typestx "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" - "github.com/cosmos/cosmos-sdk/x/auth/tx" -) - -func (s *MWTestSuite) TestExtensionOptionsMiddleware() { - testCases := []struct { - msg string - allow bool - }{ - {"allow extension", true}, - {"reject extension", false}, - } - for _, tc := range testCases { - s.Run(tc.msg, func() { - ctx := s.SetupTest(true) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() - - txHandler := middleware.ComposeMiddlewares(noopTxHandler, middleware.NewExtensionOptionsMiddleware(func(_ *codectypes.Any) bool { - return tc.allow - })) - - // no extension options should not trigger an error - theTx := txBuilder.GetTx() - _, _, err := txHandler.CheckTx(sdk.WrapSDKContext(ctx), typestx.Request{Tx: theTx}, typestx.RequestCheckTx{}) - s.Require().NoError(err) - - extOptsTxBldr, ok := txBuilder.(tx.ExtensionOptionsTxBuilder) - if !ok { - // if we can't set extension options, this middleware doesn't apply and we're done - return - } - - // set an extension option and check - any, err := types.NewAnyWithValue(testdata.NewTestMsg()) - s.Require().NoError(err) - extOptsTxBldr.SetExtensionOptions(any) - theTx = txBuilder.GetTx() - _, _, err = txHandler.CheckTx(sdk.WrapSDKContext(ctx), typestx.Request{Tx: theTx}, typestx.RequestCheckTx{}) - if tc.allow { - s.Require().NoError(err) - } else { - s.Require().EqualError(err, "unknown extension options") - } - }) - } -} diff --git a/x/auth/middleware/fee.go b/x/auth/middleware/fee.go deleted file mode 100644 index 2ae83c37269e..000000000000 --- a/x/auth/middleware/fee.go +++ /dev/null @@ -1,153 +0,0 @@ -package middleware - -import ( - "context" - "fmt" - - sdk "github.com/cosmos/cosmos-sdk/types" - sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/x/auth/types" -) - -// TxFeeChecker check if the provided fee is enough and returns the effective fee and tx priority, -// the effective fee should be deducted later, and the priority should be returned in abci response. -type TxFeeChecker func(ctx sdk.Context, tx sdk.Tx) (sdk.Coins, int64, error) - -var _ tx.Handler = deductFeeTxHandler{} - -type deductFeeTxHandler struct { - accountKeeper AccountKeeper - bankKeeper types.BankKeeper - feegrantKeeper FeegrantKeeper - txFeeChecker TxFeeChecker - next tx.Handler -} - -// DeductFeeMiddleware deducts fees from the first signer of the tx -// If the first signer does not have the funds to pay for the fees, return with InsufficientFunds error -// Call next middleware if fees successfully deducted -// CONTRACT: Tx must implement FeeTx interface to use deductFeeTxHandler -func DeductFeeMiddleware(ak AccountKeeper, bk types.BankKeeper, fk FeegrantKeeper, tfc TxFeeChecker) tx.Middleware { - if tfc == nil { - tfc = checkTxFeeWithValidatorMinGasPrices - } - return func(txh tx.Handler) tx.Handler { - return deductFeeTxHandler{ - accountKeeper: ak, - bankKeeper: bk, - feegrantKeeper: fk, - txFeeChecker: tfc, - next: txh, - } - } -} - -func (dfd deductFeeTxHandler) checkDeductFee(ctx sdk.Context, sdkTx sdk.Tx, fee sdk.Coins) error { - feeTx, ok := sdkTx.(sdk.FeeTx) - if !ok { - return sdkerrors.Wrap(sdkerrors.ErrTxDecode, "Tx must be a FeeTx") - } - - if addr := dfd.accountKeeper.GetModuleAddress(types.FeeCollectorName); addr == nil { - return fmt.Errorf("Fee collector module account (%s) has not been set", types.FeeCollectorName) - } - - feePayer := feeTx.FeePayer() - feeGranter := feeTx.FeeGranter() - deductFeesFrom := feePayer - - // if feegranter set deduct fee from feegranter account. - // this works with only when feegrant enabled. - if feeGranter != nil { - if dfd.feegrantKeeper == nil { - return sdkerrors.ErrInvalidRequest.Wrap("fee grants are not enabled") - } else if !feeGranter.Equals(feePayer) { - err := dfd.feegrantKeeper.UseGrantedFees(ctx, feeGranter, feePayer, fee, sdkTx.GetMsgs()) - if err != nil { - return sdkerrors.Wrapf(err, "%s does not not allow to pay fees for %s", feeGranter, feePayer) - } - } - - deductFeesFrom = feeGranter - } - - deductFeesFromAcc := dfd.accountKeeper.GetAccount(ctx, deductFeesFrom) - if deductFeesFromAcc == nil { - return sdkerrors.ErrUnknownAddress.Wrapf("fee payer address: %s does not exist", deductFeesFrom) - } - - // deduct the fees - if !fee.IsZero() { - err := DeductFees(dfd.bankKeeper, ctx, deductFeesFromAcc, fee) - if err != nil { - return err - } - } - - events := sdk.Events{sdk.NewEvent(sdk.EventTypeTx, - sdk.NewAttribute(sdk.AttributeKeyFee, fee.String()), - )} - ctx.EventManager().EmitEvents(events) - - return nil -} - -// CheckTx implements tx.Handler.CheckTx. -func (dfd deductFeeTxHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - sdkCtx := sdk.UnwrapSDKContext(ctx) - fee, priority, err := dfd.txFeeChecker(sdkCtx, req.Tx) - if err != nil { - return tx.Response{}, tx.ResponseCheckTx{}, err - } - if err := dfd.checkDeductFee(sdkCtx, req.Tx, fee); err != nil { - return tx.Response{}, tx.ResponseCheckTx{}, err - } - - res, checkRes, err := dfd.next.CheckTx(ctx, req, checkReq) - checkRes.Priority = priority - - return res, checkRes, err -} - -// DeliverTx implements tx.Handler.DeliverTx. -func (dfd deductFeeTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - sdkCtx := sdk.UnwrapSDKContext(ctx) - fee, _, err := dfd.txFeeChecker(sdkCtx, req.Tx) - if err != nil { - return tx.Response{}, err - } - if err := dfd.checkDeductFee(sdkCtx, req.Tx, fee); err != nil { - return tx.Response{}, err - } - - return dfd.next.DeliverTx(ctx, req) -} - -func (dfd deductFeeTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - sdkCtx := sdk.UnwrapSDKContext(ctx) - fee, _, err := dfd.txFeeChecker(sdkCtx, req.Tx) - if err != nil { - return tx.Response{}, err - } - if err := dfd.checkDeductFee(sdkCtx, req.Tx, fee); err != nil { - return tx.Response{}, err - } - - return dfd.next.SimulateTx(ctx, req) -} - -// Deprecated: DeductFees deducts fees from the given account. -// This function will be private in the next release. -func DeductFees(bankKeeper types.BankKeeper, ctx sdk.Context, acc types.AccountI, fees sdk.Coins) error { - if !fees.IsValid() { - return sdkerrors.ErrInsufficientFee.Wrapf("invalid fee amount: %s", fees) - } - - err := bankKeeper.SendCoinsFromAccountToModule(ctx, acc.GetAddress(), types.FeeCollectorName, fees) - if err != nil { - return sdkerrors.ErrInsufficientFunds.Wrap(err.Error()) - } - - return nil -} diff --git a/x/auth/middleware/fee_test.go b/x/auth/middleware/fee_test.go deleted file mode 100644 index 9899e2aa63c3..000000000000 --- a/x/auth/middleware/fee_test.go +++ /dev/null @@ -1,126 +0,0 @@ -package middleware_test - -import ( - cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" - "github.com/cosmos/cosmos-sdk/testutil/testdata" - sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" - "github.com/cosmos/cosmos-sdk/x/bank/testutil" -) - -func (s *MWTestSuite) TestEnsureMempoolFees() { - ctx := s.SetupTest(true) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() - - txHandler := middleware.ComposeMiddlewares(noopTxHandler, middleware.DeductFeeMiddleware( - s.app.AccountKeeper, - s.app.BankKeeper, - s.app.FeeGrantKeeper, - nil, - )) - - // keys and addresses - priv1, _, addr1 := testdata.KeyTestPubAddr() - - // msg and signatures - msg := testdata.NewTestMsg(addr1) - atomCoin := sdk.NewCoin("atom", sdk.NewInt(150)) - apeCoin := sdk.NewInt64Coin("ape", 1500000) - feeAmount := sdk.NewCoins(apeCoin, atomCoin) - gasLimit := testdata.NewTestGasLimit() - s.Require().NoError(txBuilder.SetMsgs(msg)) - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(gasLimit) - - privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{0}, []uint64{0} - testTx, _, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) - - // Set high gas price so standard test fee fails - atomPrice := sdk.NewDecCoinFromDec("atom", sdk.NewDec(200).Quo(sdk.NewDec(100000))) - highGasPrice := []sdk.DecCoin{atomPrice} - ctx = ctx.WithMinGasPrices(highGasPrice) - - // txHandler errors with insufficient fees - _, _, err = txHandler.CheckTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}, tx.RequestCheckTx{}) - s.Require().NotNil(err, "Middleware should have errored on too low fee for local gasPrice") - - // txHandler should fail since we also check minGasPrice in DeliverTx - _, err = txHandler.DeliverTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}) - s.Require().Error(err, "MempoolFeeMiddleware don't error in DeliverTx") - - atomPrice = sdk.NewDecCoinFromDec("atom", sdk.NewDec(0).Quo(sdk.NewDec(100000))) - lowGasPrice := []sdk.DecCoin{atomPrice} - ctx = ctx.WithMinGasPrices(lowGasPrice) - - // Set account with sufficient funds - acc := s.app.AccountKeeper.NewAccountWithAddress(ctx, addr1) - s.app.AccountKeeper.SetAccount(ctx, acc) - err = testutil.FundAccount(s.app.BankKeeper, ctx, addr1, feeAmount) - s.Require().NoError(err) - - _, checkTxRes, err := txHandler.CheckTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}, tx.RequestCheckTx{}) - s.Require().Nil(err, "Middleware should not have errored on fee higher than local gasPrice") - s.Require().Equal(atomCoin.Amount.Int64(), checkTxRes.Priority, "priority should be atom amount") -} - -func (s *MWTestSuite) TestDeductFees() { - ctx := s.SetupTest(false) // setup - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() - txHandler := middleware.ComposeMiddlewares( - noopTxHandler, - middleware.DeductFeeMiddleware( - s.app.AccountKeeper, - s.app.BankKeeper, - s.app.FeeGrantKeeper, - nil, - ), - ) - - // keys and addresses - priv1, _, addr1 := testdata.KeyTestPubAddr() - - // msg and signatures - msg := testdata.NewTestMsg(addr1) - feeAmount := testdata.NewTestFeeAmount() - gasLimit := testdata.NewTestGasLimit() - s.Require().NoError(txBuilder.SetMsgs(msg)) - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(gasLimit) - - privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{0}, []uint64{0} - testTx, _, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) - - // Set account with insufficient funds - acc := s.app.AccountKeeper.NewAccountWithAddress(ctx, addr1) - s.app.AccountKeeper.SetAccount(ctx, acc) - coins := sdk.NewCoins(sdk.NewCoin("atom", sdk.NewInt(10))) - err = testutil.FundAccount(s.app.BankKeeper, ctx, addr1, coins) - s.Require().NoError(err) - - // DeliverTx - _, err = txHandler.DeliverTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}) - s.Require().NotNil(err, "Tx errored when fee payer had insufficient funds") - - // SimulateTx - _, err = txHandler.SimulateTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}) - s.Require().NotNil(err, "Tx errored when fee payer had insufficient funds") - - // Set account with sufficient funds - s.app.AccountKeeper.SetAccount(ctx, acc) - err = testutil.FundAccount(s.app.BankKeeper, ctx, addr1, sdk.NewCoins(sdk.NewCoin("atom", sdk.NewInt(200)))) - s.Require().NoError(err) - - // DeliverTx - _, err = txHandler.DeliverTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}) - s.Require().Nil(err, "Tx did not error after account has been set with sufficient funds") - - err = testutil.FundAccount(s.app.BankKeeper, ctx, addr1, sdk.NewCoins(sdk.NewCoin("atom", sdk.NewInt(200)))) - s.Require().NoError(err) - - // SimulateTx - _, err = txHandler.SimulateTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx}) - s.Require().Nil(err, "Tx did not error after account has been set with sufficient funds") -} diff --git a/x/auth/middleware/gas.go b/x/auth/middleware/gas.go deleted file mode 100644 index 2957566b1682..000000000000 --- a/x/auth/middleware/gas.go +++ /dev/null @@ -1,96 +0,0 @@ -package middleware - -import ( - "context" - - sdk "github.com/cosmos/cosmos-sdk/types" - sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/tx" -) - -// GasTx defines a Tx with a GetGas() method which is needed to use gasTxHandler. -type GasTx interface { - sdk.Tx - GetGas() uint64 -} - -type gasTxHandler struct { - next tx.Handler -} - -// GasTxMiddleware defines a simple middleware that sets a new GasMeter on -// the sdk.Context, and sets the GasInfo on the result. It reads the tx.GetGas() -// by default, or sets to infinity in simulate mode. -func GasTxMiddleware(txh tx.Handler) tx.Handler { - return gasTxHandler{next: txh} -} - -var _ tx.Handler = gasTxHandler{} - -// CheckTx implements tx.Handler.CheckTx. -func (txh gasTxHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - sdkCtx, err := gasContext(sdk.UnwrapSDKContext(ctx), req.Tx, false) - if err != nil { - return tx.Response{}, tx.ResponseCheckTx{}, err - } - - res, resCheckTx, err := txh.next.CheckTx(sdk.WrapSDKContext(sdkCtx), req, checkReq) - - return populateGas(res, sdkCtx), resCheckTx, err -} - -// DeliverTx implements tx.Handler.DeliverTx. -func (txh gasTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - sdkCtx, err := gasContext(sdk.UnwrapSDKContext(ctx), req.Tx, false) - if err != nil { - return tx.Response{}, err - } - - res, err := txh.next.DeliverTx(sdk.WrapSDKContext(sdkCtx), req) - - return populateGas(res, sdkCtx), err -} - -// SimulateTx implements tx.Handler.SimulateTx method. -func (txh gasTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - sdkCtx, err := gasContext(sdk.UnwrapSDKContext(ctx), req.Tx, true) - if err != nil { - return tx.Response{}, err - } - - res, err := txh.next.SimulateTx(sdk.WrapSDKContext(sdkCtx), req) - - return populateGas(res, sdkCtx), err -} - -// populateGas returns a new tx.Response with gas fields populated. -func populateGas(res tx.Response, sdkCtx sdk.Context) tx.Response { - res.GasWanted = sdkCtx.GasMeter().Limit() - res.GasUsed = sdkCtx.GasMeter().GasConsumed() - - return res -} - -// gasContext returns a new context with a gas meter set from a given context. -func gasContext(ctx sdk.Context, tx sdk.Tx, isSimulate bool) (sdk.Context, error) { - // all transactions must implement GasTx - gasTx, ok := tx.(GasTx) - if !ok { - // Set a gas meter with limit 0 as to prevent an infinite gas meter attack execution. - newCtx := setGasMeter(ctx, 0, isSimulate) - return newCtx, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "Tx must be GasTx") - } - - return setGasMeter(ctx, gasTx.GetGas(), isSimulate), nil -} - -// setGasMeter returns a new context with a gas meter set from a given context. -func setGasMeter(ctx sdk.Context, gasLimit uint64, simulate bool) sdk.Context { - // In various cases such as simulation and during the genesis block, we do not - // meter any gas utilization. - if simulate || ctx.BlockHeight() == 0 { - return ctx.WithGasMeter(sdk.NewInfiniteGasMeter()) - } - - return ctx.WithGasMeter(sdk.NewGasMeter(gasLimit)) -} diff --git a/x/auth/middleware/gas_test.go b/x/auth/middleware/gas_test.go deleted file mode 100644 index 480077762149..000000000000 --- a/x/auth/middleware/gas_test.go +++ /dev/null @@ -1,127 +0,0 @@ -package middleware_test - -import ( - "context" - "errors" - - cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" - "github.com/cosmos/cosmos-sdk/testutil/testdata" - sdk "github.com/cosmos/cosmos-sdk/types" - sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" - "github.com/cosmos/cosmos-sdk/x/auth/signing" -) - -// txTest is a dummy tx that doesn't implement GasTx. It should set the GasMeter -// to 0 in this case. -type txTest struct{} - -var _ sdk.Tx = txTest{} - -func (t txTest) GetMsgs() []sdk.Msg { return []sdk.Msg{} } -func (t txTest) ValidateBasic() error { return nil } - -func (s *MWTestSuite) setupGasTx() (signing.Tx, []byte, sdk.Context, uint64) { - ctx := s.SetupTest(true) - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() - - // keys and addresses - priv1, _, addr1 := testdata.KeyTestPubAddr() - - // msg and signatures - msg := testdata.NewTestMsg(addr1) - feeAmount := testdata.NewTestFeeAmount() - gasLimit := testdata.NewTestGasLimit() - s.Require().NoError(txBuilder.SetMsgs(msg)) - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(gasLimit) - - // test tx - privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{0}, []uint64{0} - tx, txBytes, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) - - // Set height to non-zero value for GasMeter to be set - ctx = ctx.WithBlockHeight(1) - - return tx, txBytes, ctx, gasLimit -} - -func (s *MWTestSuite) TestSetup() { - testTx, _, ctx, gasLimit := s.setupGasTx() - txHandler := middleware.ComposeMiddlewares(noopTxHandler, middleware.GasTxMiddleware) - - testcases := []struct { - name string - tx sdk.Tx - expGasLimit uint64 - expErr bool - errorStr string - }{ - {"not a gas tx", txTest{}, 0, true, "Tx must be GasTx: tx parse error"}, - {"tx with its own gas limit", testTx, gasLimit, false, ""}, - } - for _, tc := range testcases { - s.Run(tc.name, func() { - res, _, err := txHandler.CheckTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: tc.tx}, tx.RequestCheckTx{}) - _, simErr := txHandler.SimulateTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: tc.tx}) - if tc.expErr { - s.Require().EqualError(err, tc.errorStr) - s.Require().EqualError(simErr, tc.errorStr) - } else { - s.Require().Nil(err, "SetUpContextMiddleware returned error") - s.Require().Nil(simErr, "SetUpContextMiddleware returned error") - s.Require().Equal(tc.expGasLimit, uint64(res.GasWanted)) - } - }) - } -} - -func (s *MWTestSuite) TestRecoverPanic() { - testTx, txBytes, ctx, gasLimit := s.setupGasTx() - txHandler := middleware.ComposeMiddlewares(outOfGasTxHandler, middleware.GasTxMiddleware, middleware.RecoveryTxMiddleware) - res, _, err := txHandler.CheckTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx, TxBytes: txBytes}, tx.RequestCheckTx{}) - s.Require().Error(err, "Did not return error on OutOfGas panic") - s.Require().True(errors.Is(sdkerrors.ErrOutOfGas, err), "Returned error is not an out of gas error") - s.Require().Equal(gasLimit, uint64(res.GasWanted)) - - txHandler = middleware.ComposeMiddlewares(outOfGasTxHandler, middleware.GasTxMiddleware) - s.Require().Panics(func() { - txHandler.CheckTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx, TxBytes: txBytes}, tx.RequestCheckTx{}) - }, "Recovered from non-Out-of-Gas panic") -} - -// customTxHandler is a test middleware that will run a custom function. -type customTxHandler struct { - fn func(context.Context, tx.Request) (tx.Response, error) -} - -var _ tx.Handler = customTxHandler{} - -func (h customTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - return h.fn(ctx, req) -} -func (h customTxHandler) CheckTx(ctx context.Context, req tx.Request, _ tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - res, err := h.fn(ctx, req) - return res, tx.ResponseCheckTx{}, err -} -func (h customTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - return h.fn(ctx, req) -} - -// noopTxHandler is a test middleware that returns an empty response. -var noopTxHandler = customTxHandler{func(_ context.Context, _ tx.Request) (tx.Response, error) { - return tx.Response{}, nil -}} - -// outOfGasTxHandler is a test middleware that panics with an outOfGas error. -var outOfGasTxHandler = customTxHandler{func(ctx context.Context, _ tx.Request) (tx.Response, error) { - sdkCtx := sdk.UnwrapSDKContext(ctx) - overLimit := sdkCtx.GasMeter().Limit() + 1 - - // Should panic with outofgas error - sdkCtx.GasMeter().ConsumeGas(overLimit, "test panic") - - panic("not reached") -}} diff --git a/x/auth/middleware/index_events.go b/x/auth/middleware/index_events.go deleted file mode 100644 index 2dd831417c2e..000000000000 --- a/x/auth/middleware/index_events.go +++ /dev/null @@ -1,61 +0,0 @@ -package middleware - -import ( - "context" - - sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/types/tx" -) - -type indexEventsTxHandler struct { - // indexEvents defines the set of events in the form {eventType}.{attributeKey}, - // which informs Tendermint what to index. If empty, all events will be indexed. - indexEvents map[string]struct{} - next tx.Handler -} - -// NewIndexEventsTxMiddleware defines a middleware to optionally only index a -// subset of the emitted events inside the Tendermint events indexer. -func NewIndexEventsTxMiddleware(indexEvents map[string]struct{}) tx.Middleware { - return func(txHandler tx.Handler) tx.Handler { - return indexEventsTxHandler{ - indexEvents: indexEvents, - next: txHandler, - } - } -} - -var _ tx.Handler = indexEventsTxHandler{} - -// CheckTx implements tx.Handler.CheckTx method. -func (txh indexEventsTxHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - res, resCheckTx, err := txh.next.CheckTx(ctx, req, checkReq) - if err != nil { - return res, tx.ResponseCheckTx{}, err - } - - res.Events = sdk.MarkEventsToIndex(res.Events, txh.indexEvents) - return res, resCheckTx, nil -} - -// DeliverTx implements tx.Handler.DeliverTx method. -func (txh indexEventsTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - res, err := txh.next.DeliverTx(ctx, req) - if err != nil { - return res, err - } - - res.Events = sdk.MarkEventsToIndex(res.Events, txh.indexEvents) - return res, nil -} - -// SimulateTx implements tx.Handler.SimulateTx method. -func (txh indexEventsTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - res, err := txh.next.SimulateTx(ctx, req) - if err != nil { - return res, err - } - - res.Events = sdk.MarkEventsToIndex(res.Events, txh.indexEvents) - return res, nil -} diff --git a/x/auth/middleware/legacy_router.go b/x/auth/middleware/legacy_router.go deleted file mode 100644 index caf4424c9829..000000000000 --- a/x/auth/middleware/legacy_router.go +++ /dev/null @@ -1,41 +0,0 @@ -package middleware - -import ( - "fmt" - - sdk "github.com/cosmos/cosmos-sdk/types" -) - -type LegacyRouter struct { - routes map[string]sdk.Handler -} - -var _ sdk.Router = NewLegacyRouter() - -// NewRouter returns a reference to a new router. -func NewLegacyRouter() *LegacyRouter { - return &LegacyRouter{ - routes: make(map[string]sdk.Handler), - } -} - -// AddRoute adds a route path to the router with a given handler. The route must -// be alphanumeric. -func (rtr *LegacyRouter) AddRoute(route sdk.Route) sdk.Router { - if !sdk.IsAlphaNumeric(route.Path()) { - panic("route expressions can only contain alphanumeric characters") - } - if rtr.routes[route.Path()] != nil { - panic(fmt.Sprintf("route %s has already been initialized", route.Path())) - } - - rtr.routes[route.Path()] = route.Handler() - return rtr -} - -// Route returns a handler for a given route path. -// -// TODO: Handle expressive matches. -func (rtr *LegacyRouter) Route(_ sdk.Context, path string) sdk.Handler { - return rtr.routes[path] -} diff --git a/x/auth/middleware/legacy_router_test.go b/x/auth/middleware/legacy_router_test.go deleted file mode 100644 index 97517dcdf9b5..000000000000 --- a/x/auth/middleware/legacy_router_test.go +++ /dev/null @@ -1,32 +0,0 @@ -package middleware_test - -import ( - "testing" - - "github.com/stretchr/testify/require" - - sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" -) - -var testHandler = func(_ sdk.Context, _ sdk.Msg) (*sdk.Result, error) { - return &sdk.Result{}, nil -} - -func TestLegacyRouter(t *testing.T) { - rtr := middleware.NewLegacyRouter() - - // require panic on invalid route - require.Panics(t, func() { - rtr.AddRoute(sdk.NewRoute("*", testHandler)) - }) - - rtr.AddRoute(sdk.NewRoute("testRoute", testHandler)) - h := rtr.Route(sdk.Context{}, "testRoute") - require.NotNil(t, h) - - // require panic on duplicate route - require.Panics(t, func() { - rtr.AddRoute(sdk.NewRoute("testRoute", testHandler)) - }) -} diff --git a/x/auth/middleware/middleware.go b/x/auth/middleware/middleware.go deleted file mode 100644 index 812bd96038a3..000000000000 --- a/x/auth/middleware/middleware.go +++ /dev/null @@ -1,115 +0,0 @@ -package middleware - -import ( - sdk "github.com/cosmos/cosmos-sdk/types" - sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/types/tx/signing" - authsigning "github.com/cosmos/cosmos-sdk/x/auth/signing" - "github.com/cosmos/cosmos-sdk/x/auth/types" -) - -// ComposeMiddlewares compose multiple middlewares on top of a tx.Handler. The -// middleware order in the variadic arguments is from outer to inner. -// -// Example: Given a base tx.Handler H, and two middlewares A and B, the -// middleware stack: -// ``` -// A.pre -// B.pre -// H -// B.post -// A.post -// ``` -// is created by calling `ComposeMiddlewares(H, A, B)`. -func ComposeMiddlewares(txHandler tx.Handler, middlewares ...tx.Middleware) tx.Handler { - for i := len(middlewares) - 1; i >= 0; i-- { - txHandler = middlewares[i](txHandler) - } - - return txHandler -} - -type TxHandlerOptions struct { - Debug bool - - // TxDecoder is used to decode the raw tx bytes into a sdk.Tx. - TxDecoder sdk.TxDecoder - - // IndexEvents defines the set of events in the form {eventType}.{attributeKey}, - // which informs Tendermint what to index. If empty, all events will be indexed. - IndexEvents map[string]struct{} - - LegacyRouter sdk.Router - MsgServiceRouter *MsgServiceRouter - - AccountKeeper AccountKeeper - BankKeeper types.BankKeeper - FeegrantKeeper FeegrantKeeper - SignModeHandler authsigning.SignModeHandler - SigGasConsumer func(meter sdk.GasMeter, sig signing.SignatureV2, params types.Params) error - ExtensionOptionChecker ExtensionOptionChecker - TxFeeChecker TxFeeChecker -} - -// NewDefaultTxHandler defines a TxHandler middleware stacks that should work -// for most applications. -func NewDefaultTxHandler(options TxHandlerOptions) (tx.Handler, error) { - if options.TxDecoder == nil { - return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "txDecoder is required for middlewares") - } - - if options.AccountKeeper == nil { - return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "account keeper is required for middlewares") - } - - if options.BankKeeper == nil { - return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "bank keeper is required for middlewares") - } - - if options.SignModeHandler == nil { - return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "sign mode handler is required for middlewares") - } - - return ComposeMiddlewares( - NewRunMsgsTxHandler(options.MsgServiceRouter, options.LegacyRouter), - NewTxDecoderMiddleware(options.TxDecoder), - // Set a new GasMeter on sdk.Context. - // - // Make sure the Gas middleware is outside of all other middlewares - // that reads the GasMeter. In our case, the Recovery middleware reads - // the GasMeter to populate GasInfo. - GasTxMiddleware, - // Recover from panics. Panics outside of this middleware won't be - // caught, be careful! - RecoveryTxMiddleware, - // Choose which events to index in Tendermint. Make sure no events are - // emitted outside of this middleware. - NewIndexEventsTxMiddleware(options.IndexEvents), - // Reject all extension options other than the ones needed by the feemarket. - NewExtensionOptionsMiddleware(options.ExtensionOptionChecker), - ValidateBasicMiddleware, - TxTimeoutHeightMiddleware, - ValidateMemoMiddleware(options.AccountKeeper), - ConsumeTxSizeGasMiddleware(options.AccountKeeper), - // No gas should be consumed in any middleware above in a "post" handler part. See - // ComposeMiddlewares godoc for details. - // `DeductFeeMiddleware` and `IncrementSequenceMiddleware` should be put outside of `WithBranchedStore` middleware, - // so their storage writes are not discarded when tx fails. - DeductFeeMiddleware(options.AccountKeeper, options.BankKeeper, options.FeegrantKeeper, options.TxFeeChecker), - SetPubKeyMiddleware(options.AccountKeeper), - ValidateSigCountMiddleware(options.AccountKeeper), - SigGasConsumeMiddleware(options.AccountKeeper, options.SigGasConsumer), - SigVerificationMiddleware(options.AccountKeeper, options.SignModeHandler), - IncrementSequenceMiddleware(options.AccountKeeper), - // Creates a new MultiStore branch, discards downstream writes if the downstream returns error. - // These kinds of middlewares should be put under this: - // - Could return error after messages executed succesfully. - // - Storage writes should be discarded together when tx failed. - WithBranchedStore, - // Consume block gas. All middlewares whose gas consumption after their `next` handler - // should be accounted for, should go below this middleware. - ConsumeBlockGasMiddleware, - NewTipMiddleware(options.BankKeeper), - ), nil -} diff --git a/x/auth/middleware/msg_service_router.go b/x/auth/middleware/msg_service_router.go deleted file mode 100644 index 79aea3350c91..000000000000 --- a/x/auth/middleware/msg_service_router.go +++ /dev/null @@ -1,136 +0,0 @@ -package middleware - -import ( - "context" - "fmt" - - gogogrpc "github.com/gogo/protobuf/grpc" - "github.com/gogo/protobuf/proto" - "google.golang.org/grpc" - - codectypes "github.com/cosmos/cosmos-sdk/codec/types" - sdk "github.com/cosmos/cosmos-sdk/types" - sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" -) - -// MsgServiceRouter routes fully-qualified Msg service methods to their handler. -type MsgServiceRouter struct { - interfaceRegistry codectypes.InterfaceRegistry - routes map[string]MsgServiceHandler -} - -var _ gogogrpc.Server = &MsgServiceRouter{} - -// NewMsgServiceRouter creates a new MsgServiceRouter. -func NewMsgServiceRouter(registry codectypes.InterfaceRegistry) *MsgServiceRouter { - return &MsgServiceRouter{ - interfaceRegistry: registry, - routes: map[string]MsgServiceHandler{}, - } -} - -// MsgServiceHandler defines a function type which handles Msg service message. -type MsgServiceHandler = func(ctx sdk.Context, req sdk.Msg) (*sdk.Result, error) - -// Handler returns the MsgServiceHandler for a given msg or nil if not found. -func (msr *MsgServiceRouter) Handler(msg sdk.Msg) MsgServiceHandler { - return msr.routes[sdk.MsgTypeURL(msg)] -} - -// HandlerByTypeURL returns the MsgServiceHandler for a given query route path or nil -// if not found. -func (msr *MsgServiceRouter) HandlerByTypeURL(typeURL string) MsgServiceHandler { - return msr.routes[typeURL] -} - -// RegisterService implements the gRPC Server.RegisterService method. sd is a gRPC -// service description, handler is an object which implements that gRPC service. -// -// This function PANICs: -// - if it is called before the service `Msg`s have been registered using -// RegisterInterfaces, -// - or if a service is being registered twice. -func (msr *MsgServiceRouter) RegisterService(sd *grpc.ServiceDesc, handler interface{}) { - // Adds a top-level query handler based on the gRPC service name. - for _, method := range sd.Methods { - fqMethod := fmt.Sprintf("/%s/%s", sd.ServiceName, method.MethodName) - methodHandler := method.Handler - - var requestTypeName string - - // NOTE: This is how we pull the concrete request type for each handler for registering in the InterfaceRegistry. - // This approach is maybe a bit hacky, but less hacky than reflecting on the handler object itself. - // We use a no-op interceptor to avoid actually calling into the handler itself. - _, _ = methodHandler(nil, context.Background(), func(i interface{}) error { - msg, ok := i.(sdk.Msg) - if !ok { - // We panic here because there is no other alternative and the app cannot be initialized correctly - // this should only happen if there is a problem with code generation in which case the app won't - // work correctly anyway. - panic(fmt.Errorf("unable to register service method %s: %T does not implement sdk.Msg", fqMethod, i)) - } - - requestTypeName = sdk.MsgTypeURL(msg) - return nil - }, noopInterceptor) - - // Check that the service Msg fully-qualified method name has already - // been registered (via RegisterInterfaces). If the user registers a - // service without registering according service Msg type, there might be - // some unexpected behavior down the road. Since we can't return an error - // (`Server.RegisterService` interface restriction) we panic (at startup). - reqType, err := msr.interfaceRegistry.Resolve(requestTypeName) - if err != nil || reqType == nil { - panic( - fmt.Errorf( - "type_url %s has not been registered yet. "+ - "Before calling RegisterService, you must register all interfaces by calling the `RegisterInterfaces` "+ - "method on module.BasicManager. Each module should call `msgservice.RegisterMsgServiceDesc` inside its "+ - "`RegisterInterfaces` method with the `_Msg_serviceDesc` generated by proto-gen", - requestTypeName, - ), - ) - } - - // Check that each service is only registered once. If a service is - // registered more than once, then we should error. Since we can't - // return an error (`Server.RegisterService` interface restriction) we - // panic (at startup). - _, found := msr.routes[requestTypeName] - if found { - panic( - fmt.Errorf( - "msg service %s has already been registered. Please make sure to only register each service once. "+ - "This usually means that there are conflicting modules registering the same msg service", - fqMethod, - ), - ) - } - - msr.routes[requestTypeName] = func(ctx sdk.Context, req sdk.Msg) (*sdk.Result, error) { - ctx = ctx.WithEventManager(sdk.NewEventManager()) - interceptor := func(goCtx context.Context, _ interface{}, _ *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) { - goCtx = context.WithValue(goCtx, sdk.SdkContextKey, ctx) - return handler(goCtx, req) - } - // Call the method handler from the service description with the handler object. - // We don't do any decoding here because the decoding was already done. - res, err := methodHandler(handler, sdk.WrapSDKContext(ctx), noopDecoder, interceptor) - if err != nil { - return nil, err - } - - resMsg, ok := res.(proto.Message) - if !ok { - return nil, sdkerrors.Wrapf(sdkerrors.ErrInvalidType, "Expecting proto.Message, got %T", resMsg) - } - - return sdk.WrapServiceResult(ctx, resMsg, err) - } - } -} - -func noopDecoder(_ interface{}) error { return nil } -func noopInterceptor(_ context.Context, _ interface{}, _ *grpc.UnaryServerInfo, _ grpc.UnaryHandler) (interface{}, error) { - return nil, nil -} diff --git a/x/auth/middleware/msg_service_router_test.go b/x/auth/middleware/msg_service_router_test.go deleted file mode 100644 index ca6ec79b5b9a..000000000000 --- a/x/auth/middleware/msg_service_router_test.go +++ /dev/null @@ -1,55 +0,0 @@ -package middleware_test - -import ( - "testing" - - "github.com/stretchr/testify/require" - - "github.com/cosmos/cosmos-sdk/simapp" - "github.com/cosmos/cosmos-sdk/testutil/testdata" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" -) - -func TestRegisterMsgService(t *testing.T) { - // Create an encoding config that doesn't register testdata Msg services. - encCfg := simapp.MakeTestEncodingConfig() - msr := middleware.NewMsgServiceRouter(encCfg.InterfaceRegistry) - require.Panics(t, func() { - testdata.RegisterMsgServer( - msr, - testdata.MsgServerImpl{}, - ) - }) - - // Register testdata Msg services, and rerun `RegisterService`. - testdata.RegisterInterfaces(encCfg.InterfaceRegistry) - require.NotPanics(t, func() { - testdata.RegisterMsgServer( - msr, - testdata.MsgServerImpl{}, - ) - }) -} - -func TestRegisterMsgServiceTwice(t *testing.T) { - // Setup baseapp. - encCfg := simapp.MakeTestEncodingConfig() - msr := middleware.NewMsgServiceRouter(encCfg.InterfaceRegistry) - testdata.RegisterInterfaces(encCfg.InterfaceRegistry) - - // First time registering service shouldn't panic. - require.NotPanics(t, func() { - testdata.RegisterMsgServer( - msr, - testdata.MsgServerImpl{}, - ) - }) - - // Second time should panic. - require.Panics(t, func() { - testdata.RegisterMsgServer( - msr, - testdata.MsgServerImpl{}, - ) - }) -} diff --git a/x/auth/middleware/recovery.go b/x/auth/middleware/recovery.go deleted file mode 100644 index 563389d85fc5..000000000000 --- a/x/auth/middleware/recovery.go +++ /dev/null @@ -1,78 +0,0 @@ -package middleware - -import ( - "context" - "runtime/debug" - - sdk "github.com/cosmos/cosmos-sdk/types" - sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/tx" -) - -type recoveryTxHandler struct { - next tx.Handler -} - -// RecoveryTxMiddleware defines a middleware that catches all panics that -// happen in inner middlewares. -// -// Be careful, it won't catch any panics happening outside! -func RecoveryTxMiddleware(txh tx.Handler) tx.Handler { - return recoveryTxHandler{next: txh} -} - -var _ tx.Handler = recoveryTxHandler{} - -// CheckTx implements tx.Handler.CheckTx method. -func (txh recoveryTxHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (res tx.Response, resCheckTx tx.ResponseCheckTx, err error) { - sdkCtx := sdk.UnwrapSDKContext(ctx) - // Panic recovery. - defer func() { - if r := recover(); r != nil { - err = handleRecovery(r, sdkCtx) - } - }() - - return txh.next.CheckTx(ctx, req, checkReq) -} - -// DeliverTx implements tx.Handler.DeliverTx method. -func (txh recoveryTxHandler) DeliverTx(ctx context.Context, req tx.Request) (res tx.Response, err error) { - sdkCtx := sdk.UnwrapSDKContext(ctx) - // Panic recovery. - defer func() { - if r := recover(); r != nil { - err = handleRecovery(r, sdkCtx) - } - }() - - return txh.next.DeliverTx(ctx, req) -} - -// SimulateTx implements tx.Handler.SimulateTx method. -func (txh recoveryTxHandler) SimulateTx(ctx context.Context, req tx.Request) (res tx.Response, err error) { - sdkCtx := sdk.UnwrapSDKContext(ctx) - // Panic recovery. - defer func() { - if r := recover(); r != nil { - err = handleRecovery(r, sdkCtx) - } - }() - - return txh.next.SimulateTx(ctx, req) -} - -func handleRecovery(r interface{}, sdkCtx sdk.Context) error { - switch r := r.(type) { - case sdk.ErrorOutOfGas: - return sdkerrors.Wrapf(sdkerrors.ErrOutOfGas, - "out of gas in location: %v; gasWanted: %d, gasUsed: %d", - r.Descriptor, sdkCtx.GasMeter().Limit(), sdkCtx.GasMeter().GasConsumed(), - ) - - default: - return sdkerrors.ErrPanic.Wrapf( - "recovered: %v\nstack:\n%v", r, string(debug.Stack()), - ) - } -} diff --git a/x/auth/middleware/run_msgs.go b/x/auth/middleware/run_msgs.go deleted file mode 100644 index d333e5918995..000000000000 --- a/x/auth/middleware/run_msgs.go +++ /dev/null @@ -1,117 +0,0 @@ -package middleware - -import ( - "context" - "strings" - - codectypes "github.com/cosmos/cosmos-sdk/codec/types" - sdk "github.com/cosmos/cosmos-sdk/types" - sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/x/auth/migrations/legacytx" -) - -type runMsgsTxHandler struct { - legacyRouter sdk.Router // router for redirecting legacy Msgs - msgServiceRouter *MsgServiceRouter // router for redirecting Msg service messages -} - -func NewRunMsgsTxHandler(msr *MsgServiceRouter, legacyRouter sdk.Router) tx.Handler { - return runMsgsTxHandler{ - legacyRouter: legacyRouter, - msgServiceRouter: msr, - } -} - -var _ tx.Handler = runMsgsTxHandler{} - -// CheckTx implements tx.Handler.CheckTx method. -func (txh runMsgsTxHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - // Don't run Msgs during CheckTx. - return tx.Response{}, tx.ResponseCheckTx{}, nil -} - -// DeliverTx implements tx.Handler.DeliverTx method. -func (txh runMsgsTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - return txh.runMsgs(sdk.UnwrapSDKContext(ctx), req.Tx.GetMsgs(), req.TxBytes) -} - -// SimulateTx implements tx.Handler.SimulateTx method. -func (txh runMsgsTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - return txh.runMsgs(sdk.UnwrapSDKContext(ctx), req.Tx.GetMsgs(), req.TxBytes) -} - -// runMsgs iterates through a list of messages and executes them with the provided -// Context and execution mode. Messages will only be executed during simulation -// and DeliverTx. An error is returned if any single message fails or if a -// Handler does not exist for a given message route. Otherwise, a reference to a -// Result is returned. The caller must not commit state if an error is returned. -func (txh runMsgsTxHandler) runMsgs(sdkCtx sdk.Context, msgs []sdk.Msg, txBytes []byte) (tx.Response, error) { - // Attempt to execute all messages and only update state if all messages pass - // and we're in DeliverTx. Note, runMsgs will never return a reference to a - // Result if any single message fails or does not have a registered Handler. - msgLogs := make(sdk.ABCIMessageLogs, 0, len(msgs)) - events := sdkCtx.EventManager().Events() - msgResponses := make([]*codectypes.Any, len(msgs)) - - // NOTE: GasWanted is determined by the Gas TxHandler and GasUsed by the GasMeter. - for i, msg := range msgs { - var ( - msgResult *sdk.Result - eventMsgName string // name to use as value in event `message.action` - err error - ) - - if handler := txh.msgServiceRouter.Handler(msg); handler != nil { - // ADR 031 request type routing - msgResult, err = handler(sdkCtx, msg) - eventMsgName = sdk.MsgTypeURL(msg) - } else if legacyMsg, ok := msg.(legacytx.LegacyMsg); ok { - // legacy sdk.Msg routing - // Assuming that the app developer has migrated all their Msgs to - // proto messages and has registered all `Msg services`, then this - // path should never be called, because all those Msgs should be - // registered within the `MsgServiceRouter` already. - msgRoute := legacyMsg.Route() - eventMsgName = legacyMsg.Type() - handler := txh.legacyRouter.Route(sdkCtx, msgRoute) - if handler == nil { - return tx.Response{}, sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "unrecognized message route: %s; message index: %d", msgRoute, i) - } - - msgResult, err = handler(sdkCtx, msg) - } else { - return tx.Response{}, sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "can't route message %+v", msg) - } - - if err != nil { - return tx.Response{}, sdkerrors.Wrapf(err, "failed to execute message; message index: %d", i) - } - - msgEvents := sdk.Events{ - sdk.NewEvent(sdk.EventTypeMessage, sdk.NewAttribute(sdk.AttributeKeyAction, eventMsgName)), - } - msgEvents = msgEvents.AppendEvents(msgResult.GetEvents()) - - // append message events, data and logs - // - // Note: Each message result's data must be length-prefixed in order to - // separate each result. - events = events.AppendEvents(msgEvents) - - // Each individual sdk.Result has exactly one Msg response. We aggregate here. - msgResponse := msgResult.MsgResponses[0] - if msgResponse == nil { - return tx.Response{}, sdkerrors.ErrLogic.Wrapf("got nil Msg response at index %d for msg %s", i, sdk.MsgTypeURL(msg)) - } - msgResponses[i] = msgResponse - msgLogs = append(msgLogs, sdk.NewABCIMessageLog(uint32(i), msgResult.Log, msgEvents)) - } - - return tx.Response{ - // GasInfo will be populated by the Gas middleware. - Log: strings.TrimSpace(msgLogs.String()), - Events: events.ToABCIEvents(), - MsgResponses: msgResponses, - }, nil -} diff --git a/x/auth/middleware/run_msgs_test.go b/x/auth/middleware/run_msgs_test.go deleted file mode 100644 index 143a21975380..000000000000 --- a/x/auth/middleware/run_msgs_test.go +++ /dev/null @@ -1,40 +0,0 @@ -package middleware_test - -import ( - "fmt" - - "github.com/gogo/protobuf/proto" - - cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" - "github.com/cosmos/cosmos-sdk/testutil/testdata" - sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" -) - -func (s *MWTestSuite) TestRunMsgs() { - ctx := s.SetupTest(true) // setup - - msr := middleware.NewMsgServiceRouter(s.clientCtx.InterfaceRegistry) - testdata.RegisterMsgServer(msr, testdata.MsgServerImpl{}) - txHandler := middleware.NewRunMsgsTxHandler(msr, nil) - - priv, _, _ := testdata.KeyTestPubAddr() - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() - txBuilder.SetMsgs(&testdata.MsgCreateDog{Dog: &testdata.Dog{Name: "Spot"}}) - privs, accNums, accSeqs := []cryptotypes.PrivKey{priv}, []uint64{0}, []uint64{0} - testTx, _, err := s.createTestTx(txBuilder, privs, accNums, accSeqs, ctx.ChainID()) - s.Require().NoError(err) - txBytes, err := s.clientCtx.TxConfig.TxEncoder()(testTx) - s.Require().NoError(err) - - // DeliverTx - res, err := txHandler.DeliverTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx, TxBytes: txBytes}) - s.Require().NoError(err) - s.Require().Len(res.MsgResponses, 1) - s.Require().Equal(fmt.Sprintf("/%s", proto.MessageName(&testdata.MsgCreateDogResponse{})), res.MsgResponses[0].TypeUrl) - - // SimulateTx - _, err = txHandler.SimulateTx(sdk.WrapSDKContext(ctx), tx.Request{Tx: testTx, TxBytes: txBytes}) - s.Require().NoError(err) -} diff --git a/x/auth/middleware/testutil_test.go b/x/auth/middleware/testutil_test.go deleted file mode 100644 index f493a463fa2a..000000000000 --- a/x/auth/middleware/testutil_test.go +++ /dev/null @@ -1,222 +0,0 @@ -package middleware_test - -import ( - "errors" - "fmt" - "testing" - - "github.com/stretchr/testify/suite" - tmproto "github.com/tendermint/tendermint/proto/tendermint/types" - - "github.com/cosmos/cosmos-sdk/client" - "github.com/cosmos/cosmos-sdk/client/tx" - "github.com/cosmos/cosmos-sdk/codec" - codectypes "github.com/cosmos/cosmos-sdk/codec/types" - cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" - "github.com/cosmos/cosmos-sdk/simapp" - "github.com/cosmos/cosmos-sdk/testutil/testdata" - sdk "github.com/cosmos/cosmos-sdk/types" - txtypes "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/types/tx/signing" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" - xauthsigning "github.com/cosmos/cosmos-sdk/x/auth/signing" - authtypes "github.com/cosmos/cosmos-sdk/x/auth/types" - minttypes "github.com/cosmos/cosmos-sdk/x/mint/types" -) - -// testAccount represents an account used in the tests in x/auth/middleware. -type testAccount struct { - acc authtypes.AccountI - priv cryptotypes.PrivKey - accNum uint64 -} - -// MWTestSuite is a test suite to be used with middleware tests. -type MWTestSuite struct { - suite.Suite - - app *simapp.SimApp - clientCtx client.Context - txHandler txtypes.Handler -} - -// returns context and app with params set on account keeper -func createTestApp(t *testing.T, isCheckTx bool) (*simapp.SimApp, sdk.Context) { - app := simapp.Setup(t, isCheckTx) - ctx := app.BaseApp.NewContext(isCheckTx, tmproto.Header{Height: app.LastBlockHeight() + 1}).WithBlockGasMeter(sdk.NewInfiniteGasMeter()) - app.AccountKeeper.SetParams(ctx, authtypes.DefaultParams()) - - return app, ctx -} - -// setupTest setups a new test, with new app and context. -func (s *MWTestSuite) SetupTest(isCheckTx bool) sdk.Context { - var ctx sdk.Context - s.app, ctx = createTestApp(s.T(), isCheckTx) - - // Set up TxConfig. - encodingConfig := simapp.MakeTestEncodingConfig() - // We're using TestMsg encoding in some tests, so register it here. - encodingConfig.Amino.RegisterConcrete(&testdata.TestMsg{}, "testdata.TestMsg", nil) - testdata.RegisterInterfaces(encodingConfig.InterfaceRegistry) - - s.clientCtx = client.Context{}. - WithTxConfig(encodingConfig.TxConfig). - WithInterfaceRegistry(encodingConfig.InterfaceRegistry). - WithCodec(codec.NewAminoCodec(encodingConfig.Amino)) - - // We don't use simapp's own txHandler. For more flexibility (i.e. around - // using testdata), we create own own txHandler for this test suite. - msr := middleware.NewMsgServiceRouter(encodingConfig.InterfaceRegistry) - testdata.RegisterMsgServer(msr, testdata.MsgServerImpl{}) - legacyRouter := middleware.NewLegacyRouter() - legacyRouter.AddRoute(sdk.NewRoute((&testdata.TestMsg{}).Route(), func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { - any, err := codectypes.NewAnyWithValue(msg) - if err != nil { - return nil, err - } - - return &sdk.Result{ - MsgResponses: []*codectypes.Any{any}, - }, nil - })) - txHandler, err := middleware.NewDefaultTxHandler(middleware.TxHandlerOptions{ - Debug: s.app.Trace(), - MsgServiceRouter: msr, - LegacyRouter: legacyRouter, - AccountKeeper: s.app.AccountKeeper, - BankKeeper: s.app.BankKeeper, - FeegrantKeeper: s.app.FeeGrantKeeper, - SignModeHandler: encodingConfig.TxConfig.SignModeHandler(), - SigGasConsumer: middleware.DefaultSigVerificationGasConsumer, - TxDecoder: s.clientCtx.TxConfig.TxDecoder(), - }) - s.Require().NoError(err) - s.txHandler = txHandler - - return ctx -} - -// createTestAccounts creates `numAccs` accounts, and return all relevant -// information about them including their private keys. -func (s *MWTestSuite) createTestAccounts(ctx sdk.Context, numAccs int, coins sdk.Coins) []testAccount { - var accounts []testAccount - - for i := 0; i < numAccs; i++ { - priv, _, addr := testdata.KeyTestPubAddr() - acc := s.app.AccountKeeper.NewAccountWithAddress(ctx, addr) - accNum := uint64(i) - err := acc.SetAccountNumber(accNum) - s.Require().NoError(err) - s.app.AccountKeeper.SetAccount(ctx, acc) - err = s.app.BankKeeper.MintCoins(ctx, minttypes.ModuleName, coins) - s.Require().NoError(err) - - err = s.app.BankKeeper.SendCoinsFromModuleToAccount(ctx, minttypes.ModuleName, addr, coins) - s.Require().NoError(err) - - accounts = append(accounts, testAccount{acc, priv, accNum}) - } - - return accounts -} - -// createTestTx is a helper function to create a tx given multiple inputs. -func (s *MWTestSuite) createTestTx(txBuilder client.TxBuilder, privs []cryptotypes.PrivKey, accNums []uint64, accSeqs []uint64, chainID string) (xauthsigning.Tx, []byte, error) { - // First round: we gather all the signer infos. We use the "set empty - // signature" hack to do that. - var sigsV2 []signing.SignatureV2 - for i, priv := range privs { - sigV2 := signing.SignatureV2{ - PubKey: priv.PubKey(), - Data: &signing.SingleSignatureData{ - SignMode: s.clientCtx.TxConfig.SignModeHandler().DefaultMode(), - Signature: nil, - }, - Sequence: accSeqs[i], - } - - sigsV2 = append(sigsV2, sigV2) - } - err := txBuilder.SetSignatures(sigsV2...) - if err != nil { - return nil, nil, err - } - - // Second round: all signer infos are set, so each signer can sign. - sigsV2 = []signing.SignatureV2{} - for i, priv := range privs { - signerData := xauthsigning.SignerData{ - Address: sdk.AccAddress(priv.PubKey().Address()).String(), - ChainID: chainID, - AccountNumber: accNums[i], - Sequence: accSeqs[i], - PubKey: priv.PubKey(), - } - sigV2, err := tx.SignWithPrivKey( - s.clientCtx.TxConfig.SignModeHandler().DefaultMode(), signerData, - txBuilder, priv, s.clientCtx.TxConfig, accSeqs[i]) - if err != nil { - return nil, nil, err - } - - sigsV2 = append(sigsV2, sigV2) - } - err = txBuilder.SetSignatures(sigsV2...) - if err != nil { - return nil, nil, err - } - - txBytes, err := s.clientCtx.TxConfig.TxEncoder()(txBuilder.GetTx()) - if err != nil { - return nil, nil, err - } - - return txBuilder.GetTx(), txBytes, nil -} - -func (s *MWTestSuite) runTestCase(ctx sdk.Context, txBuilder client.TxBuilder, privs []cryptotypes.PrivKey, msgs []sdk.Msg, feeAmount sdk.Coins, gasLimit uint64, accNums, accSeqs []uint64, chainID string, tc TestCase) { - s.Run(fmt.Sprintf("Case %s", tc.desc), func() { - s.Require().NoError(txBuilder.SetMsgs(msgs...)) - txBuilder.SetFeeAmount(feeAmount) - txBuilder.SetGasLimit(gasLimit) - - // Theoretically speaking, middleware unit tests should only test - // middlewares, but here we sometimes also test the tx creation - // process. - testTx, _, txErr := s.createTestTx(txBuilder, privs, accNums, accSeqs, chainID) - newCtx, txHandlerErr := s.txHandler.DeliverTx(sdk.WrapSDKContext(ctx), txtypes.Request{Tx: testTx}) - - if tc.expPass { - s.Require().NoError(txErr) - s.Require().NoError(txHandlerErr) - s.Require().NotNil(newCtx) - } else { - switch { - case txErr != nil: - s.Require().Error(txErr) - s.Require().True(errors.Is(txErr, tc.expErr)) - - case txHandlerErr != nil: - s.Require().Error(txHandlerErr) - s.Require().True(errors.Is(txHandlerErr, tc.expErr)) - - default: - s.Fail("expected one of txErr,txHandlerErr to be an error") - } - } - }) -} - -// TestCase represents a test case used in test tables. -type TestCase struct { - desc string - malleate func() - simulate bool - expPass bool - expErr error -} - -func TestMWTestSuite(t *testing.T) { - suite.Run(t, new(MWTestSuite)) -} diff --git a/x/auth/middleware/tips.go b/x/auth/middleware/tips.go deleted file mode 100644 index 68749b020647..000000000000 --- a/x/auth/middleware/tips.go +++ /dev/null @@ -1,69 +0,0 @@ -package middleware - -import ( - "context" - - sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/x/auth/types" -) - -type tipsTxHandler struct { - next tx.Handler - bankKeeper types.BankKeeper -} - -// NewTipMiddleware returns a new middleware for handling transactions with -// tips. -func NewTipMiddleware(bankKeeper types.BankKeeper) tx.Middleware { - return func(txh tx.Handler) tx.Handler { - return tipsTxHandler{txh, bankKeeper} - } -} - -var _ tx.Handler = tipsTxHandler{} - -// CheckTx implements tx.Handler.CheckTx. -func (txh tipsTxHandler) CheckTx(ctx context.Context, req tx.Request, checkTx tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - res, resCheckTx, err := txh.next.CheckTx(ctx, req, checkTx) - res, err = txh.transferTip(ctx, req, res, err) - - return res, resCheckTx, err -} - -// DeliverTx implements tx.Handler.DeliverTx. -func (txh tipsTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - res, err := txh.next.DeliverTx(ctx, req) - - return txh.transferTip(ctx, req, res, err) -} - -// SimulateTx implements tx.Handler.SimulateTx method. -func (txh tipsTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - res, err := txh.next.SimulateTx(ctx, req) - - return txh.transferTip(ctx, req, res, err) -} - -// transferTip transfers the tip from the tipper to the fee payer. -func (txh tipsTxHandler) transferTip(ctx context.Context, req tx.Request, res tx.Response, err error) (tx.Response, error) { - tipTx, ok := req.Tx.(tx.TipTx) - - // No-op if the tx doesn't have tips. - if !ok || tipTx.GetTip() == nil { - return res, err - } - - sdkCtx := sdk.UnwrapSDKContext(ctx) - tipper, err := sdk.AccAddressFromBech32(tipTx.GetTip().Tipper) - if err != nil { - return tx.Response{}, err - } - - err = txh.bankKeeper.SendCoins(sdkCtx, tipper, tipTx.FeePayer(), tipTx.GetTip().Amount) - if err != nil { - return tx.Response{}, err - } - - return res, nil -} diff --git a/x/auth/middleware/tips_test.go b/x/auth/middleware/tips_test.go deleted file mode 100644 index 7fb50f7e0f0f..000000000000 --- a/x/auth/middleware/tips_test.go +++ /dev/null @@ -1,206 +0,0 @@ -package middleware_test - -import ( - "time" - - abci "github.com/tendermint/tendermint/abci/types" - tmproto "github.com/tendermint/tendermint/proto/tendermint/types" - - "github.com/cosmos/cosmos-sdk/client" - clienttx "github.com/cosmos/cosmos-sdk/client/tx" - cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" - sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/types/tx/signing" - authsigning "github.com/cosmos/cosmos-sdk/x/auth/signing" - banktypes "github.com/cosmos/cosmos-sdk/x/bank/types" - govtypes "github.com/cosmos/cosmos-sdk/x/gov/types/v1" - minttypes "github.com/cosmos/cosmos-sdk/x/mint/types" -) - -var initialRegens = sdk.NewCoins(sdk.NewCoin("regen", sdk.NewInt(1000))) -var initialAtoms = sdk.NewCoins(sdk.NewCoin("atom", sdk.NewInt(1000))) - -// setupAcctsForTips sets up 2 accounts: -// - tipper has 1000 regens -// - feePayer has 1000 atoms and 1000 regens -func (s *MWTestSuite) setupAcctsForTips(ctx sdk.Context) (sdk.Context, []testAccount) { - accts := s.createTestAccounts(ctx, 2, initialRegens) - feePayer := accts[1] - err := s.app.BankKeeper.MintCoins(ctx, minttypes.ModuleName, initialAtoms) - s.Require().NoError(err) - err = s.app.BankKeeper.SendCoinsFromModuleToAccount(ctx, minttypes.ModuleName, feePayer.acc.GetAddress(), initialAtoms) - s.Require().NoError(err) - - // Create dummy proposal for tipper to vote on. - prop, err := govtypes.NewProposal([]sdk.Msg{banktypes.NewMsgSend(accts[0].acc.GetAddress(), accts[0].acc.GetAddress(), initialRegens)}, 1, "", time.Now(), time.Now().Add(time.Hour)) - s.Require().NoError(err) - s.app.GovKeeper.SetProposal(ctx, prop) - s.app.GovKeeper.ActivateVotingPeriod(ctx, prop) - - // Move to next block to commit previous data to state. - s.app.EndBlock(abci.RequestEndBlock{Height: ctx.BlockHeight()}) - s.app.Commit() - - ctx = ctx.WithBlockHeight(ctx.BlockHeight() + 1) - s.app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: ctx.BlockHeight()}}) - - return ctx, accts -} - -func (s *MWTestSuite) TestTips() { - var msg sdk.Msg - - testcases := []struct { - name string - tip sdk.Coins - fee sdk.Coins - gasLimit uint64 - expErr bool - expErrStr string - }{ - { - "wrong tip denom", - sdk.NewCoins(sdk.NewCoin("foobar", sdk.NewInt(1000))), initialAtoms, 200000, - true, "0foobar is smaller than 1000foobar: insufficient funds", - }, - { - "insufficient tip from tipper", - sdk.NewCoins(sdk.NewCoin("regen", sdk.NewInt(5000))), initialAtoms, 200000, - true, "1000regen is smaller than 5000regen: insufficient funds", - }, - { - "insufficient fees from feePayer", - initialRegens, sdk.NewCoins(sdk.NewCoin("atom", sdk.NewInt(5000))), 200000, - true, "1000atom is smaller than 5000atom: insufficient funds: insufficient funds", - }, - { - "insufficient gas", - initialRegens, initialAtoms, 100, - true, "out of gas in location: ReadFlat; gasWanted: 100, gasUsed: 1000: out of gas", - }, - { - "happy case", - initialRegens, initialAtoms, 200000, - false, "", - }, - } - - for _, tc := range testcases { - tc := tc - s.Run(tc.name, func() { - ctx := s.SetupTest(false) // reset - ctx, accts := s.setupAcctsForTips(ctx) - tipper, feePayer := accts[0], accts[1] - - msg = govtypes.NewMsgVote(tipper.acc.GetAddress(), 1, govtypes.OptionYes, "") - - auxSignerData := s.mkTipperAuxSignerData(tipper.priv, msg, tc.tip, signing.SignMode_SIGN_MODE_DIRECT_AUX, tipper.accNum, 0, ctx.ChainID()) - feePayerTxBuilder := s.mkFeePayerTxBuilder(s.clientCtx, auxSignerData, feePayer.priv, signing.SignMode_SIGN_MODE_DIRECT, tx.Fee{Amount: tc.fee, GasLimit: tc.gasLimit}, feePayer.accNum, 0, ctx.ChainID()) - - _, res, err := s.app.SimDeliver(s.clientCtx.TxConfig.TxEncoder(), feePayerTxBuilder.GetTx()) - - if tc.expErr { - s.Require().Error(err) - s.Require().Contains(err.Error(), tc.expErrStr) - } else { - s.Require().NoError(err) - s.Require().NotNil(res) - - // Move to next block to commit previous data to state. - s.app.EndBlock(abci.RequestEndBlock{Height: ctx.BlockHeight()}) - s.app.Commit() - - ctx = ctx.WithBlockHeight(ctx.BlockHeight() + 1) - s.app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: ctx.BlockHeight()}}) - - // Make sure tip is correctly transferred to feepayer, and fee is paid. - expTipperRegens := initialRegens.Sub(tc.tip...) - expFeePayerRegens := initialRegens.Add(tc.tip...) - expFeePayerAtoms := initialAtoms.Sub(tc.fee...) - s.Require().True(expTipperRegens.AmountOf("regen").Equal(s.app.BankKeeper.GetBalance(ctx, tipper.acc.GetAddress(), "regen").Amount)) - s.Require().True(expFeePayerRegens.AmountOf("regen").Equal(s.app.BankKeeper.GetBalance(ctx, feePayer.acc.GetAddress(), "regen").Amount)) - s.Require().True(expFeePayerAtoms.AmountOf("atom").Equal(s.app.BankKeeper.GetBalance(ctx, feePayer.acc.GetAddress(), "atom").Amount)) - // Make sure MsgVote has been submitted by tipper. - votes := s.app.GovKeeper.GetAllVotes(ctx) - s.Require().Len(votes, 1) - s.Require().Equal(tipper.acc.GetAddress().String(), votes[0].Voter) - } - }) - } -} - -func (s *MWTestSuite) mkTipperAuxSignerData( - tipperPriv cryptotypes.PrivKey, msg sdk.Msg, tip sdk.Coins, - signMode signing.SignMode, accNum, accSeq uint64, chainID string, -) tx.AuxSignerData { - tipperAddr := sdk.AccAddress(tipperPriv.PubKey().Address()).String() - b := clienttx.NewAuxTxBuilder() - b.SetAddress(tipperAddr) - b.SetAccountNumber(accNum) - b.SetSequence(accSeq) - err := b.SetMsgs(msg) - s.Require().NoError(err) - b.SetTip(&tx.Tip{Amount: tip, Tipper: tipperAddr}) - err = b.SetSignMode(signMode) - s.Require().NoError(err) - b.SetSequence(accSeq) - err = b.SetPubKey(tipperPriv.PubKey()) - s.Require().NoError(err) - b.SetChainID(chainID) - - signBz, err := b.GetSignBytes() - s.Require().NoError(err) - sig, err := tipperPriv.Sign(signBz) - s.Require().NoError(err) - b.SetSignature(sig) - - auxSignerData, err := b.GetAuxSignerData() - s.Require().NoError(err) - - return auxSignerData -} - -func (s *MWTestSuite) mkFeePayerTxBuilder( - clientCtx client.Context, - auxSignerData tx.AuxSignerData, - feePayerPriv cryptotypes.PrivKey, signMode signing.SignMode, - fee tx.Fee, accNum, accSeq uint64, chainID string, -) client.TxBuilder { - txBuilder := clientCtx.TxConfig.NewTxBuilder() - err := txBuilder.AddAuxSignerData(auxSignerData) - s.Require().NoError(err) - txBuilder.SetFeePayer(sdk.AccAddress(feePayerPriv.PubKey().Address())) - txBuilder.SetFeeAmount(fee.Amount) - txBuilder.SetGasLimit(fee.GasLimit) - - // Calling SetSignatures with empty sig to populate AuthInfo. - tipperSigsV2, err := auxSignerData.GetSignatureV2() - s.Require().NoError(err) - feePayerSigV2 := signing.SignatureV2{ - PubKey: feePayerPriv.PubKey(), - Data: &signing.SingleSignatureData{ - SignMode: signMode, - Signature: nil, - }} - sigsV2 := append([]signing.SignatureV2{tipperSigsV2}, feePayerSigV2) - txBuilder.SetSignatures(sigsV2...) - - // Actually sign the data. - signerData := authsigning.SignerData{ - Address: sdk.AccAddress(feePayerPriv.PubKey().Address()).String(), - ChainID: chainID, - AccountNumber: accNum, - Sequence: accSeq, - PubKey: feePayerPriv.PubKey(), - } - feePayerSigV2, err = clienttx.SignWithPrivKey( - signMode, signerData, - txBuilder, feePayerPriv, clientCtx.TxConfig, accSeq) - s.Require().NoError(err) - sigsV2 = append([]signing.SignatureV2{tipperSigsV2}, feePayerSigV2) - err = txBuilder.SetSignatures(sigsV2...) - s.Require().NoError(err) - - return txBuilder -} diff --git a/x/auth/middleware/tx.go b/x/auth/middleware/tx.go deleted file mode 100644 index f543524f3fa7..000000000000 --- a/x/auth/middleware/tx.go +++ /dev/null @@ -1,77 +0,0 @@ -package middleware - -import ( - "context" - - sdk "github.com/cosmos/cosmos-sdk/types" - sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/tx" -) - -type txDecoderHandler struct { - next tx.Handler - txDecoder sdk.TxDecoder -} - -// NewTxDecoderMiddleware creates a new middleware that will decode tx bytes -// into a sdk.Tx. As input request, at least one of Tx or TxBytes must be set. -// If only TxBytes is set, then TxDecoderMiddleware will populate the Tx field. -// If only Tx is set, then TxBytes will be left empty, but some middlewares -// such as signature verification might fail. -func NewTxDecoderMiddleware(txDecoder sdk.TxDecoder) tx.Middleware { - return func(txh tx.Handler) tx.Handler { - return txDecoderHandler{next: txh, txDecoder: txDecoder} - } -} - -var _ tx.Handler = gasTxHandler{} - -// CheckTx implements tx.Handler.CheckTx. -func (h txDecoderHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - newReq, err := h.populateReq(req) - if err != nil { - return tx.Response{}, tx.ResponseCheckTx{}, err - } - - return h.next.CheckTx(ctx, newReq, checkReq) -} - -// DeliverTx implements tx.Handler.DeliverTx. -func (h txDecoderHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - newReq, err := h.populateReq(req) - if err != nil { - return tx.Response{}, err - } - - return h.next.DeliverTx(ctx, newReq) -} - -// SimulateTx implements tx.Handler.SimulateTx method. -func (h txDecoderHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - newReq, err := h.populateReq(req) - if err != nil { - return tx.Response{}, err - } - - return h.next.SimulateTx(ctx, newReq) -} - -// populateReq takes a tx.Request, and if its Tx field is not set, then -// decodes the TxBytes and populates the decoded Tx field. It leaves -// req.TxBytes untouched. -func (h txDecoderHandler) populateReq(req tx.Request) (tx.Request, error) { - if len(req.TxBytes) == 0 && req.Tx == nil { - return tx.Request{}, sdkerrors.ErrInvalidRequest.Wrap("got empty tx request") - } - - sdkTx := req.Tx - var err error - if len(req.TxBytes) != 0 { - sdkTx, err = h.txDecoder(req.TxBytes) - if err != nil { - return tx.Request{}, err - } - } - - return tx.Request{Tx: sdkTx, TxBytes: req.TxBytes}, nil -} diff --git a/x/auth/middleware/tx_test.go b/x/auth/middleware/tx_test.go deleted file mode 100644 index 9e4ce65ed648..000000000000 --- a/x/auth/middleware/tx_test.go +++ /dev/null @@ -1,64 +0,0 @@ -package middleware_test - -import ( - "context" - - cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" - "github.com/cosmos/cosmos-sdk/testutil/testdata" - sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" -) - -func (s *MWTestSuite) TestTxDecoderMiddleware() { - ctx := s.SetupTest(true) // setup - require := s.Require() - - // Create a tx. - priv1, _, addr1 := testdata.KeyTestPubAddr() - txBuilder := s.clientCtx.TxConfig.NewTxBuilder() - err := txBuilder.SetMsgs(testdata.NewTestMsg(addr1)) - require.NoError(err) - sdkTx, txBz, err := s.createTestTx(txBuilder, []cryptotypes.PrivKey{priv1}, []uint64{1}, []uint64{0}, ctx.ChainID()) - require.NoError(err) - - // Create a custom tx.Handler that checks that the req.Tx field is - // correctly populated. - txReqChecker := customTxHandler{func(c context.Context, r tx.Request) (tx.Response, error) { - require.NotNil(r.Tx) - require.Equal(sdkTx.GetMsgs()[0], r.Tx.GetMsgs()[0]) - return tx.Response{}, nil - }} - - testcases := []struct { - name string - req tx.Request - expErr bool - }{ - {"empty tx bz", tx.Request{}, true}, - {"tx bz and tx both given as inputs", tx.Request{Tx: sdkTx, TxBytes: txBz}, false}, - {"tx bz only given as input", tx.Request{TxBytes: txBz}, false}, - {"tx only given as input", tx.Request{Tx: sdkTx}, false}, - } - for _, tc := range testcases { - s.Run(tc.name, func() { - txHandler := middleware.ComposeMiddlewares( - txReqChecker, - middleware.NewTxDecoderMiddleware(s.clientCtx.TxConfig.TxDecoder()), - ) - - // DeliverTx - _, err := txHandler.DeliverTx(sdk.WrapSDKContext(ctx), tc.req) - - // SimulateTx - _, simErr := txHandler.SimulateTx(sdk.WrapSDKContext(ctx), tc.req) - if tc.expErr { - require.Error(err) - require.Error(simErr) - } else { - require.NoError(err) - require.NoError(simErr) - } - }) - } -} diff --git a/x/auth/middleware/validator_tx_fee.go b/x/auth/middleware/validator_tx_fee.go deleted file mode 100644 index 687608da55a9..000000000000 --- a/x/auth/middleware/validator_tx_fee.go +++ /dev/null @@ -1,59 +0,0 @@ -package middleware - -import ( - "math" - - sdk "github.com/cosmos/cosmos-sdk/types" - sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" -) - -// checkTxFeeWithValidatorMinGasPrices implements the default fee logic, where the minimum price per -// unit of gas is fixed and set by each validator, can the tx priority is computed from the gas price. -func checkTxFeeWithValidatorMinGasPrices(ctx sdk.Context, tx sdk.Tx) (sdk.Coins, int64, error) { - feeTx, ok := tx.(sdk.FeeTx) - if !ok { - return nil, 0, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "Tx must be a FeeTx") - } - - feeCoins := feeTx.GetFee() - gas := feeTx.GetGas() - - // Ensure that the provided fees meet a minimum threshold for the validator, - // This is only for local mempool purposes, if this is a DeliverTx, the `MinGasPrices` should be zero. - minGasPrices := ctx.MinGasPrices() - if !minGasPrices.IsZero() { - requiredFees := make(sdk.Coins, len(minGasPrices)) - - // Determine the required fees by multiplying each required minimum gas - // price by the gas limit, where fee = ceil(minGasPrice * gasLimit). - glDec := sdk.NewDec(int64(gas)) - for i, gp := range minGasPrices { - fee := gp.Amount.Mul(glDec) - requiredFees[i] = sdk.NewCoin(gp.Denom, fee.Ceil().RoundInt()) - } - - if !feeCoins.IsAnyGTE(requiredFees) { - return nil, 0, sdkerrors.Wrapf(sdkerrors.ErrInsufficientFee, "insufficient fees; got: %s required: %s", feeCoins, requiredFees) - } - } - - priority := getTxPriority(feeCoins) - return feeCoins, priority, nil -} - -// getTxPriority returns a naive tx priority based on the amount of the smallest denomination of the fee -// provided in a transaction. -func getTxPriority(fee sdk.Coins) int64 { - var priority int64 - for _, c := range fee { - p := int64(math.MaxInt64) - if c.Amount.IsInt64() { - p = c.Amount.Int64() - } - if priority == 0 || p < priority { - priority = p - } - } - - return priority -} diff --git a/x/auth/signing/verify_test.go b/x/auth/signing/verify_test.go index 3d138e1f7742..db91bc15e06d 100644 --- a/x/auth/signing/verify_test.go +++ b/x/auth/signing/verify_test.go @@ -13,7 +13,7 @@ import ( "github.com/cosmos/cosmos-sdk/simapp" "github.com/cosmos/cosmos-sdk/testutil/testdata" sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" + "github.com/cosmos/cosmos-sdk/x/auth/ante" "github.com/cosmos/cosmos-sdk/x/auth/migrations/legacytx" "github.com/cosmos/cosmos-sdk/x/auth/signing" "github.com/cosmos/cosmos-sdk/x/auth/types" @@ -42,7 +42,7 @@ func TestVerifySignature(t *testing.T) { app.AccountKeeper.SetAccount(ctx, acc1) balances := sdk.NewCoins(sdk.NewInt64Coin("atom", 200)) require.NoError(t, testutil.FundAccount(app.BankKeeper, ctx, addr, balances)) - acc, err := middleware.GetSignerAcc(ctx, app.AccountKeeper, addr) + acc, err := ante.GetSignerAcc(ctx, app.AccountKeeper, addr) require.NoError(t, err) require.NoError(t, testutil.FundAccount(app.BankKeeper, ctx, addr, balances)) diff --git a/x/auth/tx/builder.go b/x/auth/tx/builder.go index edcfd630ca82..052da8785ee8 100644 --- a/x/auth/tx/builder.go +++ b/x/auth/tx/builder.go @@ -11,7 +11,7 @@ import ( sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" "github.com/cosmos/cosmos-sdk/types/tx" "github.com/cosmos/cosmos-sdk/types/tx/signing" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" + "github.com/cosmos/cosmos-sdk/x/auth/ante" authsigning "github.com/cosmos/cosmos-sdk/x/auth/signing" ) @@ -37,7 +37,7 @@ var ( _ authsigning.Tx = &wrapper{} _ client.TxBuilder = &wrapper{} _ tx.TipTx = &wrapper{} - _ middleware.HasExtensionOptionsTx = &wrapper{} + _ ante.HasExtensionOptionsTx = &wrapper{} _ ExtensionOptionsTxBuilder = &wrapper{} _ tx.TipTx = &wrapper{} ) diff --git a/x/authz/keeper/keeper.go b/x/authz/keeper/keeper.go index 0a5f28234133..1b9cb7855db6 100644 --- a/x/authz/keeper/keeper.go +++ b/x/authz/keeper/keeper.go @@ -8,13 +8,13 @@ import ( "github.com/tendermint/tendermint/libs/log" + "github.com/cosmos/cosmos-sdk/baseapp" "github.com/cosmos/cosmos-sdk/codec" codectypes "github.com/cosmos/cosmos-sdk/codec/types" storetypes "github.com/cosmos/cosmos-sdk/store/types" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" authkeeper "github.com/cosmos/cosmos-sdk/x/auth/keeper" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" "github.com/cosmos/cosmos-sdk/x/authz" ) @@ -26,12 +26,12 @@ const gasCostPerIteration = uint64(20) type Keeper struct { storeKey storetypes.StoreKey cdc codec.BinaryCodec - router *middleware.MsgServiceRouter + router *baseapp.MsgServiceRouter authKeeper authkeeper.AccountKeeper } // NewKeeper constructs a message authorization Keeper -func NewKeeper(storeKey storetypes.StoreKey, cdc codec.BinaryCodec, router *middleware.MsgServiceRouter, ak authkeeper.AccountKeeper) Keeper { +func NewKeeper(storeKey storetypes.StoreKey, cdc codec.BinaryCodec, router *baseapp.MsgServiceRouter, ak authkeeper.AccountKeeper) Keeper { return Keeper{ storeKey: storeKey, cdc: cdc, diff --git a/x/feegrant/keeper/keeper.go b/x/feegrant/keeper/keeper.go index 4800d512f242..ae4bdc10ef18 100644 --- a/x/feegrant/keeper/keeper.go +++ b/x/feegrant/keeper/keeper.go @@ -10,7 +10,7 @@ import ( "github.com/cosmos/cosmos-sdk/codec" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" + "github.com/cosmos/cosmos-sdk/x/auth/ante" "github.com/cosmos/cosmos-sdk/x/feegrant" ) @@ -22,7 +22,7 @@ type Keeper struct { authKeeper feegrant.AccountKeeper } -var _ middleware.FeegrantKeeper = &Keeper{} +var _ ante.FeegrantKeeper = &Keeper{} // NewKeeper creates a fee grant Keeper func NewKeeper(cdc codec.BinaryCodec, storeKey storetypes.StoreKey, ak feegrant.AccountKeeper) Keeper { diff --git a/x/gov/keeper/keeper.go b/x/gov/keeper/keeper.go index bb4491eeb329..74db3fa704a1 100644 --- a/x/gov/keeper/keeper.go +++ b/x/gov/keeper/keeper.go @@ -6,10 +6,10 @@ import ( "github.com/tendermint/tendermint/libs/log" + "github.com/cosmos/cosmos-sdk/baseapp" "github.com/cosmos/cosmos-sdk/codec" storetypes "github.com/cosmos/cosmos-sdk/store/types" sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/x/auth/middleware" authtypes "github.com/cosmos/cosmos-sdk/x/auth/types" "github.com/cosmos/cosmos-sdk/x/gov/types" v1 "github.com/cosmos/cosmos-sdk/x/gov/types/v1" @@ -40,7 +40,7 @@ type Keeper struct { legacyRouter v1beta1.Router // Msg server router - router *middleware.MsgServiceRouter + router *baseapp.MsgServiceRouter config types.Config } @@ -55,7 +55,7 @@ type Keeper struct { func NewKeeper( cdc codec.BinaryCodec, key storetypes.StoreKey, paramSpace types.ParamSubspace, authKeeper types.AccountKeeper, bankKeeper types.BankKeeper, sk types.StakingKeeper, - legacyRouter v1beta1.Router, router *middleware.MsgServiceRouter, + legacyRouter v1beta1.Router, router *baseapp.MsgServiceRouter, config types.Config, ) Keeper { @@ -104,7 +104,7 @@ func (keeper Keeper) Logger(ctx sdk.Context) log.Logger { } // Router returns the gov keeper's router -func (keeper Keeper) Router() *middleware.MsgServiceRouter { +func (keeper Keeper) Router() *baseapp.MsgServiceRouter { return keeper.router } diff --git a/x/group/keeper/keeper.go b/x/group/keeper/keeper.go index 55cdd9de64bb..2924854bcc32 100644 --- a/x/group/keeper/keeper.go +++ b/x/group/keeper/keeper.go @@ -6,11 +6,11 @@ import ( "github.com/tendermint/tendermint/libs/log" + "github.com/cosmos/cosmos-sdk/baseapp" "github.com/cosmos/cosmos-sdk/codec" storetypes "github.com/cosmos/cosmos-sdk/store/types" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - authmiddleware "github.com/cosmos/cosmos-sdk/x/auth/middleware" "github.com/cosmos/cosmos-sdk/x/group" "github.com/cosmos/cosmos-sdk/x/group/errors" "github.com/cosmos/cosmos-sdk/x/group/internal/orm" @@ -75,13 +75,13 @@ type Keeper struct { voteByProposalIndex orm.Index voteByVoterIndex orm.Index - router *authmiddleware.MsgServiceRouter + router *baseapp.MsgServiceRouter config group.Config } // NewKeeper creates a new group keeper. -func NewKeeper(storeKey storetypes.StoreKey, cdc codec.Codec, router *authmiddleware.MsgServiceRouter, accKeeper group.AccountKeeper, config group.Config) Keeper { +func NewKeeper(storeKey storetypes.StoreKey, cdc codec.Codec, router *baseapp.MsgServiceRouter, accKeeper group.AccountKeeper, config group.Config) Keeper { k := Keeper{ key: storeKey, router: router, diff --git a/x/group/keeper/proposal_executor.go b/x/group/keeper/proposal_executor.go index 506fa0075859..29d9d1e8bed9 100644 --- a/x/group/keeper/proposal_executor.go +++ b/x/group/keeper/proposal_executor.go @@ -3,16 +3,16 @@ package keeper import ( "fmt" + "github.com/cosmos/cosmos-sdk/baseapp" sdk "github.com/cosmos/cosmos-sdk/types" "github.com/cosmos/cosmos-sdk/types/errors" - authmiddleware "github.com/cosmos/cosmos-sdk/x/auth/middleware" "github.com/cosmos/cosmos-sdk/x/group" grouperrors "github.com/cosmos/cosmos-sdk/x/group/errors" ) // doExecuteMsgs routes the messages to the registered handlers. Messages are limited to those that require no authZ or // by the account of group policy only. Otherwise this gives access to other peoples accounts as the sdk middlewares are bypassed -func (s Keeper) doExecuteMsgs(ctx sdk.Context, router *authmiddleware.MsgServiceRouter, proposal group.Proposal, groupPolicyAcc sdk.AccAddress) ([]sdk.Result, error) { +func (s Keeper) doExecuteMsgs(ctx sdk.Context, router *baseapp.MsgServiceRouter, proposal group.Proposal, groupPolicyAcc sdk.AccAddress) ([]sdk.Result, error) { // Ensure it's not too late to execute the messages. // After https://github.com/cosmos/cosmos-sdk/issues/11245, proposals should // be pruned automatically, so this function should not even be called, as From abd3ca65c8f1b5ee4f54eb8e253b12ef5435c661 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Tue, 17 May 2022 16:34:58 +0200 Subject: [PATCH 03/29] Fix baseapp_test build --- baseapp/abci_test.go | 32 +-- baseapp/baseapp.go | 7 + baseapp/baseapp_test.go | 338 +++++++++++++++++++++-------- baseapp/custom_txhandler_test.go | 116 ---------- baseapp/grpcrouter_test.go | 105 +-------- baseapp/msg_service_router_test.go | 6 +- baseapp/options.go | 1 + baseapp/router_test.go | 4 +- 8 files changed, 285 insertions(+), 324 deletions(-) delete mode 100644 baseapp/custom_txhandler_test.go diff --git a/baseapp/abci_test.go b/baseapp/abci_test.go index b382a5a38909..ef306411b2c2 100644 --- a/baseapp/abci_test.go +++ b/baseapp/abci_test.go @@ -9,7 +9,10 @@ import ( tmprototypes "github.com/tendermint/tendermint/proto/tendermint/types" dbm "github.com/tendermint/tm-db" - sdk "github.com/cosmos/cosmos-sdk/types" + pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types" + "github.com/cosmos/cosmos-sdk/snapshots" + snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" + "github.com/cosmos/cosmos-sdk/testutil" ) func TestGetBlockRentionHeight(t *testing.T) { @@ -17,6 +20,9 @@ func TestGetBlockRentionHeight(t *testing.T) { db := dbm.NewMemDB() name := t.Name() + snapshotStore, err := snapshots.NewStore(dbm.NewMemDB(), testutil.GetTempDir(t)) + require.NoError(t, err) + testCases := map[string]struct { bapp *BaseApp maxAgeBlocks int64 @@ -38,18 +44,18 @@ func TestGetBlockRentionHeight(t *testing.T) { "pruning iavl snapshot only": { bapp: NewBaseApp( name, logger, db, nil, - SetPruning(sdk.PruningOptions{KeepEvery: 10000}), + SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)), SetMinRetainBlocks(1), + SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(10000, 1)), ), maxAgeBlocks: 0, commitHeight: 499000, - expected: 490000, + expected: 489000, }, "pruning state sync snapshot only": { bapp: NewBaseApp( name, logger, db, nil, - SetSnapshotInterval(50000), - SetSnapshotKeepRecent(3), + SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(50000, 3)), SetMinRetainBlocks(1), ), maxAgeBlocks: 0, @@ -68,9 +74,9 @@ func TestGetBlockRentionHeight(t *testing.T) { "pruning all conditions": { bapp: NewBaseApp( name, logger, db, nil, - SetPruning(sdk.PruningOptions{KeepEvery: 10000}), + SetPruning(pruningtypes.NewCustomPruningOptions(0, 0)), SetMinRetainBlocks(400000), - SetSnapshotInterval(50000), SetSnapshotKeepRecent(3), + SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(50000, 3)), ), maxAgeBlocks: 362880, commitHeight: 499000, @@ -79,9 +85,9 @@ func TestGetBlockRentionHeight(t *testing.T) { "no pruning due to no persisted state": { bapp: NewBaseApp( name, logger, db, nil, - SetPruning(sdk.PruningOptions{KeepEvery: 10000}), + SetPruning(pruningtypes.NewCustomPruningOptions(0, 0)), SetMinRetainBlocks(400000), - SetSnapshotInterval(50000), SetSnapshotKeepRecent(3), + SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(50000, 3)), ), maxAgeBlocks: 362880, commitHeight: 10000, @@ -90,9 +96,9 @@ func TestGetBlockRentionHeight(t *testing.T) { "disable pruning": { bapp: NewBaseApp( name, logger, db, nil, - SetPruning(sdk.PruningOptions{KeepEvery: 10000}), + SetPruning(pruningtypes.NewCustomPruningOptions(0, 0)), SetMinRetainBlocks(0), - SetSnapshotInterval(50000), SetSnapshotKeepRecent(3), + SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(50000, 3)), ), maxAgeBlocks: 362880, commitHeight: 499000, @@ -105,7 +111,7 @@ func TestGetBlockRentionHeight(t *testing.T) { tc.bapp.SetParamStore(¶mStore{db: dbm.NewMemDB()}) tc.bapp.InitChain(abci.RequestInitChain{ - ConsensusParams: &abci.ConsensusParams{ + ConsensusParams: &tmprototypes.ConsensusParams{ Evidence: &tmprototypes.EvidenceParams{ MaxAgeNumBlocks: tc.maxAgeBlocks, }, @@ -128,7 +134,7 @@ func TestBaseAppCreateQueryContext(t *testing.T) { logger := defaultLogger() db := dbm.NewMemDB() name := t.Name() - app := NewBaseApp(name, logger, db, nil) + app := NewBaseApp(name, logger, db, nil, nil) app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: 1}}) app.Commit() diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index f629626aa33a..23fe0448d8f3 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -440,6 +440,13 @@ func (app *BaseApp) GetConsensusParams(ctx sdk.Context) *tmproto.ConsensusParams return cp } +// AddRunTxRecoveryHandler adds custom app.runTx method panic handlers. +func (app *BaseApp) AddRunTxRecoveryHandler(handlers ...RecoveryHandler) { + for _, h := range handlers { + app.runTxRecoveryMiddleware = newRecoveryMiddleware(h, app.runTxRecoveryMiddleware) + } +} + // StoreConsensusParams sets the consensus parameters to the baseapp's param store. func (app *BaseApp) StoreConsensusParams(ctx sdk.Context, cp *tmproto.ConsensusParams) { if app.paramStore == nil { diff --git a/baseapp/baseapp_test.go b/baseapp/baseapp_test.go index f12557e8dc8c..651d47963ba5 100644 --- a/baseapp/baseapp_test.go +++ b/baseapp/baseapp_test.go @@ -5,9 +5,7 @@ import ( "encoding/binary" "encoding/json" "fmt" - "io/ioutil" "math/rand" - "os" "strings" "sync" "testing" @@ -22,10 +20,12 @@ import ( dbm "github.com/tendermint/tm-db" "github.com/cosmos/cosmos-sdk/codec" + pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types" "github.com/cosmos/cosmos-sdk/snapshots" snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" "github.com/cosmos/cosmos-sdk/store/rootmulti" - store "github.com/cosmos/cosmos-sdk/store/types" + storetypes "github.com/cosmos/cosmos-sdk/store/types" + "github.com/cosmos/cosmos-sdk/testutil" "github.com/cosmos/cosmos-sdk/testutil/testdata" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" @@ -41,6 +41,14 @@ type paramStore struct { db *dbm.MemDB } +type setupConfig struct { + blocks uint64 + blockTxs int + snapshotInterval uint64 + snapshotKeepRecent uint32 + pruningOpts pruningtypes.PruningOptions +} + func (ps *paramStore) Set(_ sdk.Context, key []byte, value interface{}) { bz, err := json.Marshal(value) if err != nil { @@ -121,7 +129,7 @@ func setupBaseApp(t *testing.T, options ...func(*BaseApp)) *BaseApp { } // simple one store baseapp with data and snapshots. Each tx is 1 MB in size (uncompressed). -func setupBaseAppWithSnapshots(t *testing.T, blocks uint, blockTxs int, options ...func(*BaseApp)) (*BaseApp, func()) { +func setupBaseAppWithSnapshots(t *testing.T, config *setupConfig) (*BaseApp, error) { codec := codec.NewLegacyAmino() registerTestCodec(codec) routerOpt := func(bapp *BaseApp) { @@ -132,29 +140,19 @@ func setupBaseAppWithSnapshots(t *testing.T, blocks uint, blockTxs int, options })) } - snapshotInterval := uint64(2) snapshotTimeout := 1 * time.Minute - snapshotDir, err := ioutil.TempDir("", "baseapp") + snapshotStore, err := snapshots.NewStore(dbm.NewMemDB(), testutil.GetTempDir(t)) require.NoError(t, err) - snapshotStore, err := snapshots.NewStore(dbm.NewMemDB(), snapshotDir) - require.NoError(t, err) - teardown := func() { - os.RemoveAll(snapshotDir) - } - app := setupBaseApp(t, append(options, - SetSnapshotStore(snapshotStore), - SetSnapshotInterval(snapshotInterval), - SetPruning(sdk.PruningOptions{KeepEvery: 1}), - routerOpt)...) + app := setupBaseApp(t, routerOpt, SetSnapshot(snapshotStore, snapshottypes.NewSnapshotOptions(config.snapshotInterval, uint32(config.snapshotKeepRecent))), SetPruning(config.pruningOpts)) app.InitChain(abci.RequestInitChain{}) r := rand.New(rand.NewSource(3920758213583)) keyCounter := 0 - for height := int64(1); height <= int64(blocks); height++ { + for height := int64(1); height <= int64(config.blocks); height++ { app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: height}}) - for txNum := 0; txNum < blockTxs; txNum++ { + for txNum := 0; txNum < config.blockTxs; txNum++ { tx := txTest{Msgs: []sdk.Msg{}} for msgNum := 0; msgNum < 100; msgNum++ { key := []byte(fmt.Sprintf("%v", keyCounter)) @@ -173,7 +171,7 @@ func setupBaseAppWithSnapshots(t *testing.T, blocks uint, blockTxs int, options app.Commit() // Wait for snapshot to be taken, since it happens asynchronously. - if uint64(height)%snapshotInterval == 0 { + if config.snapshotInterval > 0 && uint64(height)%config.snapshotInterval == 0 { start := time.Now() for { if time.Since(start) > snapshotTimeout { @@ -189,7 +187,7 @@ func setupBaseAppWithSnapshots(t *testing.T, blocks uint, blockTxs int, options } } - return app, teardown + return app, nil } func TestMountStores(t *testing.T) { @@ -206,7 +204,7 @@ func TestMountStores(t *testing.T) { // Test that LoadLatestVersion actually does. func TestLoadVersion(t *testing.T) { logger := defaultLogger() - pruningOpt := SetPruning(store.PruneNothing) + pruningOpt := SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)) db := dbm.NewMemDB() name := t.Name() app := NewBaseApp(name, logger, db, nil, pruningOpt) @@ -215,7 +213,7 @@ func TestLoadVersion(t *testing.T) { err := app.LoadLatestVersion() // needed to make stores non-nil require.Nil(t, err) - emptyCommitID := sdk.CommitID{} + emptyCommitID := storetypes.CommitID{} // fresh store has zero/empty last commit lastHeight := app.LastBlockHeight() @@ -227,13 +225,13 @@ func TestLoadVersion(t *testing.T) { header := tmproto.Header{Height: 1} app.BeginBlock(abci.RequestBeginBlock{Header: header}) res := app.Commit() - commitID1 := sdk.CommitID{Version: 1, Hash: res.Data} + commitID1 := storetypes.CommitID{Version: 1, Hash: res.Data} // execute a block, collect commit ID header = tmproto.Header{Height: 2} app.BeginBlock(abci.RequestBeginBlock{Header: header}) res = app.Commit() - commitID2 := sdk.CommitID{Version: 2, Hash: res.Data} + commitID2 := storetypes.CommitID{Version: 2, Hash: res.Data} // reload with LoadLatestVersion app = NewBaseApp(name, logger, db, nil, pruningOpt) @@ -258,16 +256,16 @@ func useDefaultLoader(app *BaseApp) { } func initStore(t *testing.T, db dbm.DB, storeKey string, k, v []byte) { - rs := rootmulti.NewStore(db) - rs.SetPruning(store.PruneNothing) + rs := rootmulti.NewStore(db, log.NewNopLogger()) + rs.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)) key := sdk.NewKVStoreKey(storeKey) - rs.MountStoreWithDB(key, store.StoreTypeIAVL, nil) + rs.MountStoreWithDB(key, storetypes.StoreTypeIAVL, nil) err := rs.LoadLatestVersion() require.Nil(t, err) require.Equal(t, int64(0), rs.LastCommitID().Version) // write some data in substore - kv, _ := rs.GetStore(key).(store.KVStore) + kv, _ := rs.GetStore(key).(storetypes.KVStore) require.NotNil(t, kv) kv.Set(k, v) commitID := rs.Commit() @@ -275,16 +273,16 @@ func initStore(t *testing.T, db dbm.DB, storeKey string, k, v []byte) { } func checkStore(t *testing.T, db dbm.DB, ver int64, storeKey string, k, v []byte) { - rs := rootmulti.NewStore(db) - rs.SetPruning(store.PruneDefault) + rs := rootmulti.NewStore(db, log.NewNopLogger()) + rs.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningDefault)) key := sdk.NewKVStoreKey(storeKey) - rs.MountStoreWithDB(key, store.StoreTypeIAVL, nil) + rs.MountStoreWithDB(key, storetypes.StoreTypeIAVL, nil) err := rs.LoadLatestVersion() require.Nil(t, err) require.Equal(t, ver, rs.LastCommitID().Version) // query data in substore - kv, _ := rs.GetStore(key).(store.KVStore) + kv, _ := rs.GetStore(key).(storetypes.KVStore) require.NotNil(t, kv) require.Equal(t, v, kv.Get(k)) } @@ -319,7 +317,7 @@ func TestSetLoader(t *testing.T) { initStore(t, db, tc.origStoreKey, k, v) // load the app with the existing db - opts := []func(*BaseApp){SetPruning(store.PruneNothing)} + opts := []func(*BaseApp){SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))} if tc.setLoader != nil { opts = append(opts, tc.setLoader) } @@ -342,7 +340,7 @@ func TestSetLoader(t *testing.T) { func TestVersionSetterGetter(t *testing.T) { logger := defaultLogger() - pruningOpt := SetPruning(store.PruneDefault) + pruningOpt := SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningDefault)) db := dbm.NewMemDB() name := t.Name() app := NewBaseApp(name, logger, db, nil, pruningOpt) @@ -362,7 +360,7 @@ func TestVersionSetterGetter(t *testing.T) { func TestLoadVersionInvalid(t *testing.T) { logger := log.NewNopLogger() - pruningOpt := SetPruning(store.PruneNothing) + pruningOpt := SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing)) db := dbm.NewMemDB() name := t.Name() app := NewBaseApp(name, logger, db, nil, pruningOpt) @@ -377,7 +375,7 @@ func TestLoadVersionInvalid(t *testing.T) { header := tmproto.Header{Height: 1} app.BeginBlock(abci.RequestBeginBlock{Header: header}) res := app.Commit() - commitID1 := sdk.CommitID{Version: 1, Hash: res.Data} + commitID1 := storetypes.CommitID{Version: 1, Hash: res.Data} // create a new app with the stores mounted under the same cap key app = NewBaseApp(name, logger, db, nil, pruningOpt) @@ -394,11 +392,7 @@ func TestLoadVersionInvalid(t *testing.T) { func TestLoadVersionPruning(t *testing.T) { logger := log.NewNopLogger() - pruningOptions := store.PruningOptions{ - KeepRecent: 2, - KeepEvery: 3, - Interval: 1, - } + pruningOptions := pruningtypes.NewCustomPruningOptions(10, 15) pruningOpt := SetPruning(pruningOptions) db := dbm.NewMemDB() name := t.Name() @@ -411,7 +405,7 @@ func TestLoadVersionPruning(t *testing.T) { err := app.LoadLatestVersion() // needed to make stores non-nil require.Nil(t, err) - emptyCommitID := sdk.CommitID{} + emptyCommitID := storetypes.CommitID{} // fresh store has zero/empty last commit lastHeight := app.LastBlockHeight() @@ -419,14 +413,14 @@ func TestLoadVersionPruning(t *testing.T) { require.Equal(t, int64(0), lastHeight) require.Equal(t, emptyCommitID, lastID) - var lastCommitID sdk.CommitID + var lastCommitID storetypes.CommitID // Commit seven blocks, of which 7 (latest) is kept in addition to 6, 5 // (keep recent) and 3 (keep every). for i := int64(1); i <= 7; i++ { app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: i}}) res := app.Commit() - lastCommitID = sdk.CommitID{Version: i, Hash: res.Data} + lastCommitID = storetypes.CommitID{Version: i, Hash: res.Data} } for _, v := range []int64{1, 2, 4} { @@ -448,7 +442,7 @@ func TestLoadVersionPruning(t *testing.T) { testLoadVersionHelper(t, app, int64(7), lastCommitID) } -func testLoadVersionHelper(t *testing.T, app *BaseApp, expectedHeight int64, expectedID sdk.CommitID) { +func testLoadVersionHelper(t *testing.T, app *BaseApp, expectedHeight int64, expectedID storetypes.CommitID) { lastHeight := app.LastBlockHeight() lastID := app.LastCommitID() require.Equal(t, expectedHeight, lastHeight) @@ -540,7 +534,7 @@ func TestBaseAppOptionSeal(t *testing.T) { app.SetFauxMerkleMode() }) require.Panics(t, func() { - app.SetRouter(NewRouter()) + app.SetRouter(NewLegacyRouter()) }) } @@ -813,7 +807,7 @@ func testTxDecoder(cdc *codec.LegacyAmino) sdk.TxDecoder { } } -func anteHandlerTxTest(t *testing.T, capKey sdk.StoreKey, storeKey []byte) sdk.AnteHandler { +func anteHandlerTxTest(t *testing.T, capKey storetypes.StoreKey, storeKey []byte) sdk.AnteHandler { return func(ctx sdk.Context, tx sdk.Tx, simulate bool) (sdk.Context, error) { store := ctx.KVStore(capKey) txTest := tx.(txTest) @@ -844,7 +838,7 @@ func counterEvent(evType string, msgCount int64) sdk.Events { } } -func handlerMsgCounter(t *testing.T, capKey sdk.StoreKey, deliverKey []byte) sdk.Handler { +func handlerMsgCounter(t *testing.T, capKey storetypes.StoreKey, deliverKey []byte) sdk.Handler { return func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { ctx = ctx.WithEventManager(sdk.NewEventManager()) store := ctx.KVStore(capKey) @@ -1184,7 +1178,7 @@ func TestRunInvalidTransaction(t *testing.T) { // transaction with no messages { emptyTx := &txTest{} - _, result, err := app.Deliver(aminoTxEncoder(), emptyTx) + _, result, err := app.SimDeliver(aminoTxEncoder(), emptyTx) require.Error(t, err) require.Nil(t, result) @@ -1211,7 +1205,7 @@ func TestRunInvalidTransaction(t *testing.T) { for _, testCase := range testCases { tx := testCase.tx - _, result, err := app.Deliver(aminoTxEncoder(), tx) + _, result, err := app.SimDeliver(aminoTxEncoder(), tx) if testCase.fail { require.Error(t, err) @@ -1228,7 +1222,7 @@ func TestRunInvalidTransaction(t *testing.T) { // transaction with no known route { unknownRouteTx := txTest{[]sdk.Msg{msgNoRoute{}}, 0, false} - _, result, err := app.Deliver(aminoTxEncoder(), unknownRouteTx) + _, result, err := app.SimDeliver(aminoTxEncoder(), unknownRouteTx) require.Error(t, err) require.Nil(t, result) @@ -1237,7 +1231,7 @@ func TestRunInvalidTransaction(t *testing.T) { require.EqualValues(t, sdkerrors.ErrUnknownRequest.ABCICode(), code, err) unknownRouteTx = txTest{[]sdk.Msg{msgCounter{}, msgNoRoute{}}, 0, false} - _, result, err = app.Deliver(aminoTxEncoder(), unknownRouteTx) + _, result, err = app.SimDeliver(aminoTxEncoder(), unknownRouteTx) require.Error(t, err) require.Nil(t, result) @@ -1335,7 +1329,7 @@ func TestTxGasLimits(t *testing.T) { for i, tc := range testCases { tx := tc.tx - gInfo, result, err := app.Deliver(aminoTxEncoder(), tx) + gInfo, result, err := app.SimDeliver(aminoTxEncoder(), tx) // check gas used and wanted require.Equal(t, tc.gasUsed, gInfo.GasUsed, fmt.Sprintf("tc #%d; gas: %v, result: %v, err: %s", i, gInfo, result, err)) @@ -1390,8 +1384,8 @@ func TestMaxBlockGasLimits(t *testing.T) { app := setupBaseApp(t, anteOpt, routerOpt) app.InitChain(abci.RequestInitChain{ - ConsensusParams: &abci.ConsensusParams{ - Block: &abci.BlockParams{ + ConsensusParams: &tmproto.ConsensusParams{ + Block: &tmproto.BlockParams{ MaxGas: 100, }, }, @@ -1425,7 +1419,7 @@ func TestMaxBlockGasLimits(t *testing.T) { // execute the transaction multiple times for j := 0; j < tc.numDelivers; j++ { - _, result, err := app.Deliver(aminoTxEncoder(), tx) + _, result, err := app.SimDeliver(aminoTxEncoder(), tx) ctx := app.getState(runTxModeDeliver).ctx @@ -1493,7 +1487,7 @@ func TestCustomRunTxPanicHandler(t *testing.T) { { tx := newTxCounter(0, 0) - require.PanicsWithValue(t, customPanicMsg, func() { app.Deliver(aminoTxEncoder(), tx) }) + require.PanicsWithValue(t, customPanicMsg, func() { app.SimDeliver(aminoTxEncoder(), tx) }) } } @@ -1615,8 +1609,8 @@ func TestGasConsumptionBadTx(t *testing.T) { app := setupBaseApp(t, anteOpt, routerOpt) app.InitChain(abci.RequestInitChain{ - ConsensusParams: &abci.ConsensusParams{ - Block: &abci.BlockParams{ + ConsensusParams: &tmproto.ConsensusParams{ + Block: &tmproto.BlockParams{ MaxGas: 9, }, }, @@ -1682,7 +1676,7 @@ func TestQuery(t *testing.T) { require.Equal(t, 0, len(res.Value)) // query is still empty after a CheckTx - _, resTx, err := app.Check(aminoTxEncoder(), tx) + _, resTx, err := app.SimCheck(aminoTxEncoder(), tx) require.NoError(t, err) require.NotNil(t, resTx) res = app.Query(query) @@ -1692,7 +1686,7 @@ func TestQuery(t *testing.T) { header := tmproto.Header{Height: app.LastBlockHeight() + 1} app.BeginBlock(abci.RequestBeginBlock{Header: header}) - _, resTx, err = app.Deliver(aminoTxEncoder(), tx) + _, resTx, err = app.SimDeliver(aminoTxEncoder(), tx) require.NoError(t, err) require.NotNil(t, resTx) res = app.Query(query) @@ -1774,22 +1768,30 @@ func TestGetMaximumBlockGas(t *testing.T) { app.InitChain(abci.RequestInitChain{}) ctx := app.NewContext(true, tmproto.Header{}) - app.StoreConsensusParams(ctx, &abci.ConsensusParams{Block: &abci.BlockParams{MaxGas: 0}}) + app.StoreConsensusParams(ctx, &tmproto.ConsensusParams{Block: &tmproto.BlockParams{MaxGas: 0}}) require.Equal(t, uint64(0), app.getMaximumBlockGas(ctx)) - app.StoreConsensusParams(ctx, &abci.ConsensusParams{Block: &abci.BlockParams{MaxGas: -1}}) + app.StoreConsensusParams(ctx, &tmproto.ConsensusParams{Block: &tmproto.BlockParams{MaxGas: -1}}) require.Equal(t, uint64(0), app.getMaximumBlockGas(ctx)) - app.StoreConsensusParams(ctx, &abci.ConsensusParams{Block: &abci.BlockParams{MaxGas: 5000000}}) + app.StoreConsensusParams(ctx, &tmproto.ConsensusParams{Block: &tmproto.BlockParams{MaxGas: 5000000}}) require.Equal(t, uint64(5000000), app.getMaximumBlockGas(ctx)) - app.StoreConsensusParams(ctx, &abci.ConsensusParams{Block: &abci.BlockParams{MaxGas: -5000000}}) + app.StoreConsensusParams(ctx, &tmproto.ConsensusParams{Block: &tmproto.BlockParams{MaxGas: -5000000}}) require.Panics(t, func() { app.getMaximumBlockGas(ctx) }) } func TestListSnapshots(t *testing.T) { - app, teardown := setupBaseAppWithSnapshots(t, 5, 4) - defer teardown() + setupConfig := &setupConfig{ + blocks: 5, + blockTxs: 4, + snapshotInterval: 2, + snapshotKeepRecent: 2, + pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), + } + + app, err := setupBaseAppWithSnapshots(t, setupConfig) + require.NoError(t, err) resp := app.ListSnapshots(abci.RequestListSnapshots{}) for _, s := range resp.Snapshots { @@ -1799,14 +1801,153 @@ func TestListSnapshots(t *testing.T) { s.Metadata = nil } assert.Equal(t, abci.ResponseListSnapshots{Snapshots: []*abci.Snapshot{ - {Height: 4, Format: 1, Chunks: 2}, - {Height: 2, Format: 1, Chunks: 1}, + {Height: 4, Format: snapshottypes.CurrentFormat, Chunks: 2}, + {Height: 2, Format: snapshottypes.CurrentFormat, Chunks: 1}, }}, resp) } +func TestSnapshotWithPruning(t *testing.T) { + testcases := map[string]struct { + config *setupConfig + expectedSnapshots []*abci.Snapshot + expectedErr error + }{ + "prune nothing with snapshot": { + config: &setupConfig{ + blocks: 20, + blockTxs: 2, + snapshotInterval: 5, + snapshotKeepRecent: 1, + pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), + }, + expectedSnapshots: []*abci.Snapshot{ + {Height: 20, Format: 2, Chunks: 5}, + }, + }, + "prune everything with snapshot": { + config: &setupConfig{ + blocks: 20, + blockTxs: 2, + snapshotInterval: 5, + snapshotKeepRecent: 1, + pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningEverything), + }, + expectedSnapshots: []*abci.Snapshot{ + {Height: 20, Format: 2, Chunks: 5}, + }, + }, + "default pruning with snapshot": { + config: &setupConfig{ + blocks: 20, + blockTxs: 2, + snapshotInterval: 5, + snapshotKeepRecent: 1, + pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningDefault), + }, + expectedSnapshots: []*abci.Snapshot{ + {Height: 20, Format: 2, Chunks: 5}, + }, + }, + "custom": { + config: &setupConfig{ + blocks: 25, + blockTxs: 2, + snapshotInterval: 5, + snapshotKeepRecent: 2, + pruningOpts: pruningtypes.NewCustomPruningOptions(12, 12), + }, + expectedSnapshots: []*abci.Snapshot{ + {Height: 25, Format: 2, Chunks: 6}, + {Height: 20, Format: 2, Chunks: 5}, + }, + }, + "no snapshots": { + config: &setupConfig{ + blocks: 10, + blockTxs: 2, + snapshotInterval: 0, // 0 implies disable snapshots + pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), + }, + expectedSnapshots: []*abci.Snapshot{}, + }, + "keep all snapshots": { + config: &setupConfig{ + blocks: 10, + blockTxs: 2, + snapshotInterval: 3, + snapshotKeepRecent: 0, // 0 implies keep all snapshots + pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), + }, + expectedSnapshots: []*abci.Snapshot{ + {Height: 9, Format: 2, Chunks: 2}, + {Height: 6, Format: 2, Chunks: 2}, + {Height: 3, Format: 2, Chunks: 1}, + }, + }, + } + + for name, tc := range testcases { + t.Run(name, func(t *testing.T) { + app, err := setupBaseAppWithSnapshots(t, tc.config) + + if tc.expectedErr != nil { + require.Error(t, err) + require.Equal(t, tc.expectedErr.Error(), err.Error()) + return + } + require.NoError(t, err) + + resp := app.ListSnapshots(abci.RequestListSnapshots{}) + for _, s := range resp.Snapshots { + assert.NotEmpty(t, s.Hash) + assert.NotEmpty(t, s.Metadata) + s.Hash = nil + s.Metadata = nil + } + fmt.Println(resp) + assert.Equal(t, abci.ResponseListSnapshots{Snapshots: tc.expectedSnapshots}, resp) + + // Validate that heights were pruned correctly by querying the state at the last height that should be present relative to latest + // and the first height that should be pruned. + // + // Exceptions: + // * Prune nothing: should be able to query all heights (we only test first and latest) + // * Prune default: should be able to query all heights (we only test first and latest) + // * The reason for default behaving this way is that we only commit 20 heights but default has 100_000 keep-recent + var lastExistingHeight int64 + if tc.config.pruningOpts.GetPruningStrategy() == pruningtypes.PruningNothing || tc.config.pruningOpts.GetPruningStrategy() == pruningtypes.PruningDefault { + lastExistingHeight = 1 + } else { + // Integer division rounds down so by multiplying back we get the last height at which we pruned + lastExistingHeight = int64((tc.config.blocks/tc.config.pruningOpts.Interval)*tc.config.pruningOpts.Interval - tc.config.pruningOpts.KeepRecent) + } + + // Query 1 + res := app.Query(abci.RequestQuery{Path: fmt.Sprintf("/store/%s/key", capKey2.Name()), Data: []byte("0"), Height: lastExistingHeight}) + require.NotNil(t, res, "height: %d", lastExistingHeight) + require.NotNil(t, res.Value, "height: %d", lastExistingHeight) + + // Query 2 + res = app.Query(abci.RequestQuery{Path: fmt.Sprintf("/store/%s/key", capKey2.Name()), Data: []byte("0"), Height: lastExistingHeight - 1}) + require.NotNil(t, res, "height: %d", lastExistingHeight-1) + if tc.config.pruningOpts.GetPruningStrategy() == pruningtypes.PruningNothing || tc.config.pruningOpts.GetPruningStrategy() == pruningtypes.PruningDefault { + // With prune nothing or default, we query height 0 which translates to the latest height. + require.NotNil(t, res.Value, "height: %d", lastExistingHeight-1) + } + }) + } +} + func TestLoadSnapshotChunk(t *testing.T) { - app, teardown := setupBaseAppWithSnapshots(t, 2, 5) - defer teardown() + setupConfig := &setupConfig{ + blocks: 2, + blockTxs: 5, + snapshotInterval: 2, + snapshotKeepRecent: 2, + pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), + } + app, err := setupBaseAppWithSnapshots(t, setupConfig) + require.NoError(t, err) testcases := map[string]struct { height uint64 @@ -1814,13 +1955,13 @@ func TestLoadSnapshotChunk(t *testing.T) { chunk uint32 expectEmpty bool }{ - "Existing snapshot": {2, 1, 1, false}, - "Missing height": {100, 1, 1, true}, - "Missing format": {2, 2, 1, true}, - "Missing chunk": {2, 1, 9, true}, - "Zero height": {0, 1, 1, true}, + "Existing snapshot": {2, snapshottypes.CurrentFormat, 1, false}, + "Missing height": {100, snapshottypes.CurrentFormat, 1, true}, + "Missing format": {2, 3, 1, true}, + "Missing chunk": {2, snapshottypes.CurrentFormat, 9, true}, + "Zero height": {0, snapshottypes.CurrentFormat, 1, true}, "Zero format": {2, 0, 1, true}, - "Zero chunk": {2, 1, 0, false}, + "Zero chunk": {2, snapshottypes.CurrentFormat, 0, false}, } for name, tc := range testcases { tc := tc @@ -1841,8 +1982,15 @@ func TestLoadSnapshotChunk(t *testing.T) { func TestOfferSnapshot_Errors(t *testing.T) { // Set up app before test cases, since it's fairly expensive. - app, teardown := setupBaseAppWithSnapshots(t, 0, 0) - defer teardown() + setupConfig := &setupConfig{ + blocks: 0, + blockTxs: 0, + snapshotInterval: 2, + snapshotKeepRecent: 2, + pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), + } + app, err := setupBaseAppWithSnapshots(t, setupConfig) + require.NoError(t, err) m := snapshottypes.Metadata{ChunkHashes: [][]byte{{1}, {2}, {3}}} metadata, err := m.Marshal() @@ -1858,13 +2006,13 @@ func TestOfferSnapshot_Errors(t *testing.T) { Height: 1, Format: 9, Chunks: 3, Hash: hash, Metadata: metadata, }, abci.ResponseOfferSnapshot_REJECT_FORMAT}, "incorrect chunk count": {&abci.Snapshot{ - Height: 1, Format: 1, Chunks: 2, Hash: hash, Metadata: metadata, + Height: 1, Format: snapshottypes.CurrentFormat, Chunks: 2, Hash: hash, Metadata: metadata, }, abci.ResponseOfferSnapshot_REJECT}, "no chunks": {&abci.Snapshot{ - Height: 1, Format: 1, Chunks: 0, Hash: hash, Metadata: metadata, + Height: 1, Format: snapshottypes.CurrentFormat, Chunks: 0, Hash: hash, Metadata: metadata, }, abci.ResponseOfferSnapshot_REJECT}, "invalid metadata serialization": {&abci.Snapshot{ - Height: 1, Format: 1, Chunks: 0, Hash: hash, Metadata: []byte{3, 1, 4}, + Height: 1, Format: snapshottypes.CurrentFormat, Chunks: 0, Hash: hash, Metadata: []byte{3, 1, 4}, }, abci.ResponseOfferSnapshot_REJECT}, } for name, tc := range testcases { @@ -1896,11 +2044,25 @@ func TestOfferSnapshot_Errors(t *testing.T) { } func TestApplySnapshotChunk(t *testing.T) { - source, teardown := setupBaseAppWithSnapshots(t, 4, 10) - defer teardown() + setupConfig1 := &setupConfig{ + blocks: 4, + blockTxs: 10, + snapshotInterval: 2, + snapshotKeepRecent: 2, + pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), + } + source, err := setupBaseAppWithSnapshots(t, setupConfig1) + require.NoError(t, err) - target, teardown := setupBaseAppWithSnapshots(t, 0, 0) - defer teardown() + setupConfig2 := &setupConfig{ + blocks: 0, + blockTxs: 0, + snapshotInterval: 2, + snapshotKeepRecent: 2, + pruningOpts: pruningtypes.NewPruningOptions(pruningtypes.PruningNothing), + } + target, err := setupBaseAppWithSnapshots(t, setupConfig2) + require.NoError(t, err) // Fetch latest snapshot to restore respList := source.ListSnapshots(abci.RequestListSnapshots{}) @@ -2014,8 +2176,8 @@ func TestBaseApp_EndBlock(t *testing.T) { name := t.Name() logger := defaultLogger() - cp := &abci.ConsensusParams{ - Block: &abci.BlockParams{ + cp := &tmproto.ConsensusParams{ + Block: &tmproto.BlockParams{ MaxGas: 5000000, }, } diff --git a/baseapp/custom_txhandler_test.go b/baseapp/custom_txhandler_test.go deleted file mode 100644 index 78e4c8befe55..000000000000 --- a/baseapp/custom_txhandler_test.go +++ /dev/null @@ -1,116 +0,0 @@ -package baseapp_test - -import ( - "context" - "fmt" - - "github.com/tendermint/tendermint/crypto/tmhash" - - sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/types/tx" -) - -type handlerFun func(ctx sdk.Context, tx sdk.Tx, simulate bool) (newCtx sdk.Context, err error) - -type customTxHandler struct { - handler handlerFun - next tx.Handler -} - -var _ tx.Handler = customTxHandler{} - -// CustomTxMiddleware is being used in tests for testing custom pre-`runMsgs` logic. -func CustomTxHandlerMiddleware(handler handlerFun) tx.Middleware { - return func(txHandler tx.Handler) tx.Handler { - return customTxHandler{ - handler: handler, - next: txHandler, - } - } -} - -// CheckTx implements tx.Handler.CheckTx method. -func (txh customTxHandler) CheckTx(ctx context.Context, req tx.Request, checkReq tx.RequestCheckTx) (tx.Response, tx.ResponseCheckTx, error) { - sdkCtx, err := txh.runHandler(ctx, req.Tx, req.TxBytes, false) - if err != nil { - return tx.Response{}, tx.ResponseCheckTx{}, err - } - - return txh.next.CheckTx(sdk.WrapSDKContext(sdkCtx), req, checkReq) -} - -// DeliverTx implements tx.Handler.DeliverTx method. -func (txh customTxHandler) DeliverTx(ctx context.Context, req tx.Request) (tx.Response, error) { - sdkCtx, err := txh.runHandler(ctx, req.Tx, req.TxBytes, false) - if err != nil { - return tx.Response{}, err - } - - return txh.next.DeliverTx(sdk.WrapSDKContext(sdkCtx), req) -} - -// SimulateTx implements tx.Handler.SimulateTx method. -func (txh customTxHandler) SimulateTx(ctx context.Context, req tx.Request) (tx.Response, error) { - sdkCtx, err := txh.runHandler(ctx, req.Tx, req.TxBytes, true) - if err != nil { - return tx.Response{}, err - } - - return txh.next.SimulateTx(sdk.WrapSDKContext(sdkCtx), req) -} - -func (txh customTxHandler) runHandler(ctx context.Context, tx sdk.Tx, txBytes []byte, isSimulate bool) (sdk.Context, error) { - sdkCtx := sdk.UnwrapSDKContext(ctx) - if txh.handler == nil { - return sdkCtx, nil - } - - ms := sdkCtx.MultiStore() - - // Branch context before Handler call in case it aborts. - // This is required for both CheckTx and DeliverTx. - // Ref: https://github.com/cosmos/cosmos-sdk/issues/2772 - // - // NOTE: Alternatively, we could require that Handler ensures that - // writes do not happen if aborted/failed. This may have some - // performance benefits, but it'll be more difficult to get right. - cacheCtx, msCache := cacheTxContext(sdkCtx, txBytes) - cacheCtx = cacheCtx.WithEventManager(sdk.NewEventManager()) - newCtx, err := txh.handler(cacheCtx, tx, isSimulate) - if err != nil { - return sdk.Context{}, err - } - - if !newCtx.IsZero() { - // At this point, newCtx.MultiStore() is a store branch, or something else - // replaced by the Handler. We want the original multistore. - // - // Also, in the case of the tx aborting, we need to track gas consumed via - // the instantiated gas meter in the Handler, so we update the context - // prior to returning. - sdkCtx = newCtx.WithMultiStore(ms) - } - - msCache.Write() - - return sdkCtx, nil -} - -// cacheTxContext returns a new context based off of the provided context with -// a branched multi-store. -func cacheTxContext(sdkCtx sdk.Context, txBytes []byte) (sdk.Context, sdk.CacheMultiStore) { - ms := sdkCtx.MultiStore() - // TODO: https://github.com/cosmos/cosmos-sdk/issues/2824 - msCache := ms.CacheMultiStore() - if msCache.TracingEnabled() { - msCache = msCache.SetTracingContext( - sdk.TraceContext( - map[string]interface{}{ - "txHash": fmt.Sprintf("%X", tmhash.Sum(txBytes)), - }, - ), - ).(sdk.CacheMultiStore) - } - - return sdkCtx.WithMultiStore(msCache), msCache -} diff --git a/baseapp/grpcrouter_test.go b/baseapp/grpcrouter_test.go index 11151f0e7210..a05d3d6616f8 100644 --- a/baseapp/grpcrouter_test.go +++ b/baseapp/grpcrouter_test.go @@ -2,11 +2,8 @@ package baseapp_test import ( "context" - "sync" "testing" - "github.com/cosmos/cosmos-sdk/testutil/testdata_pulsar" - "github.com/stretchr/testify/require" "github.com/tendermint/tendermint/libs/log" dbm "github.com/tendermint/tm-db" @@ -18,11 +15,11 @@ import ( sdk "github.com/cosmos/cosmos-sdk/types" ) -func TestGRPCQueryRouter(t *testing.T) { +func TestGRPCGatewayRouter(t *testing.T) { qr := baseapp.NewGRPCQueryRouter() interfaceRegistry := testdata.NewTestInterfaceRegistry() qr.SetInterfaceRegistry(interfaceRegistry) - testdata_pulsar.RegisterQueryServer(qr, testdata_pulsar.QueryImpl{}) + testdata.RegisterQueryServer(qr, testdata.QueryImpl{}) helper := &baseapp.QueryServiceTestHelper{ GRPCQueryRouter: qr, Ctx: sdk.Context{}.WithContext(context.Background()), @@ -56,8 +53,7 @@ func TestRegisterQueryServiceTwice(t *testing.T) { // Setup baseapp. db := dbm.NewMemDB() encCfg := simapp.MakeTestEncodingConfig() - logger, _ := log.NewDefaultLogger("plain", "info", false) - app := baseapp.NewBaseApp("test", logger, db) + app := baseapp.NewBaseApp("test", log.MustNewDefaultLogger("plain", "info", false), db, encCfg.TxConfig.TxDecoder()) app.SetInterfaceRegistry(encCfg.InterfaceRegistry) testdata.RegisterInterfaces(encCfg.InterfaceRegistry) @@ -77,98 +73,3 @@ func TestRegisterQueryServiceTwice(t *testing.T) { ) }) } - -// Tests that we don't have data races per -// https://github.com/cosmos/cosmos-sdk/issues/10324 -// but with the same client connection being used concurrently. -func TestQueryDataRaces_sameConnectionToSameHandler(t *testing.T) { - var mu sync.Mutex - var helper *baseapp.QueryServiceTestHelper - makeClientConn := func(qr *baseapp.GRPCQueryRouter) *baseapp.QueryServiceTestHelper { - mu.Lock() - defer mu.Unlock() - - if helper == nil { - helper = &baseapp.QueryServiceTestHelper{ - GRPCQueryRouter: qr, - Ctx: sdk.Context{}.WithContext(context.Background()), - } - } - return helper - } - testQueryDataRacesSameHandler(t, makeClientConn) -} - -// Tests that we don't have data races per -// https://github.com/cosmos/cosmos-sdk/issues/10324 -// but with unique client connections requesting from the same handler concurrently. -func TestQueryDataRaces_uniqueConnectionsToSameHandler(t *testing.T) { - // Return a new handler for every single call. - testQueryDataRacesSameHandler(t, func(qr *baseapp.GRPCQueryRouter) *baseapp.QueryServiceTestHelper { - return &baseapp.QueryServiceTestHelper{ - GRPCQueryRouter: qr, - Ctx: sdk.Context{}.WithContext(context.Background()), - } - }) -} - -func testQueryDataRacesSameHandler(t *testing.T, makeClientConn func(*baseapp.GRPCQueryRouter) *baseapp.QueryServiceTestHelper) { - t.Parallel() - - qr := baseapp.NewGRPCQueryRouter() - interfaceRegistry := testdata.NewTestInterfaceRegistry() - qr.SetInterfaceRegistry(interfaceRegistry) - testdata.RegisterQueryServer(qr, testdata.QueryImpl{}) - - // The goal is to invoke the router concurrently and check for any data races. - // 0. Run with: go test -race - // 1. Synchronize every one of the 1,000 goroutines waiting to all query at the - // same time. - // 2. Once the greenlight is given, perform a query through the router. - var wg sync.WaitGroup - defer wg.Wait() - - greenlight := make(chan bool) - n := 1000 - ready := make(chan bool, n) - go func() { - for i := 0; i < n; i++ { - <-ready - } - close(greenlight) - }() - - for i := 0; i < n; i++ { - wg.Add(1) - go func() { - defer wg.Done() - - // Wait until we get the green light to start. - ready <- true - <-greenlight - - client := testdata.NewQueryClient(makeClientConn(qr)) - res, err := client.Echo(context.Background(), &testdata.EchoRequest{Message: "hello"}) - require.Nil(t, err) - require.NotNil(t, res) - require.Equal(t, "hello", res.Message) - - require.Panics(t, func() { - _, _ = client.Echo(context.Background(), nil) - }) - - res2, err := client.SayHello(context.Background(), &testdata.SayHelloRequest{Name: "Foo"}) - require.Nil(t, err) - require.NotNil(t, res) - require.Equal(t, "Hello Foo!", res2.Greeting) - - spot := &testdata.Dog{Name: "Spot", Size_: "big"} - any, err := types.NewAnyWithValue(spot) - require.NoError(t, err) - res3, err := client.TestAny(context.Background(), &testdata.TestAnyRequest{AnyAnimal: any}) - require.NoError(t, err) - require.NotNil(t, res3) - require.Equal(t, spot, res3.HasAnimal.Animal.GetCachedValue()) - }() - } -} diff --git a/baseapp/msg_service_router_test.go b/baseapp/msg_service_router_test.go index 471fa17d9ea6..78683d35114e 100644 --- a/baseapp/msg_service_router_test.go +++ b/baseapp/msg_service_router_test.go @@ -22,7 +22,7 @@ func TestRegisterMsgService(t *testing.T) { // Create an encoding config that doesn't register testdata Msg services. encCfg := simapp.MakeTestEncodingConfig() - app := baseapp.NewBaseApp("test", log.NewDefaultLogger("plain", "info", false), db, encCfg.TxConfig.TxDecoder()) + app := baseapp.NewBaseApp("test", log.MustNewDefaultLogger("plain", "info", false), db, encCfg.TxConfig.TxDecoder()) app.SetInterfaceRegistry(encCfg.InterfaceRegistry) require.Panics(t, func() { testdata.RegisterMsgServer( @@ -45,7 +45,7 @@ func TestRegisterMsgServiceTwice(t *testing.T) { // Setup baseapp. db := dbm.NewMemDB() encCfg := simapp.MakeTestEncodingConfig() - app := baseapp.NewBaseApp("test", log.NewDefaultLogger("plain", "info", false), db, encCfg.TxConfig.TxDecoder()) + app := baseapp.NewBaseApp("test", log.MustNewDefaultLogger("plain", "info", false), db, encCfg.TxConfig.TxDecoder()) app.SetInterfaceRegistry(encCfg.InterfaceRegistry) testdata.RegisterInterfaces(encCfg.InterfaceRegistry) @@ -71,7 +71,7 @@ func TestMsgService(t *testing.T) { encCfg := simapp.MakeTestEncodingConfig() testdata.RegisterInterfaces(encCfg.InterfaceRegistry) db := dbm.NewMemDB() - app := baseapp.NewBaseApp("test", log.NewDefaultLogger("plain", "info", false), db, encCfg.TxConfig.TxDecoder()) + app := baseapp.NewBaseApp("test", log.MustNewDefaultLogger("plain", "info", false), db, encCfg.TxConfig.TxDecoder()) app.SetInterfaceRegistry(encCfg.InterfaceRegistry) testdata.RegisterMsgServer( app.MsgServiceRouter(), diff --git a/baseapp/options.go b/baseapp/options.go index e0960a7cc2a2..139e04c8c536 100644 --- a/baseapp/options.go +++ b/baseapp/options.go @@ -217,6 +217,7 @@ func (app *BaseApp) SetSnapshot(snapshotStore *snapshots.Store, opts snapshottyp func (app *BaseApp) SetInterfaceRegistry(registry types.InterfaceRegistry) { app.interfaceRegistry = registry app.grpcQueryRouter.SetInterfaceRegistry(registry) + app.msgServiceRouter.SetInterfaceRegistry(registry) } // SetStreamingService is used to set a streaming service into the BaseApp hooks and load the listeners into the multistore diff --git a/baseapp/router_test.go b/baseapp/router_test.go index 1e11dc0ca089..970b1f4b275c 100644 --- a/baseapp/router_test.go +++ b/baseapp/router_test.go @@ -12,8 +12,8 @@ var testHandler = func(_ sdk.Context, _ sdk.Msg) (*sdk.Result, error) { return &sdk.Result{}, nil } -func TestRouter(t *testing.T) { - rtr := NewRouter() +func TestLegacyRouter(t *testing.T) { + rtr := NewLegacyRouter() // require panic on invalid route require.Panics(t, func() { From 43b5a4afda4240e184e2cb8fd902dd5c55dbb5b7 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Tue, 17 May 2022 16:40:28 +0200 Subject: [PATCH 04/29] Fix one test --- baseapp/abci_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/baseapp/abci_test.go b/baseapp/abci_test.go index ef306411b2c2..c33b3c79ecf7 100644 --- a/baseapp/abci_test.go +++ b/baseapp/abci_test.go @@ -134,7 +134,7 @@ func TestBaseAppCreateQueryContext(t *testing.T) { logger := defaultLogger() db := dbm.NewMemDB() name := t.Name() - app := NewBaseApp(name, logger, db, nil, nil) + app := NewBaseApp(name, logger, db, nil) app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: 1}}) app.Commit() From fbc63ddb93741564d28dc44eff3afd9a96f31e28 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Tue, 17 May 2022 16:57:54 +0200 Subject: [PATCH 05/29] Fix simapp --- baseapp/baseapp.go | 6 +++--- baseapp/baseapp_test.go | 2 +- baseapp/router.go | 4 ++-- baseapp/router_test.go | 4 ++-- simapp/app.go | 14 +++++--------- simapp/app_test.go | 5 ++--- 6 files changed, 15 insertions(+), 20 deletions(-) diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index 23fe0448d8f3..26e5ad5b0a77 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -51,7 +51,7 @@ type BaseApp struct { // nolint: maligned db dbm.DB // common DB backend cms sdk.CommitMultiStore // Main (uncached) state storeLoader StoreLoader // function to handle store loading, may be overridden with SetStoreLoader() - router sdk.Router // handle any kind of message + router sdk.Router // handle any kind of legacy message queryRouter sdk.QueryRouter // router for redirecting query calls grpcQueryRouter *GRPCQueryRouter // router for redirecting gRPC query calls msgServiceRouter *MsgServiceRouter // router for redirecting Msg service messages @@ -150,7 +150,7 @@ func NewBaseApp( db: db, cms: store.NewCommitMultiStore(db), storeLoader: DefaultStoreLoader, - router: NewLegacyRouter(), + router: NewRouter(), queryRouter: NewQueryRouter(), grpcQueryRouter: NewGRPCQueryRouter(), msgServiceRouter: NewMsgServiceRouter(), @@ -363,7 +363,7 @@ func (app *BaseApp) setIndexEvents(ie []string) { } } -// Router returns the router of the BaseApp. +// Router returns the legacy router of the BaseApp. func (app *BaseApp) Router() sdk.Router { if app.sealed { // We cannot return a Router when the app is sealed because we can't have diff --git a/baseapp/baseapp_test.go b/baseapp/baseapp_test.go index 651d47963ba5..05d5e1756880 100644 --- a/baseapp/baseapp_test.go +++ b/baseapp/baseapp_test.go @@ -534,7 +534,7 @@ func TestBaseAppOptionSeal(t *testing.T) { app.SetFauxMerkleMode() }) require.Panics(t, func() { - app.SetRouter(NewLegacyRouter()) + app.SetRouter(NewRouter()) }) } diff --git a/baseapp/router.go b/baseapp/router.go index 290d60afbc59..7e2e70a0c6f3 100644 --- a/baseapp/router.go +++ b/baseapp/router.go @@ -10,10 +10,10 @@ type Router struct { routes map[string]sdk.Handler } -var _ sdk.Router = NewLegacyRouter() +var _ sdk.Router = NewRouter() // NewRouter returns a reference to a new router. -func NewLegacyRouter() *Router { +func NewRouter() *Router { return &Router{ routes: make(map[string]sdk.Handler), } diff --git a/baseapp/router_test.go b/baseapp/router_test.go index 970b1f4b275c..1e11dc0ca089 100644 --- a/baseapp/router_test.go +++ b/baseapp/router_test.go @@ -12,8 +12,8 @@ var testHandler = func(_ sdk.Context, _ sdk.Msg) (*sdk.Result, error) { return &sdk.Result{}, nil } -func TestLegacyRouter(t *testing.T) { - rtr := NewLegacyRouter() +func TestRouter(t *testing.T) { + rtr := NewRouter() // require panic on invalid route require.Panics(t, func() { diff --git a/simapp/app.go b/simapp/app.go index e9c61d2640c5..0fb348432721 100644 --- a/simapp/app.go +++ b/simapp/app.go @@ -155,8 +155,6 @@ type SimApp struct { legacyAmino *codec.LegacyAmino appCodec codec.Codec interfaceRegistry types.InterfaceRegistry - msgSvcRouter *baseapp.MsgServiceRouter - legacyRouter sdk.Router invCheckPeriod uint @@ -241,8 +239,6 @@ func NewSimApp( legacyAmino: legacyAmino, appCodec: appCodec, interfaceRegistry: interfaceRegistry, - legacyRouter: baseapp.NewLegacyRouter(), - msgSvcRouter: baseapp.NewMsgServiceRouter(), invCheckPeriod: invCheckPeriod, keys: keys, tkeys: tkeys, @@ -292,14 +288,14 @@ func NewSimApp( stakingtypes.NewMultiStakingHooks(app.DistrKeeper.Hooks(), app.SlashingKeeper.Hooks()), ) - app.AuthzKeeper = authzkeeper.NewKeeper(keys[authzkeeper.StoreKey], appCodec, app.msgSvcRouter, app.AccountKeeper) + app.AuthzKeeper = authzkeeper.NewKeeper(keys[authzkeeper.StoreKey], appCodec, app.MsgServiceRouter(), app.AccountKeeper) groupConfig := group.DefaultConfig() /* Example of setting group params: groupConfig.MaxMetadataLen = 1000 */ - app.GroupKeeper = groupkeeper.NewKeeper(keys[group.StoreKey], appCodec, app.msgSvcRouter, app.AccountKeeper, groupConfig) + app.GroupKeeper = groupkeeper.NewKeeper(keys[group.StoreKey], appCodec, app.MsgServiceRouter(), app.AccountKeeper, groupConfig) // register the proposal types govRouter := govv1beta1.NewRouter() @@ -314,7 +310,7 @@ func NewSimApp( */ govKeeper := govkeeper.NewKeeper( appCodec, keys[govtypes.StoreKey], app.GetSubspace(govtypes.ModuleName), app.AccountKeeper, app.BankKeeper, - &stakingKeeper, govRouter, app.msgSvcRouter, govConfig, + &stakingKeeper, govRouter, app.MsgServiceRouter(), govConfig, ) app.GovKeeper = *govKeeper.SetHooks( @@ -408,8 +404,8 @@ func NewSimApp( // app.mm.SetOrderMigrations(custom order) app.mm.RegisterInvariants(&app.CrisisKeeper) - app.mm.RegisterRoutes(app.legacyRouter, app.QueryRouter(), encodingConfig.Amino) - app.configurator = module.NewConfigurator(app.appCodec, app.msgSvcRouter, app.GRPCQueryRouter()) + app.mm.RegisterRoutes(app.Router(), app.QueryRouter(), encodingConfig.Amino) + app.configurator = module.NewConfigurator(app.appCodec, app.MsgServiceRouter(), app.GRPCQueryRouter()) app.mm.RegisterServices(app.configurator) // add test gRPC service for testing gRPC queries in isolation diff --git a/simapp/app_test.go b/simapp/app_test.go index ad8cd8b6b781..15a65536cdca 100644 --- a/simapp/app_test.go +++ b/simapp/app_test.go @@ -16,7 +16,6 @@ import ( sdk "github.com/cosmos/cosmos-sdk/types" "github.com/cosmos/cosmos-sdk/types/module" "github.com/cosmos/cosmos-sdk/x/auth" - authmiddleware "github.com/cosmos/cosmos-sdk/x/auth/ante" "github.com/cosmos/cosmos-sdk/x/auth/vesting" authzmodule "github.com/cosmos/cosmos-sdk/x/authz/module" "github.com/cosmos/cosmos-sdk/x/bank" @@ -79,10 +78,10 @@ func TestRunMigrations(t *testing.T) { app := NewSimApp(logger, db, nil, true, map[int64]bool{}, DefaultNodeHome, 0, encCfg, EmptyAppOptions{}) // Create a new baseapp and configurator for the purpose of this test. - bApp := baseapp.NewBaseApp(appName, logger, db) + bApp := baseapp.NewBaseApp(appName, logger, db, encCfg.TxConfig.TxDecoder()) bApp.SetCommitMultiStoreTracer(nil) bApp.SetInterfaceRegistry(encCfg.InterfaceRegistry) - msr := authante.NewMsgServiceRouter(encCfg.InterfaceRegistry) + msr := baseapp.NewMsgServiceRouter() app.BaseApp = bApp app.configurator = module.NewConfigurator(app.appCodec, msr, app.GRPCQueryRouter()) From 0a0718169ba90567ebd6acde9683578bca502a37 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Tue, 17 May 2022 17:44:35 +0200 Subject: [PATCH 06/29] Fix more tests --- baseapp/abci.go | 19 ++---------- baseapp/baseapp.go | 26 +++++++++++----- baseapp/baseapp_test.go | 2 ++ server/mock/app.go | 46 +++++------------------------ server/mock/tx.go | 27 ++++++----------- simapp/app_test.go | 3 +- x/auth/ante/ante_test.go | 7 ++--- x/auth/ante/fee_test.go | 6 ++-- x/auth/ante/feegrant_test.go | 6 ++-- x/auth/tx/service_test.go | 18 ++++------- x/upgrade/types/storeloader_test.go | 4 +-- 11 files changed, 55 insertions(+), 109 deletions(-) diff --git a/baseapp/abci.go b/baseapp/abci.go index 260ff008f09a..489e0e95f59b 100644 --- a/baseapp/abci.go +++ b/baseapp/abci.go @@ -261,16 +261,11 @@ func (app *BaseApp) CheckTx(req abci.RequestCheckTx) abci.ResponseCheckTx { return sdkerrors.ResponseCheckTxWithEvents(err, gInfo.GasWanted, gInfo.GasUsed, anteEvents, app.trace) } - data, err := makeABCIData(result) - if err != nil { - return sdkerrors.ResponseCheckTxWithEvents(err, gInfo.GasWanted, gInfo.GasUsed, anteEvents, app.trace) - } - return abci.ResponseCheckTx{ GasWanted: int64(gInfo.GasWanted), // TODO: Should type accept unsigned ints? GasUsed: int64(gInfo.GasUsed), // TODO: Should type accept unsigned ints? Log: result.Log, - Data: data, + Data: result.Data, Events: sdk.MarkEventsToIndex(result.Events, app.indexEvents), } } @@ -299,16 +294,11 @@ func (app *BaseApp) DeliverTx(req abci.RequestDeliverTx) abci.ResponseDeliverTx return sdkerrors.ResponseDeliverTxWithEvents(err, gInfo.GasWanted, gInfo.GasUsed, anteEvents, app.trace) } - data, err := makeABCIData(result) - if err != nil { - return sdkerrors.ResponseDeliverTxWithEvents(err, gInfo.GasWanted, gInfo.GasUsed, anteEvents, app.trace) - } - return abci.ResponseDeliverTx{ GasWanted: int64(gInfo.GasWanted), // TODO: Should type accept unsigned ints? GasUsed: int64(gInfo.GasUsed), // TODO: Should type accept unsigned ints? Log: result.Log, - Data: data, + Data: result.Data, Events: sdk.MarkEventsToIndex(result.Events, app.indexEvents), } } @@ -872,8 +862,3 @@ func SplitABCIQueryPath(requestPath string) (path []string) { return path } - -// makeABCIData generates the Data field to be sent to ABCI Check/DeliverTx. -func makeABCIData(txRes *sdk.Result) ([]byte, error) { - return proto.Marshal(&sdk.TxMsgData{MsgResponses: txRes.MsgResponses}) -} diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index 26e5ad5b0a77..3eba4bc03a83 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -12,6 +12,7 @@ import ( dbm "github.com/tendermint/tm-db" "github.com/cosmos/cosmos-sdk/codec/types" + codectypes "github.com/cosmos/cosmos-sdk/codec/types" "github.com/cosmos/cosmos-sdk/snapshots" "github.com/cosmos/cosmos-sdk/store" "github.com/cosmos/cosmos-sdk/store/rootmulti" @@ -711,9 +712,7 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte) (gInfo sdk.GasInfo, re func (app *BaseApp) runMsgs(ctx sdk.Context, msgs []sdk.Msg, mode runTxMode) (*sdk.Result, error) { msgLogs := make(sdk.ABCIMessageLogs, 0, len(msgs)) events := sdk.EmptyEvents() - txMsgData := &sdk.TxMsgData{ - Data: make([]*sdk.MsgData, 0, len(msgs)), - } + msgResponses := make([]*codectypes.Any, len(msgs)) // NOTE: GasWanted is determined by the AnteHandler and GasUsed by the GasMeter. for i, msg := range msgs { @@ -765,18 +764,29 @@ func (app *BaseApp) runMsgs(ctx sdk.Context, msgs []sdk.Msg, mode runTxMode) (*s // separate each result. events = events.AppendEvents(msgEvents) - txMsgData.Data = append(txMsgData.Data, &sdk.MsgData{MsgType: sdk.MsgTypeURL(msg), Data: msgResult.Data}) + // Each individual sdk.Result has exactly one Msg response. We aggregate here. + msgResponse := msgResult.MsgResponses[0] + if msgResponse == nil { + return nil, sdkerrors.ErrLogic.Wrapf("got nil Msg response at index %d for msg %s", i, sdk.MsgTypeURL(msg)) + } + msgResponses[i] = msgResponse msgLogs = append(msgLogs, sdk.NewABCIMessageLog(uint32(i), msgResult.Log, msgEvents)) } - data, err := proto.Marshal(txMsgData) + data, err := makeABCIData(msgResponses) if err != nil { return nil, sdkerrors.Wrap(err, "failed to marshal tx data") } return &sdk.Result{ - Data: data, - Log: strings.TrimSpace(msgLogs.String()), - Events: events.ToABCIEvents(), + Data: data, + Log: strings.TrimSpace(msgLogs.String()), + Events: events.ToABCIEvents(), + MsgResponses: msgResponses, }, nil } + +// makeABCIData generates the Data field to be sent to ABCI Check/DeliverTx. +func makeABCIData(msgResponses []*codectypes.Any) ([]byte, error) { + return proto.Marshal(&sdk.TxMsgData{MsgResponses: msgResponses}) +} diff --git a/baseapp/baseapp_test.go b/baseapp/baseapp_test.go index 05d5e1756880..fcf8b1a788c8 100644 --- a/baseapp/baseapp_test.go +++ b/baseapp/baseapp_test.go @@ -20,6 +20,7 @@ import ( dbm "github.com/tendermint/tm-db" "github.com/cosmos/cosmos-sdk/codec" + codectypes "github.com/cosmos/cosmos-sdk/codec/types" pruningtypes "github.com/cosmos/cosmos-sdk/pruning/types" "github.com/cosmos/cosmos-sdk/snapshots" snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" @@ -1707,6 +1708,7 @@ func TestGRPCQuery(t *testing.T) { } app := setupBaseApp(t, grpcQueryOpt) + app.GRPCQueryRouter().SetInterfaceRegistry(codectypes.NewInterfaceRegistry()) app.InitChain(abci.RequestInitChain{}) header := tmproto.Header{Height: app.LastBlockHeight() + 1} diff --git a/server/mock/app.go b/server/mock/app.go index 01ea62b09318..56c006b57833 100644 --- a/server/mock/app.go +++ b/server/mock/app.go @@ -6,36 +6,22 @@ import ( "fmt" "path/filepath" + "github.com/tendermint/tendermint/types" + abci "github.com/tendermint/tendermint/abci/types" "github.com/tendermint/tendermint/libs/log" - "github.com/tendermint/tendermint/types" - dbm "github.com/tendermint/tm-db" bam "github.com/cosmos/cosmos-sdk/baseapp" "github.com/cosmos/cosmos-sdk/codec" - codectypes "github.com/cosmos/cosmos-sdk/codec/types" - "github.com/cosmos/cosmos-sdk/simapp" storetypes "github.com/cosmos/cosmos-sdk/store/types" sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/types/tx" - "github.com/cosmos/cosmos-sdk/x/auth/ante" ) -func testTxHandler(options ante.TxHandlerOptions) tx.Handler { - return ante.ComposeMiddlewares( - ante.NewRunMsgsTxHandler(options.MsgServiceRouter, options.LegacyRouter), - ante.NewTxDecoderMiddleware(options.TxDecoder), - ante.GasTxMiddleware, - ante.RecoveryTxMiddleware, - ante.NewIndexEventsTxMiddleware(options.IndexEvents), - ) -} - // NewApp creates a simple mock kvstore app for testing. It should work // similar to a real app. Make sure rootDir is empty before running the test, // in order to guarantee consistent results func NewApp(rootDir string, logger log.Logger) (abci.Application, error) { - db, err := dbm.NewDB("mock", dbm.MemDBBackend, filepath.Join(rootDir, "data")) + db, err := sdk.NewLevelDB("mock", filepath.Join(rootDir, "data")) if err != nil { return nil, err } @@ -44,7 +30,7 @@ func NewApp(rootDir string, logger log.Logger) (abci.Application, error) { capKeyMainStore := sdk.NewKVStoreKey("main") // Create BaseApp. - baseApp := bam.NewBaseApp("kvstore", logger, db) + baseApp := bam.NewBaseApp("kvstore", logger, db, decodeTx) // Set mounts for BaseApp's MultiStore. baseApp.MountStores(capKeyMainStore) @@ -52,19 +38,7 @@ func NewApp(rootDir string, logger log.Logger) (abci.Application, error) { baseApp.SetInitChainer(InitChainer(capKeyMainStore)) // Set a Route. - encCfg := simapp.MakeTestEncodingConfig() - legacyRouter := ante.NewLegacyRouter() - // We're adding a test legacy route here, which accesses the kvstore - // and simply sets the Msg's key/value pair in the kvstore. - legacyRouter.AddRoute(sdk.NewRoute("kvstore", KVStoreHandler(capKeyMainStore))) - txHandler := testTxHandler( - ante.TxHandlerOptions{ - LegacyRouter: legacyRouter, - MsgServiceRouter: ante.NewMsgServiceRouter(encCfg.InterfaceRegistry), - TxDecoder: decodeTx, - }, - ) - baseApp.SetTxHandler(txHandler) + baseApp.Router().AddRoute(sdk.NewRoute("kvstore", KVStoreHandler(capKeyMainStore))) // Load latest version. if err := baseApp.LoadLatestVersion(); err != nil { @@ -78,7 +52,7 @@ func NewApp(rootDir string, logger log.Logger) (abci.Application, error) { // them to the db func KVStoreHandler(storeKey storetypes.StoreKey) sdk.Handler { return func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { - dTx, ok := msg.(*kvstoreTx) + dTx, ok := msg.(kvstoreTx) if !ok { return nil, errors.New("KVStoreHandler should only receive kvstoreTx") } @@ -90,14 +64,8 @@ func KVStoreHandler(storeKey storetypes.StoreKey) sdk.Handler { store := ctx.KVStore(storeKey) store.Set(key, value) - any, err := codectypes.NewAnyWithValue(msg) - if err != nil { - return nil, err - } - return &sdk.Result{ - Log: fmt.Sprintf("set %s=%s", key, value), - MsgResponses: []*codectypes.Any{any}, + Log: fmt.Sprintf("set %s=%s", key, value), }, nil } } diff --git a/server/mock/tx.go b/server/mock/tx.go index 90ef3981d93f..0cb79c28986f 100644 --- a/server/mock/tx.go +++ b/server/mock/tx.go @@ -4,16 +4,12 @@ package mock import ( "bytes" "fmt" - "math" sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/x/auth/ante" ) -// kvstoreTx defines a tx for mock purposes. The `key` and `value` fields will -// set those bytes in the kvstore, and the `bytes` field represents its -// GetSignBytes value. +// An sdk.Tx which is its own sdk.Msg. type kvstoreTx struct { key []byte value []byte @@ -21,13 +17,12 @@ type kvstoreTx struct { } // dummy implementation of proto.Message -func (msg *kvstoreTx) Reset() {} -func (msg *kvstoreTx) String() string { return "TODO" } -func (msg *kvstoreTx) ProtoMessage() {} +func (msg kvstoreTx) Reset() {} +func (msg kvstoreTx) String() string { return "TODO" } +func (msg kvstoreTx) ProtoMessage() {} -var _ sdk.Tx = &kvstoreTx{} -var _ sdk.Msg = &kvstoreTx{} -var _ ante.GasTx = &kvstoreTx{} +var _ sdk.Tx = kvstoreTx{} +var _ sdk.Msg = kvstoreTx{} func NewTx(key, value string) kvstoreTx { bytes := fmt.Sprintf("%s=%s", key, value) @@ -46,7 +41,7 @@ func (tx kvstoreTx) Type() string { return "kvstore_tx" } -func (tx *kvstoreTx) GetMsgs() []sdk.Msg { +func (tx kvstoreTx) GetMsgs() []sdk.Msg { return []sdk.Msg{tx} } @@ -67,10 +62,6 @@ func (tx kvstoreTx) GetSigners() []sdk.AccAddress { return nil } -func (tx kvstoreTx) GetGas() uint64 { - return math.MaxUint64 -} - // takes raw transaction bytes and decodes them into an sdk.Tx. An sdk.Tx has // all the signatures and can be used to authenticate. func decodeTx(txBytes []byte) (sdk.Tx, error) { @@ -79,10 +70,10 @@ func decodeTx(txBytes []byte) (sdk.Tx, error) { split := bytes.Split(txBytes, []byte("=")) if len(split) == 1 { k := split[0] - tx = &kvstoreTx{k, k, txBytes} + tx = kvstoreTx{k, k, txBytes} } else if len(split) == 2 { k, v := split[0], split[1] - tx = &kvstoreTx{k, v, txBytes} + tx = kvstoreTx{k, v, txBytes} } else { return nil, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "too many '='") } diff --git a/simapp/app_test.go b/simapp/app_test.go index 15a65536cdca..e5bd3365f6b6 100644 --- a/simapp/app_test.go +++ b/simapp/app_test.go @@ -81,9 +81,8 @@ func TestRunMigrations(t *testing.T) { bApp := baseapp.NewBaseApp(appName, logger, db, encCfg.TxConfig.TxDecoder()) bApp.SetCommitMultiStoreTracer(nil) bApp.SetInterfaceRegistry(encCfg.InterfaceRegistry) - msr := baseapp.NewMsgServiceRouter() app.BaseApp = bApp - app.configurator = module.NewConfigurator(app.appCodec, msr, app.GRPCQueryRouter()) + app.configurator = module.NewConfigurator(app.appCodec, bApp.MsgServiceRouter(), app.GRPCQueryRouter()) // We register all modules on the Configurator, except x/bank. x/bank will // serve as the test subject on which we run the migration tests. diff --git a/x/auth/ante/ante_test.go b/x/auth/ante/ante_test.go index d2af80904476..b87bd6759d6a 100644 --- a/x/auth/ante/ante_test.go +++ b/x/auth/ante/ante_test.go @@ -7,8 +7,7 @@ import ( "strings" "testing" - "github.com/cosmos/cosmos-sdk/simapp" - + "github.com/cosmos/cosmos-sdk/x/bank/testutil" minttypes "github.com/cosmos/cosmos-sdk/x/mint/types" "github.com/stretchr/testify/require" @@ -472,7 +471,7 @@ func (suite *AnteTestSuite) TestAnteHandlerFees() { { "signer does not have enough funds to pay the fee", func() { - err := simapp.FundAccount(suite.app.BankKeeper, suite.ctx, addr0, sdk.NewCoins(sdk.NewInt64Coin("atom", 149))) + err := testutil.FundAccount(suite.app.BankKeeper, suite.ctx, addr0, sdk.NewCoins(sdk.NewInt64Coin("atom", 149))) suite.Require().NoError(err) }, false, @@ -489,7 +488,7 @@ func (suite *AnteTestSuite) TestAnteHandlerFees() { suite.Require().True(suite.app.BankKeeper.GetAllBalances(suite.ctx, modAcc.GetAddress()).Empty()) require.True(sdk.IntEq(suite.T(), suite.app.BankKeeper.GetAllBalances(suite.ctx, addr0).AmountOf("atom"), sdk.NewInt(149))) - err := simapp.FundAccount(suite.app.BankKeeper, suite.ctx, addr0, sdk.NewCoins(sdk.NewInt64Coin("atom", 1))) + err := testutil.FundAccount(suite.app.BankKeeper, suite.ctx, addr0, sdk.NewCoins(sdk.NewInt64Coin("atom", 1))) suite.Require().NoError(err) }, false, diff --git a/x/auth/ante/fee_test.go b/x/auth/ante/fee_test.go index 7edd8fad9257..06ccb4d3948f 100644 --- a/x/auth/ante/fee_test.go +++ b/x/auth/ante/fee_test.go @@ -2,10 +2,10 @@ package ante_test import ( cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" - "github.com/cosmos/cosmos-sdk/simapp" "github.com/cosmos/cosmos-sdk/testutil/testdata" sdk "github.com/cosmos/cosmos-sdk/types" "github.com/cosmos/cosmos-sdk/x/auth/ante" + "github.com/cosmos/cosmos-sdk/x/bank/testutil" ) func (suite *AnteTestSuite) TestEnsureMempoolFees() { @@ -83,7 +83,7 @@ func (suite *AnteTestSuite) TestDeductFees() { acc := suite.app.AccountKeeper.NewAccountWithAddress(suite.ctx, addr1) suite.app.AccountKeeper.SetAccount(suite.ctx, acc) coins := sdk.NewCoins(sdk.NewCoin("atom", sdk.NewInt(10))) - err = simapp.FundAccount(suite.app.BankKeeper, suite.ctx, addr1, coins) + err = testutil.FundAccount(suite.app.BankKeeper, suite.ctx, addr1, coins) suite.Require().NoError(err) dfd := ante.NewDeductFeeDecorator(suite.app.AccountKeeper, suite.app.BankKeeper, nil) @@ -95,7 +95,7 @@ func (suite *AnteTestSuite) TestDeductFees() { // Set account with sufficient funds suite.app.AccountKeeper.SetAccount(suite.ctx, acc) - err = simapp.FundAccount(suite.app.BankKeeper, suite.ctx, addr1, sdk.NewCoins(sdk.NewCoin("atom", sdk.NewInt(200)))) + err = testutil.FundAccount(suite.app.BankKeeper, suite.ctx, addr1, sdk.NewCoins(sdk.NewCoin("atom", sdk.NewInt(200)))) suite.Require().NoError(err) _, err = antehandler(suite.ctx, tx, false) diff --git a/x/auth/ante/feegrant_test.go b/x/auth/ante/feegrant_test.go index b23c2cec619d..756ee307e13f 100644 --- a/x/auth/ante/feegrant_test.go +++ b/x/auth/ante/feegrant_test.go @@ -11,7 +11,6 @@ import ( "github.com/cosmos/cosmos-sdk/codec" cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" - "github.com/cosmos/cosmos-sdk/simapp" "github.com/cosmos/cosmos-sdk/simapp/helpers" "github.com/cosmos/cosmos-sdk/testutil/testdata" sdk "github.com/cosmos/cosmos-sdk/types" @@ -21,6 +20,7 @@ import ( authsign "github.com/cosmos/cosmos-sdk/x/auth/signing" "github.com/cosmos/cosmos-sdk/x/auth/tx" authtypes "github.com/cosmos/cosmos-sdk/x/auth/types" + "github.com/cosmos/cosmos-sdk/x/bank/testutil" "github.com/cosmos/cosmos-sdk/x/feegrant" ) @@ -46,11 +46,11 @@ func (suite *AnteTestSuite) TestDeductFeesNoDelegation() { priv5, _, addr5 := testdata.KeyTestPubAddr() // Set addr1 with insufficient funds - err := simapp.FundAccount(suite.app.BankKeeper, suite.ctx, addr1, []sdk.Coin{sdk.NewCoin("atom", sdk.NewInt(10))}) + err := testutil.FundAccount(suite.app.BankKeeper, suite.ctx, addr1, []sdk.Coin{sdk.NewCoin("atom", sdk.NewInt(10))}) suite.Require().NoError(err) // Set addr2 with more funds - err = simapp.FundAccount(suite.app.BankKeeper, suite.ctx, addr2, []sdk.Coin{sdk.NewCoin("atom", sdk.NewInt(99999))}) + err = testutil.FundAccount(suite.app.BankKeeper, suite.ctx, addr2, []sdk.Coin{sdk.NewCoin("atom", sdk.NewInt(99999))}) suite.Require().NoError(err) // grant fee allowance from `addr2` to `addr3` (plenty to pay) diff --git a/x/auth/tx/service_test.go b/x/auth/tx/service_test.go index 28481c57c38c..019007cca92c 100644 --- a/x/auth/tx/service_test.go +++ b/x/auth/tx/service_test.go @@ -79,7 +79,7 @@ func (s *IntegrationTestSuite) SetupSuite() { ) s.Require().NoError(err) s.Require().NoError(val.ClientCtx.Codec.UnmarshalJSON(out.Bytes(), &s.txRes)) - s.Require().Equal(uint32(0), s.txRes.Code) + s.Require().Equal(uint32(0), s.txRes.Code, s.txRes) out, err = bankcli.MsgSendExec( val.ClientCtx, @@ -145,16 +145,8 @@ func (s IntegrationTestSuite) TestSimulateTx_GRPC() { } else { s.Require().NoError(err) // Check the result and gas used are correct. - // - // The 13 events are: - // - Sending Fee to the pool: coin_spent, coin_received, transfer and message.sender= - // - tx.* events: tx.fee, tx.acc_seq, tx.signature - // - Sending Amount to recipient: coin_spent, coin_received, transfer and message.sender= - // - Msg events: message.module=bank and message.action=/cosmos.bank.v1beta1.MsgSend - s.Require().Equal(len(res.GetResult().GetEvents()), 13) - s.Require().Len(res.GetResult().MsgResponses, 1) - // Check the result and gas used are correct. - s.Require().True(res.GetGasInfo().GetGasUsed() > 0) // Gas used sometimes change, just check it's not empty. + s.Require().Equal(len(res.GetResult().GetEvents()), 6) // 1 coin recv 1 coin spent, 1 transfer, 3 messages. + s.Require().True(res.GetGasInfo().GetGasUsed() > 0) // Gas used sometimes change, just check it's not empty. } }) } @@ -194,9 +186,9 @@ func (s IntegrationTestSuite) TestSimulateTx_GRPCGateway() { err = val.ClientCtx.Codec.UnmarshalJSON(res, &result) s.Require().NoError(err) // Check the result and gas used are correct. - s.Require().Equal(len(result.GetResult().GetEvents()), 13) // See TestSimulateTx_GRPC for the 13 events. s.Require().Len(result.GetResult().MsgResponses, 1) - s.Require().True(result.GetGasInfo().GetGasUsed() > 0) // Gas used sometimes change, just check it's not empty. + s.Require().Equal(len(result.GetResult().GetEvents()), 6) // 1 coin recv 1 coin spent, 1 transfer, 3 messages. + s.Require().True(result.GetGasInfo().GetGasUsed() > 0) // Gas used sometimes change, jus } }) } diff --git a/x/upgrade/types/storeloader_test.go b/x/upgrade/types/storeloader_test.go index ba3d772039b7..4f48949151ac 100644 --- a/x/upgrade/types/storeloader_test.go +++ b/x/upgrade/types/storeloader_test.go @@ -125,7 +125,7 @@ func TestSetLoader(t *testing.T) { // load the app with the existing db opts := []func(*baseapp.BaseApp){baseapp.SetPruning(pruningtypes.NewPruningOptions(pruningtypes.PruningNothing))} - origapp := baseapp.NewBaseApp(t.Name(), defaultLogger(), db, opts...) + origapp := baseapp.NewBaseApp(t.Name(), defaultLogger(), db, nil, opts...) origapp.MountStores(sdk.NewKVStoreKey(tc.origStoreKey)) err := origapp.LoadLatestVersion() require.Nil(t, err) @@ -141,7 +141,7 @@ func TestSetLoader(t *testing.T) { } // load the new app with the original app db - app := baseapp.NewBaseApp(t.Name(), defaultLogger(), db, opts...) + app := baseapp.NewBaseApp(t.Name(), defaultLogger(), db, nil, opts...) app.MountStores(sdk.NewKVStoreKey(tc.loadStoreKey)) err = app.LoadLatestVersion() require.Nil(t, err) From f0a6d115b796ee7ec3ee2193a5b85a5fab319858 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Tue, 17 May 2022 18:07:48 +0200 Subject: [PATCH 07/29] Fix another one? --- baseapp/baseapp.go | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index 3eba4bc03a83..fb2110a333c2 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -712,7 +712,7 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte) (gInfo sdk.GasInfo, re func (app *BaseApp) runMsgs(ctx sdk.Context, msgs []sdk.Msg, mode runTxMode) (*sdk.Result, error) { msgLogs := make(sdk.ABCIMessageLogs, 0, len(msgs)) events := sdk.EmptyEvents() - msgResponses := make([]*codectypes.Any, len(msgs)) + var msgResponses []*codectypes.Any // NOTE: GasWanted is determined by the AnteHandler and GasUsed by the GasMeter. for i, msg := range msgs { @@ -764,12 +764,16 @@ func (app *BaseApp) runMsgs(ctx sdk.Context, msgs []sdk.Msg, mode runTxMode) (*s // separate each result. events = events.AppendEvents(msgEvents) - // Each individual sdk.Result has exactly one Msg response. We aggregate here. - msgResponse := msgResult.MsgResponses[0] - if msgResponse == nil { - return nil, sdkerrors.ErrLogic.Wrapf("got nil Msg response at index %d for msg %s", i, sdk.MsgTypeURL(msg)) + // Each individual sdk.Result that went through the MsgServiceRouter + // has exactly one Msg response. We aggregate here. + if len(msgResult.MsgResponses) > 0 { + msgResponse := msgResult.MsgResponses[0] + if msgResponse == nil { + return nil, sdkerrors.ErrLogic.Wrapf("got nil Msg response at index %d for msg %s", i, sdk.MsgTypeURL(msg)) + } + msgResponses = append(msgResponses, msgResponse) } - msgResponses[i] = msgResponse + msgLogs = append(msgLogs, sdk.NewABCIMessageLog(uint32(i), msgResult.Log, msgEvents)) } From 659c2bd5509c7453411dce9c67c197ce89095113 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Tue, 17 May 2022 18:20:43 +0200 Subject: [PATCH 08/29] Fix ante test build --- baseapp/block_gas_test.go | 200 ++++++++++++++++++++++++++++++++++ baseapp/test_helpers.go | 4 + x/auth/ante/sigverify_test.go | 2 +- x/auth/ante/testutil_test.go | 6 +- 4 files changed, 208 insertions(+), 4 deletions(-) create mode 100644 baseapp/block_gas_test.go diff --git a/baseapp/block_gas_test.go b/baseapp/block_gas_test.go new file mode 100644 index 000000000000..8f64795bee11 --- /dev/null +++ b/baseapp/block_gas_test.go @@ -0,0 +1,200 @@ +package baseapp_test + +import ( + "encoding/json" + "fmt" + "math" + "testing" + + "github.com/stretchr/testify/require" + abci "github.com/tendermint/tendermint/abci/types" + "github.com/tendermint/tendermint/libs/log" + tmproto "github.com/tendermint/tendermint/proto/tendermint/types" + dbm "github.com/tendermint/tm-db" + + "github.com/cosmos/cosmos-sdk/baseapp" + "github.com/cosmos/cosmos-sdk/client" + "github.com/cosmos/cosmos-sdk/client/tx" + cryptotypes "github.com/cosmos/cosmos-sdk/crypto/types" + "github.com/cosmos/cosmos-sdk/simapp" + "github.com/cosmos/cosmos-sdk/testutil/testdata" + sdk "github.com/cosmos/cosmos-sdk/types" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" + txtypes "github.com/cosmos/cosmos-sdk/types/tx" + "github.com/cosmos/cosmos-sdk/types/tx/signing" + xauthsigning "github.com/cosmos/cosmos-sdk/x/auth/signing" + banktypes "github.com/cosmos/cosmos-sdk/x/bank/types" + minttypes "github.com/cosmos/cosmos-sdk/x/mint/types" +) + +var blockMaxGas = uint64(simapp.DefaultConsensusParams.Block.MaxGas) + +func TestBaseApp_BlockGas(t *testing.T) { + testcases := []struct { + name string + gasToConsume uint64 // gas to consume in the msg execution + panicTx bool // panic explicitly in tx execution + expErr bool + }{ + {"less than block gas meter", 10, false, false}, + {"more than block gas meter", blockMaxGas, false, true}, + {"more than block gas meter", uint64(float64(blockMaxGas) * 1.2), false, true}, + {"consume MaxUint64", math.MaxUint64, false, true}, + {"consume MaxGasWanted", txtypes.MaxGasWanted, false, true}, + {"consume block gas when paniced", 10, true, true}, + } + for _, tc := range testcases { + t.Run(tc.name, func(t *testing.T) { + var app *simapp.SimApp + routerOpt := func(bapp *baseapp.BaseApp) { + route := (&testdata.TestMsg{}).Route() + bapp.Router().AddRoute(sdk.NewRoute(route, func(ctx sdk.Context, msg sdk.Msg) (*sdk.Result, error) { + _, ok := msg.(*testdata.TestMsg) + if !ok { + return &sdk.Result{}, fmt.Errorf("Wrong Msg type, expected %T, got %T", (*testdata.TestMsg)(nil), msg) + } + ctx.KVStore(app.GetKey(banktypes.ModuleName)).Set([]byte("ok"), []byte("ok")) + ctx.GasMeter().ConsumeGas(tc.gasToConsume, "TestMsg") + if tc.panicTx { + panic("panic in tx execution") + } + return &sdk.Result{}, nil + })) + } + encCfg := simapp.MakeTestEncodingConfig() + encCfg.Amino.RegisterConcrete(&testdata.TestMsg{}, "testdata.TestMsg", nil) + encCfg.InterfaceRegistry.RegisterImplementations((*sdk.Msg)(nil), + &testdata.TestMsg{}, + ) + app = simapp.NewSimApp(log.NewNopLogger(), dbm.NewMemDB(), nil, true, map[int64]bool{}, "", 0, encCfg, simapp.EmptyAppOptions{}, routerOpt) + genState := simapp.NewDefaultGenesisState(encCfg.Codec) + stateBytes, err := json.MarshalIndent(genState, "", " ") + require.NoError(t, err) + app.InitChain(abci.RequestInitChain{ + Validators: []abci.ValidatorUpdate{}, + ConsensusParams: simapp.DefaultConsensusParams, + AppStateBytes: stateBytes, + }) + + ctx := app.NewContext(false, tmproto.Header{}) + + // tx fee + feeCoin := sdk.NewCoin("atom", sdk.NewInt(150)) + feeAmount := sdk.NewCoins(feeCoin) + + // test account and fund + priv1, _, addr1 := testdata.KeyTestPubAddr() + err = app.BankKeeper.MintCoins(ctx, minttypes.ModuleName, feeAmount) + require.NoError(t, err) + err = app.BankKeeper.SendCoinsFromModuleToAccount(ctx, minttypes.ModuleName, addr1, feeAmount) + require.NoError(t, err) + require.Equal(t, feeCoin.Amount, app.BankKeeper.GetBalance(ctx, addr1, feeCoin.Denom).Amount) + seq, _ := app.AccountKeeper.GetSequence(ctx, addr1) + require.Equal(t, uint64(0), seq) + + // msg and signatures + msg := testdata.NewTestMsg(addr1) + + txBuilder := encCfg.TxConfig.NewTxBuilder() + require.NoError(t, txBuilder.SetMsgs(msg)) + txBuilder.SetFeeAmount(feeAmount) + txBuilder.SetGasLimit(txtypes.MaxGasWanted) // tx validation checks that gasLimit can't be bigger than this + + privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{6}, []uint64{0} + _, txBytes, err := createTestTx(encCfg.TxConfig, txBuilder, privs, accNums, accSeqs, ctx.ChainID()) + require.NoError(t, err) + + app.BeginBlock(abci.RequestBeginBlock{Header: tmproto.Header{Height: 1}}) + rsp := app.DeliverTx(abci.RequestDeliverTx{Tx: txBytes}) + + // check result + ctx = app.GetContextForDeliverTx(txBytes) + okValue := ctx.KVStore(app.GetKey(banktypes.ModuleName)).Get([]byte("ok")) + + if tc.expErr { + if tc.panicTx { + require.Equal(t, sdkerrors.ErrPanic.ABCICode(), rsp.Code) + } else { + require.Equal(t, sdkerrors.ErrOutOfGas.ABCICode(), rsp.Code) + } + require.Empty(t, okValue) + } else { + require.Equal(t, uint32(0), rsp.Code) + require.Equal(t, []byte("ok"), okValue) + } + // check block gas is always consumed + baseGas := uint64(59142) // baseGas is the gas consumed before tx msg + expGasConsumed := addUint64Saturating(tc.gasToConsume, baseGas) + if expGasConsumed > txtypes.MaxGasWanted { + // capped by gasLimit + expGasConsumed = txtypes.MaxGasWanted + } + require.Equal(t, expGasConsumed, ctx.BlockGasMeter().GasConsumed()) + // tx fee is always deducted + require.Equal(t, int64(0), app.BankKeeper.GetBalance(ctx, addr1, feeCoin.Denom).Amount.Int64()) + // sender's sequence is always increased + seq, err = app.AccountKeeper.GetSequence(ctx, addr1) + require.NoError(t, err) + require.Equal(t, uint64(1), seq) + }) + } +} + +func createTestTx(txConfig client.TxConfig, txBuilder client.TxBuilder, privs []cryptotypes.PrivKey, accNums []uint64, accSeqs []uint64, chainID string) (xauthsigning.Tx, []byte, error) { + // First round: we gather all the signer infos. We use the "set empty + // signature" hack to do that. + var sigsV2 []signing.SignatureV2 + for i, priv := range privs { + sigV2 := signing.SignatureV2{ + PubKey: priv.PubKey(), + Data: &signing.SingleSignatureData{ + SignMode: txConfig.SignModeHandler().DefaultMode(), + Signature: nil, + }, + Sequence: accSeqs[i], + } + + sigsV2 = append(sigsV2, sigV2) + } + err := txBuilder.SetSignatures(sigsV2...) + if err != nil { + return nil, nil, err + } + + // Second round: all signer infos are set, so each signer can sign. + sigsV2 = []signing.SignatureV2{} + for i, priv := range privs { + signerData := xauthsigning.SignerData{ + ChainID: chainID, + AccountNumber: accNums[i], + Sequence: accSeqs[i], + } + sigV2, err := tx.SignWithPrivKey( + txConfig.SignModeHandler().DefaultMode(), signerData, + txBuilder, priv, txConfig, accSeqs[i]) + if err != nil { + return nil, nil, err + } + + sigsV2 = append(sigsV2, sigV2) + } + err = txBuilder.SetSignatures(sigsV2...) + if err != nil { + return nil, nil, err + } + + txBytes, err := txConfig.TxEncoder()(txBuilder.GetTx()) + if err != nil { + return nil, nil, err + } + + return txBuilder.GetTx(), txBytes, nil +} + +func addUint64Saturating(a, b uint64) uint64 { + if math.MaxUint64-a < b { + return math.MaxUint64 + } + + return a + b +} diff --git a/baseapp/test_helpers.go b/baseapp/test_helpers.go index 003d3e79bd6c..eda2815da410 100644 --- a/baseapp/test_helpers.go +++ b/baseapp/test_helpers.go @@ -49,3 +49,7 @@ func (app *BaseApp) NewContext(isCheckTx bool, header tmproto.Header) sdk.Contex func (app *BaseApp) NewUncachedContext(isCheckTx bool, header tmproto.Header) sdk.Context { return sdk.NewContext(app.cms, header, isCheckTx, app.logger) } + +func (app *BaseApp) GetContextForDeliverTx(txBytes []byte) sdk.Context { + return app.getContextForTx(runTxModeDeliver, txBytes) +} diff --git a/x/auth/ante/sigverify_test.go b/x/auth/ante/sigverify_test.go index 0ddd2d226e0f..074f4c33afc1 100644 --- a/x/auth/ante/sigverify_test.go +++ b/x/auth/ante/sigverify_test.go @@ -192,7 +192,7 @@ func (suite *AnteTestSuite) TestSigVerification() { // In the meantime, we want to make double-sure amino compatibility works. // ref: https://github.com/cosmos/cosmos-sdk/issues/7229 func (suite *AnteTestSuite) TestSigVerification_ExplicitAmino() { - suite.app, suite.ctx = createTestApp(true) + suite.app, suite.ctx = createTestApp(suite.T(), true) suite.ctx = suite.ctx.WithBlockHeight(1) // Set up TxConfig. diff --git a/x/auth/ante/testutil_test.go b/x/auth/ante/testutil_test.go index 74216420eb14..15598b3b23b0 100644 --- a/x/auth/ante/testutil_test.go +++ b/x/auth/ante/testutil_test.go @@ -41,8 +41,8 @@ type AnteTestSuite struct { } // returns context and app with params set on account keeper -func createTestApp(isCheckTx bool) (*simapp.SimApp, sdk.Context) { - app := simapp.Setup(isCheckTx) +func createTestApp(t *testing.T, isCheckTx bool) (*simapp.SimApp, sdk.Context) { + app := simapp.Setup(t, isCheckTx) ctx := app.BaseApp.NewContext(isCheckTx, tmproto.Header{}) app.AccountKeeper.SetParams(ctx, authtypes.DefaultParams()) @@ -51,7 +51,7 @@ func createTestApp(isCheckTx bool) (*simapp.SimApp, sdk.Context) { // SetupTest setups a new test, with new app, context, and anteHandler. func (suite *AnteTestSuite) SetupTest(isCheckTx bool) { - suite.app, suite.ctx = createTestApp(isCheckTx) + suite.app, suite.ctx = createTestApp(suite.T(), isCheckTx) suite.ctx = suite.ctx.WithBlockHeight(1) // Set up TxConfig. From 99c764b1f05dfac91c72344d200b83f99d1d4c44 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Tue, 17 May 2022 18:35:20 +0200 Subject: [PATCH 09/29] Fix amino tests --- baseapp/block_gas_test.go | 1 + x/auth/ante/sigverify.go | 1 + 2 files changed, 2 insertions(+) diff --git a/baseapp/block_gas_test.go b/baseapp/block_gas_test.go index 8f64795bee11..4ed29eb3ba42 100644 --- a/baseapp/block_gas_test.go +++ b/baseapp/block_gas_test.go @@ -165,6 +165,7 @@ func createTestTx(txConfig client.TxConfig, txBuilder client.TxBuilder, privs [] sigsV2 = []signing.SignatureV2{} for i, priv := range privs { signerData := xauthsigning.SignerData{ + Address: sdk.AccAddress(priv.PubKey().Bytes()).String(), ChainID: chainID, AccountNumber: accNums[i], Sequence: accSeqs[i], diff --git a/x/auth/ante/sigverify.go b/x/auth/ante/sigverify.go index 5097478da237..919ee87b997e 100644 --- a/x/auth/ante/sigverify.go +++ b/x/auth/ante/sigverify.go @@ -280,6 +280,7 @@ func (svd SigVerificationDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simul accNum = acc.GetAccountNumber() } signerData := authsigning.SignerData{ + Address: acc.GetAddress().String(), ChainID: chainID, AccountNumber: accNum, Sequence: acc.GetSequence(), From 04c72a30382a79227361fd5f0bb47bd2b30376bc Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Tue, 17 May 2022 19:25:14 +0200 Subject: [PATCH 10/29] Fix test --- baseapp/grpcrouter_test.go | 101 +++++++++++++++++++++++++++++++- baseapp/msg_service_router.go | 2 +- baseapp/queryrouter_test.go | 5 +- baseapp/recovery.go | 2 +- x/auth/ante/setup_test.go | 2 +- x/auth/client/testutil/suite.go | 1 + 6 files changed, 105 insertions(+), 8 deletions(-) diff --git a/baseapp/grpcrouter_test.go b/baseapp/grpcrouter_test.go index a05d3d6616f8..f660b5d1dbe3 100644 --- a/baseapp/grpcrouter_test.go +++ b/baseapp/grpcrouter_test.go @@ -2,6 +2,7 @@ package baseapp_test import ( "context" + "sync" "testing" "github.com/stretchr/testify/require" @@ -12,14 +13,15 @@ import ( "github.com/cosmos/cosmos-sdk/codec/types" "github.com/cosmos/cosmos-sdk/simapp" "github.com/cosmos/cosmos-sdk/testutil/testdata" + "github.com/cosmos/cosmos-sdk/testutil/testdata_pulsar" sdk "github.com/cosmos/cosmos-sdk/types" ) -func TestGRPCGatewayRouter(t *testing.T) { +func TestGRPCQueryRouter(t *testing.T) { qr := baseapp.NewGRPCQueryRouter() interfaceRegistry := testdata.NewTestInterfaceRegistry() qr.SetInterfaceRegistry(interfaceRegistry) - testdata.RegisterQueryServer(qr, testdata.QueryImpl{}) + testdata_pulsar.RegisterQueryServer(qr, testdata_pulsar.QueryImpl{}) helper := &baseapp.QueryServiceTestHelper{ GRPCQueryRouter: qr, Ctx: sdk.Context{}.WithContext(context.Background()), @@ -73,3 +75,98 @@ func TestRegisterQueryServiceTwice(t *testing.T) { ) }) } + +// Tests that we don't have data races per +// https://github.com/cosmos/cosmos-sdk/issues/10324 +// but with the same client connection being used concurrently. +func TestQueryDataRaces_sameConnectionToSameHandler(t *testing.T) { + var mu sync.Mutex + var helper *baseapp.QueryServiceTestHelper + makeClientConn := func(qr *baseapp.GRPCQueryRouter) *baseapp.QueryServiceTestHelper { + mu.Lock() + defer mu.Unlock() + + if helper == nil { + helper = &baseapp.QueryServiceTestHelper{ + GRPCQueryRouter: qr, + Ctx: sdk.Context{}.WithContext(context.Background()), + } + } + return helper + } + testQueryDataRacesSameHandler(t, makeClientConn) +} + +// Tests that we don't have data races per +// https://github.com/cosmos/cosmos-sdk/issues/10324 +// but with unique client connections requesting from the same handler concurrently. +func TestQueryDataRaces_uniqueConnectionsToSameHandler(t *testing.T) { + // Return a new handler for every single call. + testQueryDataRacesSameHandler(t, func(qr *baseapp.GRPCQueryRouter) *baseapp.QueryServiceTestHelper { + return &baseapp.QueryServiceTestHelper{ + GRPCQueryRouter: qr, + Ctx: sdk.Context{}.WithContext(context.Background()), + } + }) +} + +func testQueryDataRacesSameHandler(t *testing.T, makeClientConn func(*baseapp.GRPCQueryRouter) *baseapp.QueryServiceTestHelper) { + t.Parallel() + + qr := baseapp.NewGRPCQueryRouter() + interfaceRegistry := testdata.NewTestInterfaceRegistry() + qr.SetInterfaceRegistry(interfaceRegistry) + testdata.RegisterQueryServer(qr, testdata.QueryImpl{}) + + // The goal is to invoke the router concurrently and check for any data races. + // 0. Run with: go test -race + // 1. Synchronize every one of the 1,000 goroutines waiting to all query at the + // same time. + // 2. Once the greenlight is given, perform a query through the router. + var wg sync.WaitGroup + defer wg.Wait() + + greenlight := make(chan bool) + n := 1000 + ready := make(chan bool, n) + go func() { + for i := 0; i < n; i++ { + <-ready + } + close(greenlight) + }() + + for i := 0; i < n; i++ { + wg.Add(1) + go func() { + defer wg.Done() + + // Wait until we get the green light to start. + ready <- true + <-greenlight + + client := testdata.NewQueryClient(makeClientConn(qr)) + res, err := client.Echo(context.Background(), &testdata.EchoRequest{Message: "hello"}) + require.Nil(t, err) + require.NotNil(t, res) + require.Equal(t, "hello", res.Message) + + require.Panics(t, func() { + _, _ = client.Echo(context.Background(), nil) + }) + + res2, err := client.SayHello(context.Background(), &testdata.SayHelloRequest{Name: "Foo"}) + require.Nil(t, err) + require.NotNil(t, res) + require.Equal(t, "Hello Foo!", res2.Greeting) + + spot := &testdata.Dog{Name: "Spot", Size_: "big"} + any, err := types.NewAnyWithValue(spot) + require.NoError(t, err) + res3, err := client.TestAny(context.Background(), &testdata.TestAnyRequest{AnyAnimal: any}) + require.NoError(t, err) + require.NotNil(t, res3) + require.Equal(t, spot, res3.HasAnimal.Animal.GetCachedValue()) + }() + } +} diff --git a/baseapp/msg_service_router.go b/baseapp/msg_service_router.go index 1b7f8f89bf73..f6c8d4b21ce7 100644 --- a/baseapp/msg_service_router.go +++ b/baseapp/msg_service_router.go @@ -66,7 +66,7 @@ func (msr *MsgServiceRouter) RegisterService(sd *grpc.ServiceDesc, handler inter // We panic here because there is no other alternative and the app cannot be initialized correctly // this should only happen if there is a problem with code generation in which case the app won't // work correctly anyway. - panic(fmt.Errorf("can't register request type %T for service method %s", i, fqMethod)) + panic(fmt.Errorf("unable to register service method %s: %T does not implement sdk.Msg", fqMethod, i)) } requestTypeName = sdk.MsgTypeURL(msg) diff --git a/baseapp/queryrouter_test.go b/baseapp/queryrouter_test.go index 4b38f6458641..c7637f17000e 100644 --- a/baseapp/queryrouter_test.go +++ b/baseapp/queryrouter_test.go @@ -1,4 +1,4 @@ -package baseapp_test +package baseapp import ( "testing" @@ -7,7 +7,6 @@ import ( abci "github.com/tendermint/tendermint/abci/types" - "github.com/cosmos/cosmos-sdk/baseapp" sdk "github.com/cosmos/cosmos-sdk/types" ) @@ -16,7 +15,7 @@ var testQuerier = func(_ sdk.Context, _ []string, _ abci.RequestQuery) ([]byte, } func TestQueryRouter(t *testing.T) { - qr := baseapp.NewQueryRouter() + qr := NewQueryRouter() // require panic on invalid route require.Panics(t, func() { diff --git a/baseapp/recovery.go b/baseapp/recovery.go index 8094d3dcda9c..7f0687800c65 100644 --- a/baseapp/recovery.go +++ b/baseapp/recovery.go @@ -33,7 +33,7 @@ func processRecovery(recoveryObj interface{}, middleware recoveryMiddleware) err return processRecovery(recoveryObj, next) } -// newRecoveryMiddleware creates a RecoveryHandler ante. +// newRecoveryMiddleware creates a RecoveryHandler middleware. func newRecoveryMiddleware(handler RecoveryHandler, next recoveryMiddleware) recoveryMiddleware { return func(recoveryObj interface{}) (recoveryMiddleware, error) { if err := handler(recoveryObj); err != nil { diff --git a/x/auth/ante/setup_test.go b/x/auth/ante/setup_test.go index 4942665cac04..ed4e543b56a4 100644 --- a/x/auth/ante/setup_test.go +++ b/x/auth/ante/setup_test.go @@ -31,7 +31,7 @@ func (suite *AnteTestSuite) TestSetup() { antehandler := sdk.ChainAnteDecorators(sud) // Set height to non-zero value for GasMeter to be set - suite.ctx = suite.ctx.WithBlockHeight(1) + suite.ctx = suite.ctx.WithBlockHeight(1).WithGasMeter(sdk.NewGasMeter(0)) // Context GasMeter Limit not set suite.Require().Equal(uint64(0), suite.ctx.GasMeter().Limit(), "GasMeter set with limit before setup") diff --git a/x/auth/client/testutil/suite.go b/x/auth/client/testutil/suite.go index 40aa258dd5e6..2eba866df8c7 100644 --- a/x/auth/client/testutil/suite.go +++ b/x/auth/client/testutil/suite.go @@ -1562,6 +1562,7 @@ func (s *IntegrationTestSuite) TestAuxSigner() { } func (s *IntegrationTestSuite) TestAuxToFee() { + s.T().Skip() require := s.Require() val := s.network.Validators[0] From 6fd336ec2523a510ab4d34abec2c6646bf49c01e Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Tue, 17 May 2022 19:30:49 +0200 Subject: [PATCH 11/29] Fix block gas test --- baseapp/block_gas_test.go | 10 +++++----- .../adr-045-check-delivertx-middlewares.md | 3 ++- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/baseapp/block_gas_test.go b/baseapp/block_gas_test.go index 4ed29eb3ba42..db933cd7aa72 100644 --- a/baseapp/block_gas_test.go +++ b/baseapp/block_gas_test.go @@ -1,13 +1,13 @@ package baseapp_test import ( - "encoding/json" "fmt" "math" "testing" "github.com/stretchr/testify/require" abci "github.com/tendermint/tendermint/abci/types" + tmjson "github.com/tendermint/tendermint/libs/json" "github.com/tendermint/tendermint/libs/log" tmproto "github.com/tendermint/tendermint/proto/tendermint/types" dbm "github.com/tendermint/tm-db" @@ -67,8 +67,8 @@ func TestBaseApp_BlockGas(t *testing.T) { &testdata.TestMsg{}, ) app = simapp.NewSimApp(log.NewNopLogger(), dbm.NewMemDB(), nil, true, map[int64]bool{}, "", 0, encCfg, simapp.EmptyAppOptions{}, routerOpt) - genState := simapp.NewDefaultGenesisState(encCfg.Codec) - stateBytes, err := json.MarshalIndent(genState, "", " ") + genState := simapp.GenesisStateWithSingleValidator(t, app) + stateBytes, err := tmjson.MarshalIndent(genState, "", " ") require.NoError(t, err) app.InitChain(abci.RequestInitChain{ Validators: []abci.ValidatorUpdate{}, @@ -100,7 +100,7 @@ func TestBaseApp_BlockGas(t *testing.T) { txBuilder.SetFeeAmount(feeAmount) txBuilder.SetGasLimit(txtypes.MaxGasWanted) // tx validation checks that gasLimit can't be bigger than this - privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{6}, []uint64{0} + privs, accNums, accSeqs := []cryptotypes.PrivKey{priv1}, []uint64{7}, []uint64{0} _, txBytes, err := createTestTx(encCfg.TxConfig, txBuilder, privs, accNums, accSeqs, ctx.ChainID()) require.NoError(t, err) @@ -123,7 +123,7 @@ func TestBaseApp_BlockGas(t *testing.T) { require.Equal(t, []byte("ok"), okValue) } // check block gas is always consumed - baseGas := uint64(59142) // baseGas is the gas consumed before tx msg + baseGas := uint64(63724) // baseGas is the gas consumed before tx msg expGasConsumed := addUint64Saturating(tc.gasToConsume, baseGas) if expGasConsumed > txtypes.MaxGasWanted { // capped by gasLimit diff --git a/docs/architecture/adr-045-check-delivertx-middlewares.md b/docs/architecture/adr-045-check-delivertx-middlewares.md index 3ead7e84e5b3..60172977c980 100644 --- a/docs/architecture/adr-045-check-delivertx-middlewares.md +++ b/docs/architecture/adr-045-check-delivertx-middlewares.md @@ -4,10 +4,11 @@ * 20.08.2021: Initial draft. * 07.12.2021: Update `tx.Handler` interface ([\#10693](https://github.com/cosmos/cosmos-sdk/pull/10693)). +* 17.05.2022: ADR is abandoned, as middlewares are deemed too hard to reason about. ## Status -ACCEPTED +ABANDONED. Replacement is being discussed in [#11955](https://github.com/cosmos/cosmos-sdk/issues/11955). ## Abstract From 09fd262cf235af01248611b0c04e311655f3d571 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Tue, 17 May 2022 19:55:39 +0200 Subject: [PATCH 12/29] Small changes --- CHANGELOG.md | 7 +------ baseapp/abci.go | 5 ----- 2 files changed, 1 insertion(+), 11 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7e162a3a105b..457b2127e918 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -138,12 +138,7 @@ Ref: https://keepachangelog.com/en/1.0.0/ * [\#10248](https://github.com/cosmos/cosmos-sdk/pull/10248) Remove unused `KeyPowerReduction` variable from x/staking types. * (x/bank) [\#9832](https://github.com/cosmos/cosmos-sdk/pull/9832) `AddressFromBalancesStore` renamed to `AddressAndDenomFromBalancesStore`. * (tests) [\#9938](https://github.com/cosmos/cosmos-sdk/pull/9938) `simapp.Setup` accepts additional `testing.T` argument. -* (baseapp) [\#9920](https://github.com/cosmos/cosmos-sdk/pull/9920) BaseApp `{Check,Deliver,Simulate}Tx` methods are now replaced by a middleware stack. - * Replace the Antehandler interface with the `tx.Handler` and `tx.Middleware` interfaces. - * Replace `baseapp.SetAnteHandler` with `baseapp.SetTxHandler`. - * Move Msg routers from BaseApp to middlewares. - * Move Baseapp panic recovery into a middleware. - * Rename simulation helper methods `baseapp.{Check,Deliver}` to `baseapp.Sim{Check,Deliver}**`. +* (baseapp) [\#11979](https://github.com/cosmos/cosmos-sdk/pull/11979) Rename baseapp simulation helper methods `baseapp.{Check,Deliver}` to `baseapp.Sim{Check,Deliver}`. * (x/gov) [\#10373](https://github.com/cosmos/cosmos-sdk/pull/10373) Removed gov `keeper.{MustMarshal, MustUnmarshal}`. * [\#10348](https://github.com/cosmos/cosmos-sdk/pull/10348) StdSignBytes takes a new argument of type `*tx.Tip` for signing over tips using LEGACY_AMINO_JSON. * [\#10208](https://github.com/cosmos/cosmos-sdk/pull/10208) The `x/auth/signing.Tx` interface now also includes a new `GetTip() *tx.Tip` method for verifying tipped transactions. The `x/auth/types` expected BankKeeper interface now expects the `SendCoins` method too. diff --git a/baseapp/abci.go b/baseapp/abci.go index 489e0e95f59b..933de5850900 100644 --- a/baseapp/abci.go +++ b/baseapp/abci.go @@ -8,7 +8,6 @@ import ( "sort" "strings" "syscall" - "time" "github.com/gogo/protobuf/proto" abci "github.com/tendermint/tendermint/abci/types" @@ -241,8 +240,6 @@ func (app *BaseApp) EndBlock(req abci.RequestEndBlock) (res abci.ResponseEndBloc // will contain releveant error information. Regardless of tx execution outcome, // the ResponseCheckTx will contain relevant gas execution context. func (app *BaseApp) CheckTx(req abci.RequestCheckTx) abci.ResponseCheckTx { - defer telemetry.MeasureSince(time.Now(), "abci", "check_tx") - var mode runTxMode switch { @@ -276,8 +273,6 @@ func (app *BaseApp) CheckTx(req abci.RequestCheckTx) abci.ResponseCheckTx { // Regardless of tx execution outcome, the ResponseDeliverTx will contain relevant // gas execution context. func (app *BaseApp) DeliverTx(req abci.RequestDeliverTx) abci.ResponseDeliverTx { - defer telemetry.MeasureSince(time.Now(), "abci", "deliver_tx") - gInfo := sdk.GasInfo{} resultStr := "successful" From 6c78ed321f3565ede674d87594837e4bef96bc98 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Wed, 18 May 2022 11:19:14 +0200 Subject: [PATCH 13/29] Add posthandler --- baseapp/baseapp.go | 1 + 1 file changed, 1 insertion(+) diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index fb2110a333c2..0e2978d2a4ec 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -60,6 +60,7 @@ type BaseApp struct { // nolint: maligned txDecoder sdk.TxDecoder // unmarshal []byte into sdk.Tx anteHandler sdk.AnteHandler // ante handler for fee and auth + postHandler sdk.AnteHandler // post handler, optional, e.g. for tips initChainer sdk.InitChainer // initialize state with validators and state blob beginBlocker sdk.BeginBlocker // logic to run before any txs endBlocker sdk.EndBlocker // logic to run after all txs, and to determine valset changes From 9e414ee13b0e1df4bcb30a630b671ce926a423ee Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Wed, 18 May 2022 11:55:18 +0200 Subject: [PATCH 14/29] Add tips, priority --- baseapp/baseapp.go | 11 +++++++ baseapp/options.go | 8 +++++ simapp/app.go | 14 +++++++++ x/auth/ante/ante.go | 1 + x/auth/ante/tx_priority.go | 55 +++++++++++++++++++++++++++++++++ x/auth/client/testutil/suite.go | 1 - x/auth/posthandler/post.go | 27 ++++++++++++++++ x/auth/posthandler/tips.go | 50 ++++++++++++++++++++++++++++++ 8 files changed, 166 insertions(+), 1 deletion(-) create mode 100644 x/auth/ante/tx_priority.go create mode 100644 x/auth/posthandler/post.go create mode 100644 x/auth/posthandler/tips.go diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index 0e2978d2a4ec..4b0af9f97a2f 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -691,6 +691,17 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte) (gInfo sdk.GasInfo, re // Result if any single message fails or does not have a registered Handler. result, err = app.runMsgs(runMsgCtx, msgs, mode) if err == nil && mode == runTxModeDeliver { + // Run optional postHandlers + if app.postHandler != nil { + newCtx, err := app.postHandler(runMsgCtx, tx, mode == runTxModeSimulate) + + if err != nil { + return gInfo, nil, nil, err + } + + result.Events = append(result.Events, newCtx.EventManager().ABCIEvents()...) + } + // When block gas exceeds, it'll panic and won't commit the cached store. consumeBlockGas() diff --git a/baseapp/options.go b/baseapp/options.go index 139e04c8c536..721fb612ffd8 100644 --- a/baseapp/options.go +++ b/baseapp/options.go @@ -153,6 +153,14 @@ func (app *BaseApp) SetAnteHandler(ah sdk.AnteHandler) { app.anteHandler = ah } +func (app *BaseApp) SetPostHandler(ph sdk.AnteHandler) { + if app.sealed { + panic("SetPostHandler() on sealed BaseApp") + } + + app.postHandler = ph +} + func (app *BaseApp) SetAddrPeerFilter(pf sdk.PeerFilter) { if app.sealed { panic("SetAddrPeerFilter() on sealed BaseApp") diff --git a/simapp/app.go b/simapp/app.go index 0fb348432721..f30d4b393ffc 100644 --- a/simapp/app.go +++ b/simapp/app.go @@ -34,6 +34,7 @@ import ( "github.com/cosmos/cosmos-sdk/x/auth" "github.com/cosmos/cosmos-sdk/x/auth/ante" authkeeper "github.com/cosmos/cosmos-sdk/x/auth/keeper" + "github.com/cosmos/cosmos-sdk/x/auth/posthandler" authsims "github.com/cosmos/cosmos-sdk/x/auth/simulation" authtx "github.com/cosmos/cosmos-sdk/x/auth/tx" authtypes "github.com/cosmos/cosmos-sdk/x/auth/types" @@ -476,6 +477,19 @@ func (app *SimApp) setAnteHandler(txConfig client.TxConfig, indexEventsStr []str app.SetAnteHandler(anteHandler) } +func (app *SimApp) setPostHandler() { + postHandler, err := posthandler.NewPostHandler( + posthandler.HandlerOptions{ + BankKeeper: app.BankKeeper, + }, + ) + if err != nil { + panic(err) + } + + app.SetPostHandler(postHandler) +} + // Name returns the name of the App func (app *SimApp) Name() string { return app.BaseApp.Name() } diff --git a/x/auth/ante/ante.go b/x/auth/ante/ante.go index 3b4aa6a56f11..3343e534bfda 100644 --- a/x/auth/ante/ante.go +++ b/x/auth/ante/ante.go @@ -44,6 +44,7 @@ func NewAnteHandler(options HandlerOptions) (sdk.AnteHandler, error) { NewMempoolFeeDecorator(), NewValidateBasicDecorator(), NewTxTimeoutHeightDecorator(), + NewTxPriorityDecorator(), NewValidateMemoDecorator(options.AccountKeeper), NewConsumeGasForTxSizeDecorator(options.AccountKeeper), NewDeductFeeDecorator(options.AccountKeeper, options.BankKeeper, options.FeegrantKeeper), diff --git a/x/auth/ante/tx_priority.go b/x/auth/ante/tx_priority.go new file mode 100644 index 000000000000..4c039a8bd039 --- /dev/null +++ b/x/auth/ante/tx_priority.go @@ -0,0 +1,55 @@ +package ante + +import ( + sdk "github.com/cosmos/cosmos-sdk/types" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" +) + +// txPriorityDecorator implements tx antehandler that determines a +// transaction's priority via a naive mechanism -- the total sum of fees provided. +// It sets the Priority in CheckTx only. +// +// We set the Priority of the transaction +// to be ordered in the Tendermint mempool based naively on the total sum of all +// fees included. Applications that need more sophisticated mempool ordering +// should look to implement their own fee handling middleware instead of using +// TxPriorityHandler. +type txPriorityDecorator struct{} + +func NewTxPriorityDecorator() sdk.AnteDecorator { + return txPriorityDecorator{} +} + +// AnteHandle implements sdk.AnteDecorator. We set the Priority of the transaction +// to be ordered in the Tendermint mempool based naively on the total sum of all +// fees included. Applications that need more sophisticated mempool ordering +// should look to implement their own fee handling middleware instead of using +// TxPriorityHandler. +func (vbd txPriorityDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (sdk.Context, error) { + if !ctx.IsCheckTx() { + return next(ctx, tx, simulate) + } + + feeTx, ok := tx.(sdk.FeeTx) + if !ok { + return ctx, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "Tx must be a FeeTx") + } + + feeCoins := feeTx.GetFee() + + // TODO What's the best way to set the priority on the response? + GetTxPriority(feeCoins) + + return next(ctx, tx, simulate) +} + +// GetTxPriority returns a naive tx priority based on the total sum of all fees +// provided in a transaction. +func GetTxPriority(fee sdk.Coins) int64 { + var priority int64 + for _, c := range fee { + priority += c.Amount.Int64() + } + + return priority +} diff --git a/x/auth/client/testutil/suite.go b/x/auth/client/testutil/suite.go index 2eba866df8c7..40aa258dd5e6 100644 --- a/x/auth/client/testutil/suite.go +++ b/x/auth/client/testutil/suite.go @@ -1562,7 +1562,6 @@ func (s *IntegrationTestSuite) TestAuxSigner() { } func (s *IntegrationTestSuite) TestAuxToFee() { - s.T().Skip() require := s.Require() val := s.network.Validators[0] diff --git a/x/auth/posthandler/post.go b/x/auth/posthandler/post.go new file mode 100644 index 000000000000..604b1203c4e4 --- /dev/null +++ b/x/auth/posthandler/post.go @@ -0,0 +1,27 @@ +package posthandler + +import ( + sdk "github.com/cosmos/cosmos-sdk/types" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" + "github.com/cosmos/cosmos-sdk/x/auth/types" +) + +// HandlerOptions are the options required for constructing a default SDK PostHandler. +type HandlerOptions struct { + BankKeeper types.BankKeeper +} + +// NewAnteHandler returns an AnteHandler that checks and increments sequence +// numbers, checks signatures & account numbers, and deducts fees from the first +// signer. +func NewPostHandler(options HandlerOptions) (sdk.AnteHandler, error) { + if options.BankKeeper == nil { + return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "bank keeper is required for posthandler") + } + + postDecorators := []sdk.AnteDecorator{ + NewTipDecorator(options.BankKeeper), + } + + return sdk.ChainAnteDecorators(postDecorators...), nil +} diff --git a/x/auth/posthandler/tips.go b/x/auth/posthandler/tips.go new file mode 100644 index 000000000000..ac6f5ba39119 --- /dev/null +++ b/x/auth/posthandler/tips.go @@ -0,0 +1,50 @@ +package posthandler + +import ( + sdk "github.com/cosmos/cosmos-sdk/types" + "github.com/cosmos/cosmos-sdk/types/tx" + "github.com/cosmos/cosmos-sdk/x/auth/types" +) + +// ValidateBasicDecorator will call tx.ValidateBasic and return any non-nil error. +// If ValidateBasic passes, decorator calls next AnteHandler in chain. Note, +// ValidateBasicDecorator decorator will not get executed on ReCheckTx since it +// is not dependent on application state. +type tipDecorator struct { + bankKeeper types.BankKeeper +} + +// NewTipDecorator returns a new decorator for handling transactions with +// tips. +func NewTipDecorator(bankKeeper types.BankKeeper) sdk.AnteDecorator { + return tipDecorator{ + bankKeeper: bankKeeper, + } +} + +func (d tipDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (sdk.Context, error) { + err := d.transferTip(ctx, tx) + if err != nil { + return ctx, err + } + + return next(ctx, tx, simulate) +} + +// transferTip transfers the tip from the tipper to the fee payer. +func (d tipDecorator) transferTip(ctx sdk.Context, sdkTx sdk.Tx) error { + tipTx, ok := sdkTx.(tx.TipTx) + + // No-op if the tx doesn't have tips. + if !ok || tipTx.GetTip() == nil { + return nil + } + + tipper, err := sdk.AccAddressFromBech32(tipTx.GetTip().Tipper) + if err != nil { + return err + } + + return d.bankKeeper.SendCoins(ctx, tipper, tipTx.FeePayer(), tipTx.GetTip().Amount) + +} From b07ba755647270cbfd7a35ad6078b28b07d48035 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Wed, 18 May 2022 12:06:11 +0200 Subject: [PATCH 15/29] Add simapp posthandler --- simapp/app.go | 1 + 1 file changed, 1 insertion(+) diff --git a/simapp/app.go b/simapp/app.go index f30d4b393ffc..13cabf4d28ca 100644 --- a/simapp/app.go +++ b/simapp/app.go @@ -445,6 +445,7 @@ func NewSimApp( app.SetBeginBlocker(app.BeginBlocker) app.SetEndBlocker(app.EndBlocker) app.setAnteHandler(encodingConfig.TxConfig, cast.ToStringSlice(appOpts.Get(server.FlagIndexEvents))) + app.setPostHandler() if loadLatest { if err := app.LoadLatestVersion(); err != nil { From c622e9cc9764fb49416c10a83a1d87810fe96672 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Wed, 18 May 2022 12:18:26 +0200 Subject: [PATCH 16/29] put priority under deduct --- x/auth/ante/ante.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x/auth/ante/ante.go b/x/auth/ante/ante.go index 3343e534bfda..0b95f33f9637 100644 --- a/x/auth/ante/ante.go +++ b/x/auth/ante/ante.go @@ -44,10 +44,10 @@ func NewAnteHandler(options HandlerOptions) (sdk.AnteHandler, error) { NewMempoolFeeDecorator(), NewValidateBasicDecorator(), NewTxTimeoutHeightDecorator(), - NewTxPriorityDecorator(), NewValidateMemoDecorator(options.AccountKeeper), NewConsumeGasForTxSizeDecorator(options.AccountKeeper), NewDeductFeeDecorator(options.AccountKeeper, options.BankKeeper, options.FeegrantKeeper), + NewTxPriorityDecorator(), NewSetPubKeyDecorator(options.AccountKeeper), // SetPubKeyDecorator must be called before all signature verification decorators NewValidateSigCountDecorator(options.AccountKeeper), NewSigGasConsumeDecorator(options.AccountKeeper, sigGasConsumer), From 272497579684ffb65c803c68fbcb1ef5602ba8ad Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Wed, 18 May 2022 14:43:58 +0200 Subject: [PATCH 17/29] Fix sigverify --- x/auth/ante/sigverify.go | 1 + 1 file changed, 1 insertion(+) diff --git a/x/auth/ante/sigverify.go b/x/auth/ante/sigverify.go index 919ee87b997e..fa29f0268e1f 100644 --- a/x/auth/ante/sigverify.go +++ b/x/auth/ante/sigverify.go @@ -284,6 +284,7 @@ func (svd SigVerificationDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simul ChainID: chainID, AccountNumber: accNum, Sequence: acc.GetSequence(), + PubKey: pubKey, } if !simulate { From 801a7d6c24041b0236f75a13239919c598fe880a Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Fri, 20 May 2022 14:00:44 +0200 Subject: [PATCH 18/29] Make tips test pass --- x/auth/client/testutil/suite.go | 2 +- x/auth/posthandler/tips.go | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/x/auth/client/testutil/suite.go b/x/auth/client/testutil/suite.go index 40aa258dd5e6..15f88fefab5b 100644 --- a/x/auth/client/testutil/suite.go +++ b/x/auth/client/testutil/suite.go @@ -1734,7 +1734,7 @@ func (s *IntegrationTestSuite) TestAuxToFee() { feePayerArgs: []string{ fmt.Sprintf("--%s=true", flags.FlagSkipConfirmation), fmt.Sprintf("--%s=%s", flags.FlagSignMode, flags.SignModeDirect), - fmt.Sprintf("--%s=%s", flags.FlagBroadcastMode, flags.BroadcastSync), + fmt.Sprintf("--%s=%s", flags.FlagBroadcastMode, flags.BroadcastBlock), fmt.Sprintf("--%s=%s", flags.FlagFrom, feePayer), fmt.Sprintf("--%s=%s", flags.FlagFees, fee.String()), }, diff --git a/x/auth/posthandler/tips.go b/x/auth/posthandler/tips.go index ac6f5ba39119..a112a3e1c4a2 100644 --- a/x/auth/posthandler/tips.go +++ b/x/auth/posthandler/tips.go @@ -46,5 +46,4 @@ func (d tipDecorator) transferTip(ctx sdk.Context, sdkTx sdk.Tx) error { } return d.bankKeeper.SendCoins(ctx, tipper, tipTx.FeePayer(), tipTx.GetTip().Amount) - } From e542ea0513f645b6a573ab5d41dd21b3f7099a1b Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Fri, 20 May 2022 14:16:15 +0200 Subject: [PATCH 19/29] Add extension options --- x/auth/ante/ante.go | 20 ++++++-------- x/auth/ante/ext.go | 49 +++++++++++++++++++++++++++------- x/auth/ante/ext_test.go | 57 ++++++++++++++++++++++++++-------------- x/auth/ante/sigverify.go | 4 +++ 4 files changed, 89 insertions(+), 41 deletions(-) diff --git a/x/auth/ante/ante.go b/x/auth/ante/ante.go index 0b95f33f9637..6a5e7a73a177 100644 --- a/x/auth/ante/ante.go +++ b/x/auth/ante/ante.go @@ -10,11 +10,12 @@ import ( // HandlerOptions are the options required for constructing a default SDK AnteHandler. type HandlerOptions struct { - AccountKeeper AccountKeeper - BankKeeper types.BankKeeper - FeegrantKeeper FeegrantKeeper - SignModeHandler authsigning.SignModeHandler - SigGasConsumer func(meter sdk.GasMeter, sig signing.SignatureV2, params types.Params) error + AccountKeeper AccountKeeper + BankKeeper types.BankKeeper + ExtensionOptionChecker ExtensionOptionChecker + FeegrantKeeper FeegrantKeeper + SignModeHandler authsigning.SignModeHandler + SigGasConsumer func(meter sdk.GasMeter, sig signing.SignatureV2, params types.Params) error } // NewAnteHandler returns an AnteHandler that checks and increments sequence @@ -33,14 +34,9 @@ func NewAnteHandler(options HandlerOptions) (sdk.AnteHandler, error) { return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "sign mode handler is required for ante builder") } - var sigGasConsumer = options.SigGasConsumer - if sigGasConsumer == nil { - sigGasConsumer = DefaultSigVerificationGasConsumer - } - anteDecorators := []sdk.AnteDecorator{ NewSetUpContextDecorator(), // outermost AnteDecorator. SetUpContext must be called first - NewRejectExtensionOptionsDecorator(), + NewExtensionOptionsDecorator(options.ExtensionOptionChecker), NewMempoolFeeDecorator(), NewValidateBasicDecorator(), NewTxTimeoutHeightDecorator(), @@ -50,7 +46,7 @@ func NewAnteHandler(options HandlerOptions) (sdk.AnteHandler, error) { NewTxPriorityDecorator(), NewSetPubKeyDecorator(options.AccountKeeper), // SetPubKeyDecorator must be called before all signature verification decorators NewValidateSigCountDecorator(options.AccountKeeper), - NewSigGasConsumeDecorator(options.AccountKeeper, sigGasConsumer), + NewSigGasConsumeDecorator(options.AccountKeeper, options.SigGasConsumer), NewSigVerificationDecorator(options.AccountKeeper, options.SignModeHandler), NewIncrementSequenceDecorator(options.AccountKeeper), } diff --git a/x/auth/ante/ext.go b/x/auth/ante/ext.go index 362b8d32a971..6a072526aa0f 100644 --- a/x/auth/ante/ext.go +++ b/x/auth/ante/ext.go @@ -2,7 +2,7 @@ package ante import ( codectypes "github.com/cosmos/cosmos-sdk/codec/types" - "github.com/cosmos/cosmos-sdk/types" + sdk "github.com/cosmos/cosmos-sdk/types" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" ) @@ -11,26 +11,55 @@ type HasExtensionOptionsTx interface { GetNonCriticalExtensionOptions() []*codectypes.Any } +// ExtensionOptionChecker is a function that returns true if the extension option is accepted. +type ExtensionOptionChecker func(*codectypes.Any) bool + +// rejectExtensionOption is the default extension check that reject all tx +// extensions. +func rejectExtensionOption(*codectypes.Any) bool { + return false +} + // RejectExtensionOptionsDecorator is an AnteDecorator that rejects all extension // options which can optionally be included in protobuf transactions. Users that // need extension options should create a custom AnteHandler chain that handles // needed extension options properly and rejects unknown ones. -type RejectExtensionOptionsDecorator struct{} +type RejectExtensionOptionsDecorator struct { + checker ExtensionOptionChecker +} + +// NewExtensionOptionsDecorator creates a new antehandler that rejects all extension +// options which can optionally be included in protobuf transactions that don't pass the checker. +// Users that need extension options should pass a custom checker that returns true for the +// needed extension options. +func NewExtensionOptionsDecorator(checker ExtensionOptionChecker) sdk.AnteDecorator { + if checker == nil { + checker = rejectExtensionOption + } -// NewRejectExtensionOptionsDecorator creates a new RejectExtensionOptionsDecorator -func NewRejectExtensionOptionsDecorator() RejectExtensionOptionsDecorator { - return RejectExtensionOptionsDecorator{} + return RejectExtensionOptionsDecorator{checker: checker} } -var _ types.AnteDecorator = RejectExtensionOptionsDecorator{} +var _ sdk.AnteDecorator = RejectExtensionOptionsDecorator{} // AnteHandle implements the AnteDecorator.AnteHandle method -func (r RejectExtensionOptionsDecorator) AnteHandle(ctx types.Context, tx types.Tx, simulate bool, next types.AnteHandler) (newCtx types.Context, err error) { +func (r RejectExtensionOptionsDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (newCtx sdk.Context, err error) { + err = checkExtOpts(tx, r.checker) + if err != nil { + return ctx, err + } + + return next(ctx, tx, simulate) +} + +func checkExtOpts(tx sdk.Tx, checker ExtensionOptionChecker) error { if hasExtOptsTx, ok := tx.(HasExtensionOptionsTx); ok { - if len(hasExtOptsTx.GetExtensionOptions()) != 0 { - return ctx, sdkerrors.ErrUnknownExtensionOptions + for _, opt := range hasExtOptsTx.GetExtensionOptions() { + if !checker(opt) { + return sdkerrors.ErrUnknownExtensionOptions + } } } - return next(ctx, tx, simulate) + return nil } diff --git a/x/auth/ante/ext_test.go b/x/auth/ante/ext_test.go index 89ce6a7d649f..3f91a4a4ca98 100644 --- a/x/auth/ante/ext_test.go +++ b/x/auth/ante/ext_test.go @@ -2,6 +2,7 @@ package ante_test import ( "github.com/cosmos/cosmos-sdk/codec/types" + codectypes "github.com/cosmos/cosmos-sdk/codec/types" "github.com/cosmos/cosmos-sdk/testutil/testdata" sdk "github.com/cosmos/cosmos-sdk/types" "github.com/cosmos/cosmos-sdk/x/auth/ante" @@ -10,27 +11,45 @@ import ( func (suite *AnteTestSuite) TestRejectExtensionOptionsDecorator() { suite.SetupTest(true) // setup - suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() - reod := ante.NewRejectExtensionOptionsDecorator() - antehandler := sdk.ChainAnteDecorators(reod) + testCases := []struct { + msg string + allow bool + }{ + {"allow extension", true}, + {"reject extension", false}, + } + for _, tc := range testCases { + suite.Run(tc.msg, func() { + txBuilder := suite.clientCtx.TxConfig.NewTxBuilder() - // no extension options should not trigger an error - theTx := suite.txBuilder.GetTx() - _, err := antehandler(suite.ctx, theTx, false) - suite.Require().NoError(err) + reod := ante.NewExtensionOptionsDecorator(func(_ *codectypes.Any) bool { + return tc.allow + }) + antehandler := sdk.ChainAnteDecorators(reod) - extOptsTxBldr, ok := suite.txBuilder.(tx.ExtensionOptionsTxBuilder) - if !ok { - // if we can't set extension options, this decorator doesn't apply and we're done - return - } + // no extension options should not trigger an error + theTx := txBuilder.GetTx() + _, err := antehandler(suite.ctx, theTx, false) + suite.Require().NoError(err) - // setting any extension option should cause an error - any, err := types.NewAnyWithValue(testdata.NewTestMsg()) - suite.Require().NoError(err) - extOptsTxBldr.SetExtensionOptions(any) - theTx = suite.txBuilder.GetTx() - _, err = antehandler(suite.ctx, theTx, false) - suite.Require().EqualError(err, "unknown extension options") + extOptsTxBldr, ok := txBuilder.(tx.ExtensionOptionsTxBuilder) + if !ok { + // if we can't set extension options, this decorator doesn't apply and we're done + return + } + + // set an extension option and check + any, err := types.NewAnyWithValue(testdata.NewTestMsg()) + suite.Require().NoError(err) + extOptsTxBldr.SetExtensionOptions(any) + theTx = txBuilder.GetTx() + _, err = antehandler(suite.ctx, theTx, false) + if tc.allow { + suite.Require().NoError(err) + } else { + suite.Require().EqualError(err, "unknown extension options") + } + }) + } } diff --git a/x/auth/ante/sigverify.go b/x/auth/ante/sigverify.go index fa29f0268e1f..b32f46728ff1 100644 --- a/x/auth/ante/sigverify.go +++ b/x/auth/ante/sigverify.go @@ -136,6 +136,10 @@ type SigGasConsumeDecorator struct { } func NewSigGasConsumeDecorator(ak AccountKeeper, sigGasConsumer SignatureVerificationGasConsumer) SigGasConsumeDecorator { + if sigGasConsumer == nil { + sigGasConsumer = DefaultSigVerificationGasConsumer + } + return SigGasConsumeDecorator{ ak: ak, sigGasConsumer: sigGasConsumer, From 8f180cfac8d26590c0a063ce32b6f432ce28dc1b Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Fri, 20 May 2022 14:37:58 +0200 Subject: [PATCH 20/29] Add TxFeeChecker --- CHANGELOG.md | 2 + types/context.go | 8 +++ x/auth/ante/ante.go | 5 +- x/auth/ante/ext_test.go | 3 +- x/auth/ante/fee.go | 107 ++++++++++++-------------------- x/auth/ante/fee_test.go | 4 +- x/auth/ante/feegrant_test.go | 2 +- x/auth/ante/tx_priority.go | 55 ---------------- x/auth/ante/validator_tx_fee.go | 59 ++++++++++++++++++ 9 files changed, 116 insertions(+), 129 deletions(-) delete mode 100644 x/auth/ante/tx_priority.go create mode 100644 x/auth/ante/validator_tx_fee.go diff --git a/CHANGELOG.md b/CHANGELOG.md index c83455d36d14..d422cf82a9e9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -90,6 +90,7 @@ Ref: https://keepachangelog.com/en/1.0.0/ ### API Breaking Changes +* (x/auth/ante) [#11985](https://github.com/cosmos/cosmos-sdk/pull/11985) The `MempoolFeeDecorator` has been removed. Instead, the `DeductFeeDecorator` takes a new argument of type `TxFeeChecker`, to define custom fee models. If `nil` is passed to this `TxFeeChecker` argument, then it will default to `checkTxFeeWithValidatorMinGasPrices`, which is the exact same behavior as the old `MempoolFeeDecorator` (i.e. checking fees against validator's own min gas price). * (crypto/keyring) [#11932](https://github.com/cosmos/cosmos-sdk/pull/11932) Remove `Unsafe*` interfaces from keyring package. Please use interface casting if you wish to access those unsafe functions. * (types) [#11881](https://github.com/cosmos/cosmos-sdk/issues/11881) Rename `AccAddressFromHex` to `AccAddressFromHexUnsafe`. * (types) [#11788](https://github.com/cosmos/cosmos-sdk/pull/11788) The `Int` and `Uint` types have been moved to their own dedicated module, `math`. Aliases are kept in the SDK's root `types` package, however, it is encouraged to utilize the new `math` module. As a result, the `Int#ToDec` API has been removed. @@ -277,6 +278,7 @@ Ref: https://keepachangelog.com/en/1.0.0/ ### State Machine Breaking +* (baseapp) [\#11985](https://github.com/cosmos/cosmos-sdk/pull/11985) Add a `postHandler` to baseapp. This `postHandler` is like antehandler, but is run _after_ the `runMsgs` execution. It is in the same store branch that `runMsgs`, meaning that both `runMsgs` and `postHandler` state will be committed only if both are successful, and both will be reverted if any of the two fails. * (x/upgrade) [\#11800](https://github.com/cosmos/cosmos-sdk/pull/11800) Fix `GetLastCompleteUpgrade` to properly return the latest upgrade. * [\#10564](https://github.com/cosmos/cosmos-sdk/pull/10564) Fix bug when updating allowance inside AllowedMsgAllowance * (x/auth)[\#9596](https://github.com/cosmos/cosmos-sdk/pull/9596) Enable creating periodic vesting accounts with a transactions instead of requiring them to be created in genesis. diff --git a/types/context.go b/types/context.go index 30bec2154f5d..ec5f2c3774dd 100644 --- a/types/context.go +++ b/types/context.go @@ -38,6 +38,7 @@ type Context struct { minGasPrice DecCoins consParams *tmproto.ConsensusParams eventManager *EventManager + priority int64 // The tx priority, only relevant in CheckTx } // Proposed rename, not done to avoid API breakage @@ -58,6 +59,7 @@ func (c Context) IsCheckTx() bool { return c.checkTx } func (c Context) IsReCheckTx() bool { return c.recheckTx } func (c Context) MinGasPrices() DecCoins { return c.minGasPrice } func (c Context) EventManager() *EventManager { return c.eventManager } +func (c Context) Priority() int64 { return c.priority } // clone the header before returning func (c Context) BlockHeader() tmproto.Header { @@ -226,6 +228,12 @@ func (c Context) WithEventManager(em *EventManager) Context { return c } +// WithEventManager returns a Context with an updated tx priority +func (c Context) WithPriority(p int64) Context { + c.priority = p + return c +} + // TODO: remove??? func (c Context) IsZero() bool { return c.ms == nil diff --git a/x/auth/ante/ante.go b/x/auth/ante/ante.go index 6a5e7a73a177..a562e67ee39c 100644 --- a/x/auth/ante/ante.go +++ b/x/auth/ante/ante.go @@ -16,6 +16,7 @@ type HandlerOptions struct { FeegrantKeeper FeegrantKeeper SignModeHandler authsigning.SignModeHandler SigGasConsumer func(meter sdk.GasMeter, sig signing.SignatureV2, params types.Params) error + TxFeeChecker TxFeeChecker } // NewAnteHandler returns an AnteHandler that checks and increments sequence @@ -37,13 +38,11 @@ func NewAnteHandler(options HandlerOptions) (sdk.AnteHandler, error) { anteDecorators := []sdk.AnteDecorator{ NewSetUpContextDecorator(), // outermost AnteDecorator. SetUpContext must be called first NewExtensionOptionsDecorator(options.ExtensionOptionChecker), - NewMempoolFeeDecorator(), NewValidateBasicDecorator(), NewTxTimeoutHeightDecorator(), NewValidateMemoDecorator(options.AccountKeeper), NewConsumeGasForTxSizeDecorator(options.AccountKeeper), - NewDeductFeeDecorator(options.AccountKeeper, options.BankKeeper, options.FeegrantKeeper), - NewTxPriorityDecorator(), + NewDeductFeeDecorator(options.AccountKeeper, options.BankKeeper, options.FeegrantKeeper, options.TxFeeChecker), NewSetPubKeyDecorator(options.AccountKeeper), // SetPubKeyDecorator must be called before all signature verification decorators NewValidateSigCountDecorator(options.AccountKeeper), NewSigGasConsumeDecorator(options.AccountKeeper, options.SigGasConsumer), diff --git a/x/auth/ante/ext_test.go b/x/auth/ante/ext_test.go index 3f91a4a4ca98..3bd0f8f60275 100644 --- a/x/auth/ante/ext_test.go +++ b/x/auth/ante/ext_test.go @@ -1,7 +1,6 @@ package ante_test import ( - "github.com/cosmos/cosmos-sdk/codec/types" codectypes "github.com/cosmos/cosmos-sdk/codec/types" "github.com/cosmos/cosmos-sdk/testutil/testdata" sdk "github.com/cosmos/cosmos-sdk/types" @@ -40,7 +39,7 @@ func (suite *AnteTestSuite) TestRejectExtensionOptionsDecorator() { } // set an extension option and check - any, err := types.NewAnyWithValue(testdata.NewTestMsg()) + any, err := codectypes.NewAnyWithValue(testdata.NewTestMsg()) suite.Require().NoError(err) extOptsTxBldr.SetExtensionOptions(any) theTx = txBuilder.GetTx() diff --git a/x/auth/ante/fee.go b/x/auth/ante/fee.go index 19e8258cfa73..7a38dbc399d6 100644 --- a/x/auth/ante/fee.go +++ b/x/auth/ante/fee.go @@ -8,121 +8,96 @@ import ( "github.com/cosmos/cosmos-sdk/x/auth/types" ) -// MempoolFeeDecorator will check if the transaction's fee is at least as large -// as the local validator's minimum gasFee (defined in validator config). -// If fee is too low, decorator returns error and tx is rejected from mempool. -// Note this only applies when ctx.CheckTx = true -// If fee is high enough or not CheckTx, then call next AnteHandler -// CONTRACT: Tx must implement FeeTx to use MempoolFeeDecorator -type MempoolFeeDecorator struct{} - -func NewMempoolFeeDecorator() MempoolFeeDecorator { - return MempoolFeeDecorator{} -} - -func (mfd MempoolFeeDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (newCtx sdk.Context, err error) { - feeTx, ok := tx.(sdk.FeeTx) - if !ok { - return ctx, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "Tx must be a FeeTx") - } - - feeCoins := feeTx.GetFee() - gas := feeTx.GetGas() - - // Ensure that the provided fees meet a minimum threshold for the validator, - // if this is a CheckTx. This is only for local mempool purposes, and thus - // is only ran on check tx. - if ctx.IsCheckTx() && !simulate { - minGasPrices := ctx.MinGasPrices() - if !minGasPrices.IsZero() { - requiredFees := make(sdk.Coins, len(minGasPrices)) - - // Determine the required fees by multiplying each required minimum gas - // price by the gas limit, where fee = ceil(minGasPrice * gasLimit). - glDec := sdk.NewDec(int64(gas)) - for i, gp := range minGasPrices { - fee := gp.Amount.Mul(glDec) - requiredFees[i] = sdk.NewCoin(gp.Denom, fee.Ceil().RoundInt()) - } - - if !feeCoins.IsAnyGTE(requiredFees) { - return ctx, sdkerrors.Wrapf(sdkerrors.ErrInsufficientFee, "insufficient fees; got: %s required: %s", feeCoins, requiredFees) - } - } - } - - return next(ctx, tx, simulate) -} +// TxFeeChecker check if the provided fee is enough and returns the effective fee and tx priority, +// the effective fee should be deducted later, and the priority should be returned in abci response. +type TxFeeChecker func(ctx sdk.Context, tx sdk.Tx) (sdk.Coins, int64, error) // DeductFeeDecorator deducts fees from the first signer of the tx // If the first signer does not have the funds to pay for the fees, return with InsufficientFunds error // Call next AnteHandler if fees successfully deducted // CONTRACT: Tx must implement FeeTx interface to use DeductFeeDecorator type DeductFeeDecorator struct { - ak AccountKeeper + accountKeeper AccountKeeper bankKeeper types.BankKeeper feegrantKeeper FeegrantKeeper + txFeeChecker TxFeeChecker } -func NewDeductFeeDecorator(ak AccountKeeper, bk types.BankKeeper, fk FeegrantKeeper) DeductFeeDecorator { +func NewDeductFeeDecorator(ak AccountKeeper, bk types.BankKeeper, fk FeegrantKeeper, tfc TxFeeChecker) DeductFeeDecorator { + if tfc == nil { + tfc = checkTxFeeWithValidatorMinGasPrices + } + return DeductFeeDecorator{ - ak: ak, + accountKeeper: ak, bankKeeper: bk, feegrantKeeper: fk, + txFeeChecker: tfc, + } +} + +func (dfd DeductFeeDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (sdk.Context, error) { + fee, priority, err := dfd.txFeeChecker(ctx, tx) + if err != nil { + return ctx, err } + if err := dfd.checkDeductFee(ctx, tx, fee); err != nil { + return ctx, err + } + + newCtx := ctx.WithPriority(priority) + + return next(newCtx, tx, simulate) } -func (dfd DeductFeeDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (newCtx sdk.Context, err error) { - feeTx, ok := tx.(sdk.FeeTx) +func (dfd DeductFeeDecorator) checkDeductFee(ctx sdk.Context, sdkTx sdk.Tx, fee sdk.Coins) error { + feeTx, ok := sdkTx.(sdk.FeeTx) if !ok { - return ctx, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "Tx must be a FeeTx") + return sdkerrors.Wrap(sdkerrors.ErrTxDecode, "Tx must be a FeeTx") } - if addr := dfd.ak.GetModuleAddress(types.FeeCollectorName); addr == nil { - return ctx, fmt.Errorf("Fee collector module account (%s) has not been set", types.FeeCollectorName) + if addr := dfd.accountKeeper.GetModuleAddress(types.FeeCollectorName); addr == nil { + return fmt.Errorf("Fee collector module account (%s) has not been set", types.FeeCollectorName) } - fee := feeTx.GetFee() feePayer := feeTx.FeePayer() feeGranter := feeTx.FeeGranter() - deductFeesFrom := feePayer // if feegranter set deduct fee from feegranter account. // this works with only when feegrant enabled. if feeGranter != nil { if dfd.feegrantKeeper == nil { - return ctx, sdkerrors.Wrap(sdkerrors.ErrInvalidRequest, "fee grants are not enabled") + return sdkerrors.ErrInvalidRequest.Wrap("fee grants are not enabled") } else if !feeGranter.Equals(feePayer) { - err := dfd.feegrantKeeper.UseGrantedFees(ctx, feeGranter, feePayer, fee, tx.GetMsgs()) - + err := dfd.feegrantKeeper.UseGrantedFees(ctx, feeGranter, feePayer, fee, sdkTx.GetMsgs()) if err != nil { - return ctx, sdkerrors.Wrapf(err, "%s not allowed to pay fees from %s", feeGranter, feePayer) + return sdkerrors.Wrapf(err, "%s does not not allow to pay fees for %s", feeGranter, feePayer) } } deductFeesFrom = feeGranter } - deductFeesFromAcc := dfd.ak.GetAccount(ctx, deductFeesFrom) + deductFeesFromAcc := dfd.accountKeeper.GetAccount(ctx, deductFeesFrom) if deductFeesFromAcc == nil { - return ctx, sdkerrors.Wrapf(sdkerrors.ErrUnknownAddress, "fee payer address: %s does not exist", deductFeesFrom) + return sdkerrors.ErrUnknownAddress.Wrapf("fee payer address: %s does not exist", deductFeesFrom) } // deduct the fees - if !feeTx.GetFee().IsZero() { - err = DeductFees(dfd.bankKeeper, ctx, deductFeesFromAcc, feeTx.GetFee()) + if !fee.IsZero() { + err := DeductFees(dfd.bankKeeper, ctx, deductFeesFromAcc, fee) if err != nil { - return ctx, err + return err } } events := sdk.Events{sdk.NewEvent(sdk.EventTypeTx, - sdk.NewAttribute(sdk.AttributeKeyFee, feeTx.GetFee().String()), + sdk.NewAttribute(sdk.AttributeKeyFee, fee.String()), )} ctx.EventManager().EmitEvents(events) - return next(ctx, tx, simulate) + return nil } // DeductFees deducts fees from the given account. diff --git a/x/auth/ante/fee_test.go b/x/auth/ante/fee_test.go index 06ccb4d3948f..7f5405359e56 100644 --- a/x/auth/ante/fee_test.go +++ b/x/auth/ante/fee_test.go @@ -12,7 +12,7 @@ func (suite *AnteTestSuite) TestEnsureMempoolFees() { suite.SetupTest(true) // setup suite.txBuilder = suite.clientCtx.TxConfig.NewTxBuilder() - mfd := ante.NewMempoolFeeDecorator() + mfd := ante.NewDeductFeeDecorator(suite.app.AccountKeeper, suite.app.BankKeeper, suite.app.FeeGrantKeeper, nil) antehandler := sdk.ChainAnteDecorators(mfd) // keys and addresses @@ -86,7 +86,7 @@ func (suite *AnteTestSuite) TestDeductFees() { err = testutil.FundAccount(suite.app.BankKeeper, suite.ctx, addr1, coins) suite.Require().NoError(err) - dfd := ante.NewDeductFeeDecorator(suite.app.AccountKeeper, suite.app.BankKeeper, nil) + dfd := ante.NewDeductFeeDecorator(suite.app.AccountKeeper, suite.app.BankKeeper, nil, nil) antehandler := sdk.ChainAnteDecorators(dfd) _, err = antehandler(suite.ctx, tx, false) diff --git a/x/auth/ante/feegrant_test.go b/x/auth/ante/feegrant_test.go index c88cf781eb3a..c8689ca3c7a1 100644 --- a/x/auth/ante/feegrant_test.go +++ b/x/auth/ante/feegrant_test.go @@ -32,7 +32,7 @@ func (suite *AnteTestSuite) TestDeductFeesNoDelegation() { protoTxCfg := tx.NewTxConfig(codec.NewProtoCodec(app.InterfaceRegistry()), tx.DefaultSignModes) // this just tests our handler - dfd := ante.NewDeductFeeDecorator(app.AccountKeeper, app.BankKeeper, app.FeeGrantKeeper) + dfd := ante.NewDeductFeeDecorator(app.AccountKeeper, app.BankKeeper, app.FeeGrantKeeper, nil) feeAnteHandler := sdk.ChainAnteDecorators(dfd) // this tests the whole stack diff --git a/x/auth/ante/tx_priority.go b/x/auth/ante/tx_priority.go deleted file mode 100644 index 4c039a8bd039..000000000000 --- a/x/auth/ante/tx_priority.go +++ /dev/null @@ -1,55 +0,0 @@ -package ante - -import ( - sdk "github.com/cosmos/cosmos-sdk/types" - sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" -) - -// txPriorityDecorator implements tx antehandler that determines a -// transaction's priority via a naive mechanism -- the total sum of fees provided. -// It sets the Priority in CheckTx only. -// -// We set the Priority of the transaction -// to be ordered in the Tendermint mempool based naively on the total sum of all -// fees included. Applications that need more sophisticated mempool ordering -// should look to implement their own fee handling middleware instead of using -// TxPriorityHandler. -type txPriorityDecorator struct{} - -func NewTxPriorityDecorator() sdk.AnteDecorator { - return txPriorityDecorator{} -} - -// AnteHandle implements sdk.AnteDecorator. We set the Priority of the transaction -// to be ordered in the Tendermint mempool based naively on the total sum of all -// fees included. Applications that need more sophisticated mempool ordering -// should look to implement their own fee handling middleware instead of using -// TxPriorityHandler. -func (vbd txPriorityDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (sdk.Context, error) { - if !ctx.IsCheckTx() { - return next(ctx, tx, simulate) - } - - feeTx, ok := tx.(sdk.FeeTx) - if !ok { - return ctx, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "Tx must be a FeeTx") - } - - feeCoins := feeTx.GetFee() - - // TODO What's the best way to set the priority on the response? - GetTxPriority(feeCoins) - - return next(ctx, tx, simulate) -} - -// GetTxPriority returns a naive tx priority based on the total sum of all fees -// provided in a transaction. -func GetTxPriority(fee sdk.Coins) int64 { - var priority int64 - for _, c := range fee { - priority += c.Amount.Int64() - } - - return priority -} diff --git a/x/auth/ante/validator_tx_fee.go b/x/auth/ante/validator_tx_fee.go new file mode 100644 index 000000000000..2eee3850c7a7 --- /dev/null +++ b/x/auth/ante/validator_tx_fee.go @@ -0,0 +1,59 @@ +package ante + +import ( + "math" + + sdk "github.com/cosmos/cosmos-sdk/types" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" +) + +// checkTxFeeWithValidatorMinGasPrices implements the default fee logic, where the minimum price per +// unit of gas is fixed and set by each validator, can the tx priority is computed from the gas price. +func checkTxFeeWithValidatorMinGasPrices(ctx sdk.Context, tx sdk.Tx) (sdk.Coins, int64, error) { + feeTx, ok := tx.(sdk.FeeTx) + if !ok { + return nil, 0, sdkerrors.Wrap(sdkerrors.ErrTxDecode, "Tx must be a FeeTx") + } + + feeCoins := feeTx.GetFee() + gas := feeTx.GetGas() + + // Ensure that the provided fees meet a minimum threshold for the validator, + // This is only for local mempool purposes, if this is a DeliverTx, the `MinGasPrices` should be zero. + minGasPrices := ctx.MinGasPrices() + if !minGasPrices.IsZero() { + requiredFees := make(sdk.Coins, len(minGasPrices)) + + // Determine the required fees by multiplying each required minimum gas + // price by the gas limit, where fee = ceil(minGasPrice * gasLimit). + glDec := sdk.NewDec(int64(gas)) + for i, gp := range minGasPrices { + fee := gp.Amount.Mul(glDec) + requiredFees[i] = sdk.NewCoin(gp.Denom, fee.Ceil().RoundInt()) + } + + if !feeCoins.IsAnyGTE(requiredFees) { + return nil, 0, sdkerrors.Wrapf(sdkerrors.ErrInsufficientFee, "insufficient fees; got: %s required: %s", feeCoins, requiredFees) + } + } + + priority := getTxPriority(feeCoins) + return feeCoins, priority, nil +} + +// getTxPriority returns a naive tx priority based on the amount of the smallest denomination of the fee +// provided in a transaction. +func getTxPriority(fee sdk.Coins) int64 { + var priority int64 + for _, c := range fee { + p := int64(math.MaxInt64) + if c.Amount.IsInt64() { + p = c.Amount.Int64() + } + if priority == 0 || p < priority { + priority = p + } + } + + return priority +} From a8c8ce844bd42fb82e8dfe37250a32b703252357 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Fri, 20 May 2022 14:44:07 +0200 Subject: [PATCH 21/29] Add comment --- simapp/app.go | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/simapp/app.go b/simapp/app.go index 4b77fb7fcd18..0cddbebcf2ea 100644 --- a/simapp/app.go +++ b/simapp/app.go @@ -444,7 +444,24 @@ func NewSimApp( app.SetBeginBlocker(app.BeginBlocker) app.SetEndBlocker(app.EndBlocker) app.setAnteHandler(encodingConfig.TxConfig, cast.ToStringSlice(appOpts.Get(server.FlagIndexEvents))) - app.setPostHandler() + // In v0.46, the SDK introduces _postHandlers_. PostHandlers are like + // antehandlers, but are run _after_ the `runMsgs` execution. They are also + // defined as a chain, and have the same signature as antehandlers. + // + // In baseapp, postHandlers are run in the same store branch as `runMsgs`, + // meaning that both `runMsgs` and `postHandler` state will be committed if + // both are successful, and both will be reverted if any of the two fails. + // + // The SDK exposes a default postHandlers chain, which comprises of only + // one decorator: the Transaction Tips decorator. However, some chains do + // not need it by default, so the following line is commented out. You can + // uncomment it to include the tips decorator, or define your own + // postHandler chain. Please note that changing any of the anteHandler or + // postHandler chain is likely to be a state-machine breaking change, which + // needs a coordinated upgrade. + // + // Uncomment to enable postHandlers: + // app.setPostHandler() if loadLatest { if err := app.LoadLatestVersion(); err != nil { From 417cee3aa9d7246c15686cc53d543e3d197f7619 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Fri, 20 May 2022 15:09:41 +0200 Subject: [PATCH 22/29] Fix fee test --- CHANGELOG.md | 3 ++- x/auth/ante/fee_test.go | 2 ++ x/auth/ante/validator_tx_fee.go | 33 ++++++++++++++++++--------------- 3 files changed, 22 insertions(+), 16 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index d422cf82a9e9..40f0073e4001 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -39,6 +39,7 @@ Ref: https://keepachangelog.com/en/1.0.0/ ### Features +* (types) [#11985](https://github.com/cosmos/cosmos-sdk/pull/11985) Add a `Priority` field on `sdk.Context`, which represents the CheckTx priority field. It is only used during CheckTx. * (gRPC) [#11889](https://github.com/cosmos/cosmos-sdk/pull/11889) Support custom read and write gRPC options in `app.toml`. See `max-recv-msg-size` and `max-send-msg-size` respectively. * (cli) [\#11738](https://github.com/cosmos/cosmos-sdk/pull/11738) Add `tx auth multi-sign` as alias of `tx auth multisign` for consistency with `multi-send`. * (cli) [\#11738](https://github.com/cosmos/cosmos-sdk/pull/11738) Add `tx bank multi-send` command for bulk send of coins to multiple accounts. @@ -278,7 +279,7 @@ Ref: https://keepachangelog.com/en/1.0.0/ ### State Machine Breaking -* (baseapp) [\#11985](https://github.com/cosmos/cosmos-sdk/pull/11985) Add a `postHandler` to baseapp. This `postHandler` is like antehandler, but is run _after_ the `runMsgs` execution. It is in the same store branch that `runMsgs`, meaning that both `runMsgs` and `postHandler` state will be committed only if both are successful, and both will be reverted if any of the two fails. +* (baseapp) [\#11985](https://github.com/cosmos/cosmos-sdk/pull/11985) Add a `postHandler` to baseapp. This `postHandler` is like antehandler, but is run _after_ the `runMsgs` execution. It is in the same store branch that `runMsgs`, meaning that both `runMsgs` and `postHandler` * (x/upgrade) [\#11800](https://github.com/cosmos/cosmos-sdk/pull/11800) Fix `GetLastCompleteUpgrade` to properly return the latest upgrade. * [\#10564](https://github.com/cosmos/cosmos-sdk/pull/10564) Fix bug when updating allowance inside AllowedMsgAllowance * (x/auth)[\#9596](https://github.com/cosmos/cosmos-sdk/pull/9596) Enable creating periodic vesting accounts with a transactions instead of requiring them to be created in genesis. diff --git a/x/auth/ante/fee_test.go b/x/auth/ante/fee_test.go index 7f5405359e56..91d322c1deec 100644 --- a/x/auth/ante/fee_test.go +++ b/x/auth/ante/fee_test.go @@ -17,6 +17,8 @@ func (suite *AnteTestSuite) TestEnsureMempoolFees() { // keys and addresses priv1, _, addr1 := testdata.KeyTestPubAddr() + coins := sdk.NewCoins(sdk.NewCoin("atom", sdk.NewInt(300))) + testutil.FundAccount(suite.app.BankKeeper, suite.ctx, addr1, coins) // msg and signatures msg := testdata.NewTestMsg(addr1) diff --git a/x/auth/ante/validator_tx_fee.go b/x/auth/ante/validator_tx_fee.go index 2eee3850c7a7..b1725d62ddc0 100644 --- a/x/auth/ante/validator_tx_fee.go +++ b/x/auth/ante/validator_tx_fee.go @@ -19,21 +19,24 @@ func checkTxFeeWithValidatorMinGasPrices(ctx sdk.Context, tx sdk.Tx) (sdk.Coins, gas := feeTx.GetGas() // Ensure that the provided fees meet a minimum threshold for the validator, - // This is only for local mempool purposes, if this is a DeliverTx, the `MinGasPrices` should be zero. - minGasPrices := ctx.MinGasPrices() - if !minGasPrices.IsZero() { - requiredFees := make(sdk.Coins, len(minGasPrices)) - - // Determine the required fees by multiplying each required minimum gas - // price by the gas limit, where fee = ceil(minGasPrice * gasLimit). - glDec := sdk.NewDec(int64(gas)) - for i, gp := range minGasPrices { - fee := gp.Amount.Mul(glDec) - requiredFees[i] = sdk.NewCoin(gp.Denom, fee.Ceil().RoundInt()) - } - - if !feeCoins.IsAnyGTE(requiredFees) { - return nil, 0, sdkerrors.Wrapf(sdkerrors.ErrInsufficientFee, "insufficient fees; got: %s required: %s", feeCoins, requiredFees) + // if this is a CheckTx. This is only for local mempool purposes, and thus + // is only ran on check tx. + if ctx.IsCheckTx() { + minGasPrices := ctx.MinGasPrices() + if !minGasPrices.IsZero() { + requiredFees := make(sdk.Coins, len(minGasPrices)) + + // Determine the required fees by multiplying each required minimum gas + // price by the gas limit, where fee = ceil(minGasPrice * gasLimit). + glDec := sdk.NewDec(int64(gas)) + for i, gp := range minGasPrices { + fee := gp.Amount.Mul(glDec) + requiredFees[i] = sdk.NewCoin(gp.Denom, fee.Ceil().RoundInt()) + } + + if !feeCoins.IsAnyGTE(requiredFees) { + return nil, 0, sdkerrors.Wrapf(sdkerrors.ErrInsufficientFee, "insufficient fees; got: %s required: %s", feeCoins, requiredFees) + } } } From 5f96e319ee6c2c06137f79af378e3538dc7a9435 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Fri, 20 May 2022 15:15:24 +0200 Subject: [PATCH 23/29] Add changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 40f0073e4001..2100e71c992b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -92,6 +92,7 @@ Ref: https://keepachangelog.com/en/1.0.0/ ### API Breaking Changes * (x/auth/ante) [#11985](https://github.com/cosmos/cosmos-sdk/pull/11985) The `MempoolFeeDecorator` has been removed. Instead, the `DeductFeeDecorator` takes a new argument of type `TxFeeChecker`, to define custom fee models. If `nil` is passed to this `TxFeeChecker` argument, then it will default to `checkTxFeeWithValidatorMinGasPrices`, which is the exact same behavior as the old `MempoolFeeDecorator` (i.e. checking fees against validator's own min gas price). +* (x/auth/ante) [#11985](https://github.com/cosmos/cosmos-sdk/pull/11985) The `ExtensionOptionsDecorator` takes an argument of type `ExtensionOptionChecker`. For backwards-compatibility, you can pass `nil`, which defaults to the old behavior of rejecting all tx extensions. * (crypto/keyring) [#11932](https://github.com/cosmos/cosmos-sdk/pull/11932) Remove `Unsafe*` interfaces from keyring package. Please use interface casting if you wish to access those unsafe functions. * (types) [#11881](https://github.com/cosmos/cosmos-sdk/issues/11881) Rename `AccAddressFromHex` to `AccAddressFromHexUnsafe`. * (types) [#11788](https://github.com/cosmos/cosmos-sdk/pull/11788) The `Int` and `Uint` types have been moved to their own dedicated module, `math`. Aliases are kept in the SDK's root `types` package, however, it is encouraged to utilize the new `math` module. As a result, the `Int#ToDec` API has been removed. From d48d2536b6fc908e7095e852b76bd14c8a44274e Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Fri, 20 May 2022 16:56:14 +0200 Subject: [PATCH 24/29] Add comment on posthandler --- baseapp/baseapp.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index 08c392f699d8..106c56d7a036 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -689,10 +689,11 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte) (gInfo sdk.GasInfo, re // Result if any single message fails or does not have a registered Handler. result, err = app.runMsgs(runMsgCtx, msgs, mode) if err == nil && mode == runTxModeDeliver { - // Run optional postHandlers + // Run optional postHandlers. + // Important: if the postHandler fails, we also revert the runMsgs + // state. if app.postHandler != nil { newCtx, err := app.postHandler(runMsgCtx, tx, mode == runTxModeSimulate) - if err != nil { return gInfo, nil, nil, err } From 46e21028a546d740d7b7a52760345755638e0d24 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Fri, 20 May 2022 16:59:32 +0200 Subject: [PATCH 25/29] Run postHandler on simulate --- baseapp/baseapp.go | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index 106c56d7a036..99bce452fb7a 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -688,7 +688,7 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte) (gInfo sdk.GasInfo, re // and we're in DeliverTx. Note, runMsgs will never return a reference to a // Result if any single message fails or does not have a registered Handler. result, err = app.runMsgs(runMsgCtx, msgs, mode) - if err == nil && mode == runTxModeDeliver { + if err == nil { // Run optional postHandlers. // Important: if the postHandler fails, we also revert the runMsgs // state. @@ -701,14 +701,16 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte) (gInfo sdk.GasInfo, re result.Events = append(result.Events, newCtx.EventManager().ABCIEvents()...) } - // When block gas exceeds, it'll panic and won't commit the cached store. - consumeBlockGas() + if mode == runTxModeDeliver { + // When block gas exceeds, it'll panic and won't commit the cached store. + consumeBlockGas() - msCache.Write() + msCache.Write() - if len(anteEvents) > 0 { - // append the events in the order of occurrence - result.Events = append(anteEvents, result.Events...) + if len(anteEvents) > 0 { + // append the events in the order of occurrence + result.Events = append(anteEvents, result.Events...) + } } } From aff7e0148901d06724b5a3621790194e4d748bed Mon Sep 17 00:00:00 2001 From: Amaury <1293565+amaurym@users.noreply.github.com> Date: Fri, 20 May 2022 17:16:29 +0200 Subject: [PATCH 26/29] Update baseapp/baseapp.go Co-authored-by: Aleksandr Bezobchuk --- baseapp/baseapp.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index 99bce452fb7a..7d6e90fbf0a2 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -690,8 +690,8 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte) (gInfo sdk.GasInfo, re result, err = app.runMsgs(runMsgCtx, msgs, mode) if err == nil { // Run optional postHandlers. - // Important: if the postHandler fails, we also revert the runMsgs - // state. + // + // Note: If the postHandler fails, we also revert the runMsgs state. if app.postHandler != nil { newCtx, err := app.postHandler(runMsgCtx, tx, mode == runTxModeSimulate) if err != nil { From 35866ff1c83ced52ee138645ae4a9dc536d04227 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Fri, 20 May 2022 17:34:52 +0200 Subject: [PATCH 27/29] Add priority back to checkTx --- baseapp/abci.go | 5 +++-- baseapp/baseapp.go | 15 ++++++++------- baseapp/baseapp_test.go | 7 +++++++ baseapp/test_helpers.go | 6 +++--- x/auth/ante/fee_test.go | 5 ++++- 5 files changed, 25 insertions(+), 13 deletions(-) diff --git a/baseapp/abci.go b/baseapp/abci.go index 9b0b971343ac..1c7a36927e8b 100644 --- a/baseapp/abci.go +++ b/baseapp/abci.go @@ -251,7 +251,7 @@ func (app *BaseApp) CheckTx(req abci.RequestCheckTx) abci.ResponseCheckTx { panic(fmt.Sprintf("unknown RequestCheckTx type: %s", req.Type)) } - gInfo, result, anteEvents, err := app.runTx(mode, req.Tx) + gInfo, result, anteEvents, priority, err := app.runTx(mode, req.Tx) if err != nil { return sdkerrors.ResponseCheckTxWithEvents(err, gInfo.GasWanted, gInfo.GasUsed, anteEvents, app.trace) } @@ -262,6 +262,7 @@ func (app *BaseApp) CheckTx(req abci.RequestCheckTx) abci.ResponseCheckTx { Log: result.Log, Data: result.Data, Events: sdk.MarkEventsToIndex(result.Events, app.indexEvents), + Priority: priority, } } @@ -281,7 +282,7 @@ func (app *BaseApp) DeliverTx(req abci.RequestDeliverTx) abci.ResponseDeliverTx telemetry.SetGauge(float32(gInfo.GasWanted), "tx", "gas", "wanted") }() - gInfo, result, anteEvents, err := app.runTx(runTxModeDeliver, req.Tx) + gInfo, result, anteEvents, _, err := app.runTx(runTxModeDeliver, req.Tx) if err != nil { resultStr = "failed" return sdkerrors.ResponseDeliverTxWithEvents(err, gInfo.GasWanted, gInfo.GasUsed, anteEvents, app.trace) diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index 99bce452fb7a..f894173ab90f 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -585,7 +585,7 @@ func (app *BaseApp) cacheTxContext(ctx sdk.Context, txBytes []byte) (sdk.Context // Note, gas execution info is always returned. A reference to a Result is // returned if the tx does not run out of gas and if all the messages are valid // and execute successfully. An error is returned otherwise. -func (app *BaseApp) runTx(mode runTxMode, txBytes []byte) (gInfo sdk.GasInfo, result *sdk.Result, anteEvents []abci.Event, err error) { +func (app *BaseApp) runTx(mode runTxMode, txBytes []byte) (gInfo sdk.GasInfo, result *sdk.Result, anteEvents []abci.Event, priority int64, err error) { // NOTE: GasWanted should be returned by the AnteHandler. GasUsed is // determined by the GasMeter. We need access to the context to get the gas // meter so we initialize upfront. @@ -596,7 +596,7 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte) (gInfo sdk.GasInfo, re // only run the tx if there is block gas remaining if mode == runTxModeDeliver && ctx.BlockGasMeter().IsOutOfGas() { - return gInfo, nil, nil, sdkerrors.Wrap(sdkerrors.ErrOutOfGas, "no block gas left to run tx") + return gInfo, nil, nil, 0, sdkerrors.Wrap(sdkerrors.ErrOutOfGas, "no block gas left to run tx") } defer func() { @@ -631,12 +631,12 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte) (gInfo sdk.GasInfo, re tx, err := app.txDecoder(txBytes) if err != nil { - return sdk.GasInfo{}, nil, nil, err + return sdk.GasInfo{}, nil, nil, 0, err } msgs := tx.GetMsgs() if err := validateBasicTxMsgs(msgs); err != nil { - return sdk.GasInfo{}, nil, nil, err + return sdk.GasInfo{}, nil, nil, 0, err } if app.anteHandler != nil { @@ -672,9 +672,10 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte) (gInfo sdk.GasInfo, re gasWanted = ctx.GasMeter().Limit() if err != nil { - return gInfo, nil, nil, err + return gInfo, nil, nil, 0, err } + priority = ctx.Priority() msCache.Write() anteEvents = events.ToABCIEvents() } @@ -695,7 +696,7 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte) (gInfo sdk.GasInfo, re if app.postHandler != nil { newCtx, err := app.postHandler(runMsgCtx, tx, mode == runTxModeSimulate) if err != nil { - return gInfo, nil, nil, err + return gInfo, nil, nil, priority, err } result.Events = append(result.Events, newCtx.EventManager().ABCIEvents()...) @@ -714,7 +715,7 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte) (gInfo sdk.GasInfo, re } } - return gInfo, result, anteEvents, err + return gInfo, result, anteEvents, priority, err } // runMsgs iterates through a list of messages and executes them with the provided diff --git a/baseapp/baseapp_test.go b/baseapp/baseapp_test.go index fcf8b1a788c8..8d67a7a5f001 100644 --- a/baseapp/baseapp_test.go +++ b/baseapp/baseapp_test.go @@ -36,6 +36,10 @@ import ( var ( capKey1 = sdk.NewKVStoreKey("key1") capKey2 = sdk.NewKVStoreKey("key2") + + // testTxPriority is the CheckTx priority that we set in the test + // antehandler. + testTxPriority = int64(42) ) type paramStore struct { @@ -826,6 +830,8 @@ func anteHandlerTxTest(t *testing.T, capKey storetypes.StoreKey, storeKey []byte counterEvent("ante_handler", txTest.Counter), ) + ctx = ctx.WithPriority(testTxPriority) + return ctx, nil } } @@ -933,6 +939,7 @@ func TestCheckTx(t *testing.T) { txBytes, err := codec.Marshal(tx) require.NoError(t, err) r := app.CheckTx(abci.RequestCheckTx{Tx: txBytes}) + require.Equal(t, testTxPriority, r.Priority) require.Empty(t, r.GetEvents()) require.True(t, r.IsOK(), fmt.Sprintf("%v", r)) } diff --git a/baseapp/test_helpers.go b/baseapp/test_helpers.go index eda2815da410..6489595bc8c1 100644 --- a/baseapp/test_helpers.go +++ b/baseapp/test_helpers.go @@ -16,13 +16,13 @@ func (app *BaseApp) SimCheck(txEncoder sdk.TxEncoder, tx sdk.Tx) (sdk.GasInfo, * if err != nil { return sdk.GasInfo{}, nil, sdkerrors.Wrapf(sdkerrors.ErrInvalidRequest, "%s", err) } - gasInfo, result, _, err := app.runTx(runTxModeCheck, bz) + gasInfo, result, _, _, err := app.runTx(runTxModeCheck, bz) return gasInfo, result, err } // Simulate executes a tx in simulate mode to get result and gas info. func (app *BaseApp) Simulate(txBytes []byte) (sdk.GasInfo, *sdk.Result, error) { - gasInfo, result, _, err := app.runTx(runTxModeSimulate, txBytes) + gasInfo, result, _, _, err := app.runTx(runTxModeSimulate, txBytes) return gasInfo, result, err } @@ -32,7 +32,7 @@ func (app *BaseApp) SimDeliver(txEncoder sdk.TxEncoder, tx sdk.Tx) (sdk.GasInfo, if err != nil { return sdk.GasInfo{}, nil, sdkerrors.Wrapf(sdkerrors.ErrInvalidRequest, "%s", err) } - gasInfo, result, _, err := app.runTx(runTxModeDeliver, bz) + gasInfo, result, _, _, err := app.runTx(runTxModeDeliver, bz) return gasInfo, result, err } diff --git a/x/auth/ante/fee_test.go b/x/auth/ante/fee_test.go index 91d322c1deec..21a0a2d7ff8e 100644 --- a/x/auth/ante/fee_test.go +++ b/x/auth/ante/fee_test.go @@ -58,8 +58,11 @@ func (suite *AnteTestSuite) TestEnsureMempoolFees() { lowGasPrice := []sdk.DecCoin{atomPrice} suite.ctx = suite.ctx.WithMinGasPrices(lowGasPrice) - _, err = antehandler(suite.ctx, tx, false) + newCtx, err := antehandler(suite.ctx, tx, false) suite.Require().Nil(err, "Decorator should not have errored on fee higher than local gasPrice") + // Priority is the smallest amount in any denom. Since we have only 1 fee + // of 150atom, the priority here is 150. + suite.Require().Equal(feeAmount.AmountOf("atom").Int64(), newCtx.Priority()) } func (suite *AnteTestSuite) TestDeductFees() { From 1a39948fe65120f21dbeb0bc8d9d58901e33d370 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Mon, 23 May 2022 11:38:25 +0200 Subject: [PATCH 28/29] Add comment for skipped test --- x/auth/client/testutil/suite.go | 24 +++++++++++++++--------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/x/auth/client/testutil/suite.go b/x/auth/client/testutil/suite.go index 416bc3be768d..a8d86bfd462b 100644 --- a/x/auth/client/testutil/suite.go +++ b/x/auth/client/testutil/suite.go @@ -1561,8 +1561,14 @@ func (s *IntegrationTestSuite) TestAuxSigner() { } } -func (s *IntegrationTestSuite) TestAuxToFee() { +func (s *IntegrationTestSuite) TestAuxToFeeWithTips() { + // Currently, simapp doesn't have Tips decorator enabled by default in its + // posthandlers, so this test will fail. + // + // TODO Find a way to test Tips integratin test with a custom simapp with + // tips posthandler. s.T().Skip() + require := s.Require() val := s.network.Validators[0] @@ -1578,13 +1584,13 @@ func (s *IntegrationTestSuite) TestAuxToFee() { fee := sdk.NewCoin(s.cfg.BondDenom, sdk.NewInt(1000)) tip := sdk.NewCoin(fmt.Sprintf("%stoken", val.Moniker), sdk.NewInt(1000)) - s.Require().NoError(s.network.WaitForNextBlock()) + require.NoError(s.network.WaitForNextBlock()) _, err = s.createBankMsg(val, tipper, sdk.NewCoins(tipperInitialBal)) require.NoError(err) - s.Require().NoError(s.network.WaitForNextBlock()) + require.NoError(s.network.WaitForNextBlock()) bal := s.getBalances(val.ClientCtx, tipper, tip.Denom) - s.Require().True(bal.Equal(tipperInitialBal.Amount)) + require.True(bal.Equal(tipperInitialBal.Amount)) testCases := []struct { name string @@ -1792,21 +1798,21 @@ func (s *IntegrationTestSuite) TestAuxToFee() { require.NoError(err) var txRes sdk.TxResponse - s.Require().NoError(val.ClientCtx.Codec.UnmarshalJSON(res.Bytes(), &txRes)) + require.NoError(val.ClientCtx.Codec.UnmarshalJSON(res.Bytes(), &txRes)) require.Contains(txRes.RawLog, tc.errMsg) } else { require.NoError(err) var txRes sdk.TxResponse - s.Require().NoError(val.ClientCtx.Codec.UnmarshalJSON(res.Bytes(), &txRes)) + require.NoError(val.ClientCtx.Codec.UnmarshalJSON(res.Bytes(), &txRes)) - s.Require().Equal(uint32(0), txRes.Code) - s.Require().NotNil(int64(0), txRes.Height) + require.Equal(uint32(0), txRes.Code) + require.NotNil(int64(0), txRes.Height) bal = s.getBalances(val.ClientCtx, tipper, tc.tip.Denom) tipperInitialBal = tipperInitialBal.Sub(tc.tip) - s.Require().True(bal.Equal(tipperInitialBal.Amount)) + require.True(bal.Equal(tipperInitialBal.Amount)) } } }) From be0897817bc276e039a69fe983bdda55d2925732 Mon Sep 17 00:00:00 2001 From: Amaury M <1293565+amaurym@users.noreply.github.com> Date: Mon, 23 May 2022 11:45:23 +0200 Subject: [PATCH 29/29] Add tips comments --- simapp/app.go | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/simapp/app.go b/simapp/app.go index 0cddbebcf2ea..171fb557c58a 100644 --- a/simapp/app.go +++ b/simapp/app.go @@ -456,9 +456,12 @@ func NewSimApp( // one decorator: the Transaction Tips decorator. However, some chains do // not need it by default, so the following line is commented out. You can // uncomment it to include the tips decorator, or define your own - // postHandler chain. Please note that changing any of the anteHandler or - // postHandler chain is likely to be a state-machine breaking change, which - // needs a coordinated upgrade. + // postHandler chain. To read more about tips: + // https://docs.cosmos.network/main/core/tips.html + // + // Please note that changing any of the anteHandler or postHandler chain is + // likely to be a state-machine breaking change, which needs a coordinated + // upgrade. // // Uncomment to enable postHandlers: // app.setPostHandler()