cosmos-sdk/baseapp/baseapp.go

689 lines
21 KiB
Go
Raw Normal View History

package baseapp
import (
"encoding/json"
"errors"
2017-12-01 09:10:17 -08:00
"fmt"
"io/ioutil"
"os"
2019-02-07 17:52:24 -08:00
"reflect"
2018-01-26 04:19:33 -08:00
"runtime/debug"
2018-05-09 13:25:13 -07:00
"strings"
2017-12-01 09:10:17 -08:00
2018-11-24 18:10:59 -08:00
"github.com/gogo/protobuf/proto"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto/tmhash"
"github.com/tendermint/tendermint/libs/log"
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/store"
storetypes "github.com/cosmos/cosmos-sdk/store/types"
2018-01-12 13:59:19 -08:00
sdk "github.com/cosmos/cosmos-sdk/types"
)
2018-05-15 17:06:17 -07:00
const (
// Check a transaction
2018-05-15 17:18:25 -07:00
runTxModeCheck runTxMode = iota
2018-05-15 17:06:17 -07:00
// Simulate a transaction
2018-05-15 17:18:25 -07:00
runTxModeSimulate runTxMode = iota
2018-05-15 17:06:17 -07:00
// Deliver a transaction
2018-05-15 17:18:25 -07:00
runTxModeDeliver runTxMode = iota
// MainStoreKey is the string representation of the main store
MainStoreKey = "main"
2018-05-15 17:06:17 -07:00
)
var (
_ abci.Application = (*BaseApp)(nil)
// mainConsensusParamsKey defines a key to store the consensus params in the
// main store.
mainConsensusParamsKey = []byte("consensus_params")
)
type (
// Enum mode for app.runTx
runTxMode uint8
// StoreLoader defines a customizable function to control how we load the CommitMultiStore
// from disk. This is useful for state migration, when loading a datastore written with
// an older version of the software. In particular, if a module changed the substore key name
// (or removed a substore) between two versions of the software.
StoreLoader func(ms sdk.CommitMultiStore) error
)
// BaseApp reflects the ABCI application implementation.
type BaseApp struct {
// initialized on creation
2019-02-07 17:52:24 -08:00
logger log.Logger
2018-07-31 18:48:32 -07:00
name string // application name from abci.Info
db dbm.DB // common DB backend
cms sdk.CommitMultiStore // Main (uncached) state
storeLoader StoreLoader // function to handle store loading, may be overridden with SetStoreLoader()
Merge PR #4159: Module/Genesis Generalization * first commit * gaia cleanup * ... * staking multihooks * missing module function return args * bank module name constant * working, module interface for x/ * got this thing compiling * make test compiles and passes * remove expanded simulation invariants * genesis issue * continued * continued * register crisis routes thought mm * begin blocker to mm * end blocker to mm * empty routes not initialized * move gaia initChainer sanity check to baseapp * remove codecs from module manager * reorging genesis stuff * module manager passed by reference/bugfixes from working last commit int int * move invariant checks from gaia to crisis * typo * basic refactors cmd/gaia/init * working * MultiStakingHooks from types to x/staking/types int * default module manager order of operations from input modules * working * typo * add AppModuleBasic * moduleBasicManager / non-test code compiles * working attempting to get tests passing * make test passes * sim random genesis fix * export bug * ... * genutil module * genutil working * refactored - happy with non-testing code in cmd/ * ... * lint fixes * comment improvement * cli test fix * compile housing * working through compile errors * working gettin' compilin' * non-test code compiles * move testnet to its own module * reworking tests int * bez staging PR 1 comments * concise module function-of names * moved all tests from genesis_test.go to other genutil tests * genaccounts package, add genutil and genaccounts to app.go * docs for genutil genaccounts * genaccounts iterate fn * non-test code with genaccounts/ now compiles * working test compiling * debugging tests * resolved all make test compile errors * test debuggin * resolved all unit tests, introduced param module * cli-test compile fixes * staking initialization bug * code comment improvements, changelog entries * BasicGaiaApp -> ModuleBasics * highlevel explanation in types/module.go * @alexanderbez comment revisions * @fedekunze PR comments * @alexanderbez PR comments (x2) * @cwgoes comments (minor updates) * @fedekunze suggestions * panic on init with multiple validator updates from different modules * initchain panic makes validate genesis fail int * AppModuleGenesis seperation int * test * remove init panic logic in validate genesis replaced with TODO * set maxprocs to match system's GOMAXPROCS * Update circleci * Cap maxprocs in CI to 4 * @alexanderbez recent comments addressed * less blocks in twouble sims int * runsim error output flag * -e on import_export as well * error out int * Try to fix failures * runsim
2019-05-16 08:25:32 -07:00
router sdk.Router // handle any kind of message
queryRouter sdk.QueryRouter // router for redirecting query calls
2018-07-31 18:48:32 -07:00
txDecoder sdk.TxDecoder // unmarshal []byte into sdk.Tx
2018-07-18 16:24:16 -07:00
2018-11-20 16:44:49 -08:00
// set upon LoadVersion or LoadLatestVersion.
2019-02-07 17:52:24 -08:00
baseKey *sdk.KVStoreKey // Main KVStore in cms
2018-02-17 16:24:40 -08:00
2019-02-07 17:52:24 -08:00
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
addrPeerFilter sdk.PeerFilter // filter peers by address and port
idPeerFilter sdk.PeerFilter // filter peers by node ID
fauxMerkleMode bool // if true, IAVL MountStores uses MountStoresDB for simulation speed.
// volatile states:
//
// checkState is set on InitChain and reset on Commit
// deliverState is set on InitChain and BeginBlock and set to nil on Commit
checkState *state // for CheckTx
deliverState *state // for DeliverTx
// an inter-block write-through cache provided to the context during deliverState
interBlockCache sdk.MultiStorePersistentCache
// absent validators from begin block
voteInfos []abci.VoteInfo
2018-11-20 16:44:49 -08:00
// consensus params
// TODO: Move this in the future to baseapp param store on main store.
2018-11-20 16:44:49 -08:00
consensusParams *abci.ConsensusParams
2019-01-18 08:45:20 -08:00
// The minimum gas prices a validator is willing to accept for processing a
// transaction. This is mainly used for DoS and spam prevention.
minGasPrices sdk.DecCoins
// flag for sealing options and parameters to a BaseApp
sealed bool
// block height at which to halt the chain and gracefully shutdown
haltHeight uint64
// minimum block time (in Unix seconds) at which to halt the chain and gracefully shutdown
haltTime uint64
// application's version string
appVersion string
}
// NewBaseApp returns a reference to an initialized BaseApp. It accepts a
// variadic number of option functions, which act on the BaseApp to set
// configuration choices.
//
// NOTE: The db is used to store the version number for now.
func NewBaseApp(
name string, logger log.Logger, db dbm.DB, txDecoder sdk.TxDecoder, options ...func(*BaseApp),
) *BaseApp {
app := &BaseApp{
2019-02-07 17:52:24 -08:00
logger: logger,
name: name,
db: db,
cms: store.NewCommitMultiStore(db),
storeLoader: DefaultStoreLoader,
router: NewRouter(),
queryRouter: NewQueryRouter(),
txDecoder: txDecoder,
fauxMerkleMode: false,
2017-12-01 09:10:17 -08:00
}
for _, option := range options {
option(app)
}
if app.interBlockCache != nil {
app.cms.SetInterBlockCache(app.interBlockCache)
}
return app
2017-12-19 21:04:40 -08:00
}
// Name returns the name of the BaseApp.
func (app *BaseApp) Name() string {
2017-12-21 23:30:44 -08:00
return app.name
}
// AppVersion returns the application's version string.
func (app *BaseApp) AppVersion() string {
return app.appVersion
}
2019-02-07 17:52:24 -08:00
// Logger returns the logger of the BaseApp.
func (app *BaseApp) Logger() log.Logger {
return app.logger
}
// MountStores mounts all IAVL or DB stores to the provided keys in the BaseApp
// multistore.
2019-02-07 17:52:24 -08:00
func (app *BaseApp) MountStores(keys ...sdk.StoreKey) {
2018-02-14 17:09:00 -08:00
for _, key := range keys {
2019-02-07 17:52:24 -08:00
switch key.(type) {
case *sdk.KVStoreKey:
if !app.fauxMerkleMode {
app.MountStore(key, sdk.StoreTypeIAVL)
} else {
// StoreTypeDB doesn't do anything upon commit, and it doesn't
// retain history, but it's useful for faster simulation.
app.MountStore(key, sdk.StoreTypeDB)
}
2019-02-07 17:52:24 -08:00
case *sdk.TransientStoreKey:
app.MountStore(key, sdk.StoreTypeTransient)
2019-02-07 17:52:24 -08:00
default:
panic("Unrecognized store key type " + reflect.TypeOf(key).Name())
}
2018-02-14 17:09:00 -08:00
}
}
// MountStores mounts all IAVL or DB stores to the provided keys in the BaseApp
// multistore.
func (app *BaseApp) MountKVStores(keys map[string]*sdk.KVStoreKey) {
for _, key := range keys {
if !app.fauxMerkleMode {
app.MountStore(key, sdk.StoreTypeIAVL)
} else {
// StoreTypeDB doesn't do anything upon commit, and it doesn't
// retain history, but it's useful for faster simulation.
app.MountStore(key, sdk.StoreTypeDB)
}
}
}
// MountStores mounts all IAVL or DB stores to the provided keys in the BaseApp
// multistore.
func (app *BaseApp) MountTransientStores(keys map[string]*sdk.TransientStoreKey) {
for _, key := range keys {
app.MountStore(key, sdk.StoreTypeTransient)
}
}
// MountStoreWithDB mounts a store to the provided key in the BaseApp
// multistore, using a specified DB.
func (app *BaseApp) MountStoreWithDB(key sdk.StoreKey, typ sdk.StoreType, db dbm.DB) {
app.cms.MountStoreWithDB(key, typ, db)
2018-01-24 11:47:51 -08:00
}
// MountStore mounts a store to the provided key in the BaseApp multistore,
// using the default DB.
func (app *BaseApp) MountStore(key sdk.StoreKey, typ sdk.StoreType) {
app.cms.MountStoreWithDB(key, typ, nil)
2018-01-24 11:47:51 -08:00
}
// LoadLatestVersion loads the latest application version. It will panic if
// called more than once on a running BaseApp.
2019-02-07 17:52:24 -08:00
func (app *BaseApp) LoadLatestVersion(baseKey *sdk.KVStoreKey) error {
err := app.storeLoader(app.cms)
if err != nil {
return err
}
2019-02-07 17:52:24 -08:00
return app.initFromMainStore(baseKey)
2017-12-19 21:04:40 -08:00
}
// DefaultStoreLoader will be used by default and loads the latest version
func DefaultStoreLoader(ms sdk.CommitMultiStore) error {
return ms.LoadLatestVersion()
}
// StoreLoaderWithUpgrade is used to prepare baseapp with a fixed StoreLoader
// pattern. This is useful in test cases, or with custom upgrade loading logic.
func StoreLoaderWithUpgrade(upgrades *storetypes.StoreUpgrades) StoreLoader {
return func(ms sdk.CommitMultiStore) error {
return ms.LoadLatestVersionAndUpgrade(upgrades)
}
}
// UpgradeableStoreLoader can be configured by SetStoreLoader() to check for the
// existence of a given upgrade file - json encoded StoreUpgrades data.
//
// If not file is present, it will peform the default load (no upgrades to store).
//
// If the file is present, it will parse the file and execute those upgrades
// (rename or delete stores), while loading the data. It will also delete the
// upgrade file upon successful load, so that the upgrade is only applied once,
// and not re-applied on next restart
//
// This is useful for in place migrations when a store key is renamed between
// two versions of the software. (TODO: this code will move to x/upgrades
// when PR #4233 is merged, here mainly to help test the design)
func UpgradeableStoreLoader(upgradeInfoPath string) StoreLoader {
return func(ms sdk.CommitMultiStore) error {
_, err := os.Stat(upgradeInfoPath)
if os.IsNotExist(err) {
return DefaultStoreLoader(ms)
} else if err != nil {
return err
}
// there is a migration file, let's execute
data, err := ioutil.ReadFile(upgradeInfoPath)
if err != nil {
2019-08-19 09:06:27 -07:00
return fmt.Errorf("cannot read upgrade file %s: %v", upgradeInfoPath, err)
}
var upgrades storetypes.StoreUpgrades
err = json.Unmarshal(data, &upgrades)
if err != nil {
2019-08-19 09:06:27 -07:00
return fmt.Errorf("cannot parse upgrade file: %v", err)
}
err = ms.LoadLatestVersionAndUpgrade(&upgrades)
if err != nil {
2019-08-19 09:06:27 -07:00
return fmt.Errorf("load and upgrade database: %v", err)
}
// if we have a successful load, we delete the file
err = os.Remove(upgradeInfoPath)
if err != nil {
return fmt.Errorf("deleting upgrade file %s: %v", upgradeInfoPath, err)
}
return nil
}
}
// LoadVersion loads the BaseApp application version. It will panic if called
// more than once on a running baseapp.
2019-02-07 17:52:24 -08:00
func (app *BaseApp) LoadVersion(version int64, baseKey *sdk.KVStoreKey) error {
err := app.cms.LoadVersion(version)
if err != nil {
return err
}
2019-02-07 17:52:24 -08:00
return app.initFromMainStore(baseKey)
2017-12-20 17:34:51 -08:00
}
// LastCommitID returns the last CommitID of the multistore.
func (app *BaseApp) LastCommitID() sdk.CommitID {
2018-01-26 04:19:33 -08:00
return app.cms.LastCommitID()
2017-12-26 17:04:48 -08:00
}
// LastBlockHeight returns the last committed block height.
func (app *BaseApp) LastBlockHeight() int64 {
2018-01-26 04:19:33 -08:00
return app.cms.LastCommitID().Version
2017-12-26 17:04:48 -08:00
}
2018-02-14 08:16:06 -08:00
// initializes the remaining logic from app.cms
2019-02-07 17:52:24 -08:00
func (app *BaseApp) initFromMainStore(baseKey *sdk.KVStoreKey) error {
mainStore := app.cms.GetKVStore(baseKey)
2018-11-20 16:44:49 -08:00
if mainStore == nil {
return errors.New("baseapp expects MultiStore with 'main' KVStore")
2017-12-19 21:04:40 -08:00
}
2018-11-20 16:44:49 -08:00
2019-02-07 17:52:24 -08:00
// memoize baseKey
if app.baseKey != nil {
panic("app.baseKey expected to be nil; duplicate init?")
2018-11-20 16:44:49 -08:00
}
2019-02-07 17:52:24 -08:00
app.baseKey = baseKey
2018-11-20 16:44:49 -08:00
// Load the consensus params from the main store. If the consensus params are
// nil, it will be saved later during InitChain.
//
// TODO: assert that InitChain hasn't yet been called.
2018-11-20 16:44:49 -08:00
consensusParamsBz := mainStore.Get(mainConsensusParamsKey)
if consensusParamsBz != nil {
var consensusParams = &abci.ConsensusParams{}
2018-11-24 18:10:59 -08:00
err := proto.Unmarshal(consensusParamsBz, consensusParams)
2018-11-20 16:44:49 -08:00
if err != nil {
panic(err)
}
2018-11-20 16:44:49 -08:00
app.setConsensusParams(consensusParams)
}
// needed for the export command which inits from store but never calls initchain
2018-07-23 16:55:09 -07:00
app.setCheckState(abci.Header{})
app.Seal()
2017-12-20 17:34:51 -08:00
return nil
2017-12-01 09:10:17 -08:00
}
2019-01-18 08:45:20 -08:00
func (app *BaseApp) setMinGasPrices(gasPrices sdk.DecCoins) {
app.minGasPrices = gasPrices
}
func (app *BaseApp) setHaltHeight(haltHeight uint64) {
app.haltHeight = haltHeight
}
func (app *BaseApp) setHaltTime(haltTime uint64) {
app.haltTime = haltTime
}
func (app *BaseApp) setInterBlockCache(cache sdk.MultiStorePersistentCache) {
app.interBlockCache = cache
}
// Router returns the router of the BaseApp.
Merge PR #4159: Module/Genesis Generalization * first commit * gaia cleanup * ... * staking multihooks * missing module function return args * bank module name constant * working, module interface for x/ * got this thing compiling * make test compiles and passes * remove expanded simulation invariants * genesis issue * continued * continued * register crisis routes thought mm * begin blocker to mm * end blocker to mm * empty routes not initialized * move gaia initChainer sanity check to baseapp * remove codecs from module manager * reorging genesis stuff * module manager passed by reference/bugfixes from working last commit int int * move invariant checks from gaia to crisis * typo * basic refactors cmd/gaia/init * working * MultiStakingHooks from types to x/staking/types int * default module manager order of operations from input modules * working * typo * add AppModuleBasic * moduleBasicManager / non-test code compiles * working attempting to get tests passing * make test passes * sim random genesis fix * export bug * ... * genutil module * genutil working * refactored - happy with non-testing code in cmd/ * ... * lint fixes * comment improvement * cli test fix * compile housing * working through compile errors * working gettin' compilin' * non-test code compiles * move testnet to its own module * reworking tests int * bez staging PR 1 comments * concise module function-of names * moved all tests from genesis_test.go to other genutil tests * genaccounts package, add genutil and genaccounts to app.go * docs for genutil genaccounts * genaccounts iterate fn * non-test code with genaccounts/ now compiles * working test compiling * debugging tests * resolved all make test compile errors * test debuggin * resolved all unit tests, introduced param module * cli-test compile fixes * staking initialization bug * code comment improvements, changelog entries * BasicGaiaApp -> ModuleBasics * highlevel explanation in types/module.go * @alexanderbez comment revisions * @fedekunze PR comments * @alexanderbez PR comments (x2) * @cwgoes comments (minor updates) * @fedekunze suggestions * panic on init with multiple validator updates from different modules * initchain panic makes validate genesis fail int * AppModuleGenesis seperation int * test * remove init panic logic in validate genesis replaced with TODO * set maxprocs to match system's GOMAXPROCS * Update circleci * Cap maxprocs in CI to 4 * @alexanderbez recent comments addressed * less blocks in twouble sims int * runsim error output flag * -e on import_export as well * error out int * Try to fix failures * runsim
2019-05-16 08:25:32 -07:00
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.
Merge PR #4159: Module/Genesis Generalization * first commit * gaia cleanup * ... * staking multihooks * missing module function return args * bank module name constant * working, module interface for x/ * got this thing compiling * make test compiles and passes * remove expanded simulation invariants * genesis issue * continued * continued * register crisis routes thought mm * begin blocker to mm * end blocker to mm * empty routes not initialized * move gaia initChainer sanity check to baseapp * remove codecs from module manager * reorging genesis stuff * module manager passed by reference/bugfixes from working last commit int int * move invariant checks from gaia to crisis * typo * basic refactors cmd/gaia/init * working * MultiStakingHooks from types to x/staking/types int * default module manager order of operations from input modules * working * typo * add AppModuleBasic * moduleBasicManager / non-test code compiles * working attempting to get tests passing * make test passes * sim random genesis fix * export bug * ... * genutil module * genutil working * refactored - happy with non-testing code in cmd/ * ... * lint fixes * comment improvement * cli test fix * compile housing * working through compile errors * working gettin' compilin' * non-test code compiles * move testnet to its own module * reworking tests int * bez staging PR 1 comments * concise module function-of names * moved all tests from genesis_test.go to other genutil tests * genaccounts package, add genutil and genaccounts to app.go * docs for genutil genaccounts * genaccounts iterate fn * non-test code with genaccounts/ now compiles * working test compiling * debugging tests * resolved all make test compile errors * test debuggin * resolved all unit tests, introduced param module * cli-test compile fixes * staking initialization bug * code comment improvements, changelog entries * BasicGaiaApp -> ModuleBasics * highlevel explanation in types/module.go * @alexanderbez comment revisions * @fedekunze PR comments * @alexanderbez PR comments (x2) * @cwgoes comments (minor updates) * @fedekunze suggestions * panic on init with multiple validator updates from different modules * initchain panic makes validate genesis fail int * AppModuleGenesis seperation int * test * remove init panic logic in validate genesis replaced with TODO * set maxprocs to match system's GOMAXPROCS * Update circleci * Cap maxprocs in CI to 4 * @alexanderbez recent comments addressed * less blocks in twouble sims int * runsim error output flag * -e on import_export as well * error out int * Try to fix failures * runsim
2019-05-16 08:25:32 -07:00
func (app *BaseApp) QueryRouter() sdk.QueryRouter { return app.queryRouter }
// Seal seals a BaseApp. It prohibits any further modifications to a BaseApp.
func (app *BaseApp) Seal() { app.sealed = true }
// IsSealed returns true if the BaseApp is sealed and false otherwise.
func (app *BaseApp) IsSealed() bool { return app.sealed }
// setCheckState sets the BaseApp's checkState with a cache-wrapped multi-store
// (i.e. a CacheMultiStore) and a new Context with the cache-wrapped multi-store,
// provided header, and minimum gas prices set. It is set on InitChain and reset
// on Commit.
func (app *BaseApp) setCheckState(header abci.Header) {
ms := app.cms.CacheMultiStore()
app.checkState = &state{
ms: ms,
2019-02-07 17:52:24 -08:00
ctx: sdk.NewContext(ms, header, true, app.logger).WithMinGasPrices(app.minGasPrices),
}
}
// setDeliverState sets the BaseApp's deliverState with a cache-wrapped multi-store
// (i.e. a CacheMultiStore) and a new Context with the cache-wrapped multi-store,
// and provided header. It is set on InitChain and BeginBlock and set to nil on
// Commit.
func (app *BaseApp) setDeliverState(header abci.Header) {
ms := app.cms.CacheMultiStore()
app.deliverState = &state{
ms: ms,
2019-02-07 17:52:24 -08:00
ctx: sdk.NewContext(ms, header, false, app.logger),
}
}
2018-11-20 16:44:49 -08:00
// setConsensusParams memoizes the consensus params.
func (app *BaseApp) setConsensusParams(consensusParams *abci.ConsensusParams) {
app.consensusParams = consensusParams
}
// setConsensusParams stores the consensus params to the main store.
func (app *BaseApp) storeConsensusParams(consensusParams *abci.ConsensusParams) {
2018-11-24 18:10:59 -08:00
consensusParamsBz, err := proto.Marshal(consensusParams)
2018-11-20 16:44:49 -08:00
if err != nil {
panic(err)
}
2019-02-07 17:52:24 -08:00
mainStore := app.cms.GetKVStore(app.baseKey)
2018-11-20 16:44:49 -08:00
mainStore.Set(mainConsensusParamsKey, consensusParamsBz)
}
// getMaximumBlockGas gets the maximum gas from the consensus params. It panics
// if maximum block gas is less than negative one and returns zero if negative
// one.
func (app *BaseApp) getMaximumBlockGas() uint64 {
if app.consensusParams == nil || app.consensusParams.Block == nil {
2018-11-20 16:44:49 -08:00
return 0
}
maxGas := app.consensusParams.Block.MaxGas
switch {
case maxGas < -1:
panic(fmt.Sprintf("invalid maximum block gas: %d", maxGas))
case maxGas == -1:
return 0
default:
return uint64(maxGas)
}
2018-11-20 16:44:49 -08:00
}
func (app *BaseApp) validateHeight(req abci.RequestBeginBlock) error {
if req.Header.Height < 1 {
return fmt.Errorf("invalid height: %d", req.Header.Height)
}
prevHeight := app.LastBlockHeight()
if req.Header.Height != prevHeight+1 {
return fmt.Errorf("invalid height: %d; expected: %d", req.Header.Height, prevHeight+1)
}
return nil
}
// validateBasicTxMsgs executes basic validator calls for messages.
2018-07-09 16:16:43 -07:00
func validateBasicTxMsgs(msgs []sdk.Msg) sdk.Error {
if len(msgs) == 0 {
2019-02-07 17:52:24 -08:00
return sdk.ErrUnknownRequest("Tx.GetMsgs() must return at least one message in list")
2018-07-09 16:16:43 -07:00
}
for _, msg := range msgs {
// Validate the Msg.
err := msg.ValidateBasic()
if err != nil {
return err
}
}
2018-07-09 16:16:43 -07:00
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 {
if mode == runTxModeCheck || mode == runTxModeSimulate {
return app.checkState
}
return app.deliverState
}
// retrieve the context for the tx w/ txBytes and other memoized values.
func (app *BaseApp) getContextForTx(mode runTxMode, txBytes []byte) (ctx sdk.Context) {
ctx = app.getState(mode).ctx.
WithTxBytes(txBytes).
WithVoteInfos(app.voteInfos).
WithConsensusParams(app.consensusParams)
if mode == runTxModeSimulate {
ctx, _ = ctx.CacheContext()
}
return
}
// cacheTxContext returns a new context based off of the provided context with
// a cache wrapped 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() {
2019-02-01 17:03:09 -08:00
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. The transactions is processed via an
// anteHandler. The provided txBytes may be nil in some cases, eg. in tests. For
// further details on transaction execution, reference the BaseApp SDK
// documentation.
2018-05-15 17:18:25 -07:00
func (app *BaseApp) runTx(mode runTxMode, txBytes []byte, tx sdk.Tx) (result sdk.Result) {
// 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()
2018-11-13 11:27:15 -08:00
// only run the tx if there is block gas remaining
if mode == runTxModeDeliver && ctx.BlockGasMeter().IsOutOfGas() {
return sdk.ErrOutOfGas("no block gas left to run tx").Result()
2018-11-13 11:27:15 -08:00
}
var startingGas uint64
if mode == runTxModeDeliver {
startingGas = ctx.BlockGasMeter().GasConsumed()
}
defer func() {
if r := recover(); r != nil {
switch rType := r.(type) {
2018-05-08 08:46:02 -07:00
case sdk.ErrorOutOfGas:
log := fmt.Sprintf(
"out of gas in location: %v; gasWanted: %d, gasUsed: %d",
rType.Descriptor, gasWanted, ctx.GasMeter().GasConsumed(),
)
2018-05-08 08:46:02 -07:00
result = sdk.ErrOutOfGas(log).Result()
default:
log := fmt.Sprintf("recovered: %v\nstack:\n%v", r, string(debug.Stack()))
2018-05-08 08:46:02 -07:00
result = sdk.ErrInternal(log).Result()
}
}
2018-11-22 09:34:13 -08:00
result.GasWanted = gasWanted
result.GasUsed = ctx.GasMeter().GasConsumed()
}()
// 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.
2018-11-22 09:34:13 -08:00
defer func() {
if mode == runTxModeDeliver {
ctx.BlockGasMeter().ConsumeGas(
ctx.GasMeter().GasConsumedToLimit(),
"block gas meter",
)
if ctx.BlockGasMeter().GasConsumed() < startingGas {
panic(sdk.ErrorGasOverflow{Descriptor: "tx gas summation"})
}
}
}()
var msgs = tx.GetMsgs()
if err := validateBasicTxMsgs(msgs); err != nil {
2018-07-09 16:16:43 -07:00
return err.Result()
}
2018-02-28 19:17:48 -08:00
if app.anteHandler != nil {
var anteCtx sdk.Context
var msCache sdk.CacheMultiStore
// Cache wrap 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)
newCtx, result, abort := app.anteHandler(anteCtx, tx, mode == runTxModeSimulate)
2018-02-28 19:17:48 -08:00
if !newCtx.IsZero() {
// At this point, newCtx.MultiStore() is cache-wrapped, or something else
// replaced by the ante handler. We want the original multistore, not one
// which was cache-wrapped for the ante handler.
//
// Also, in the case of the tx aborting, we need to track gas consumed via
// the instantiated gas meter in the ante handler, so we update the context
// prior to returning.
ctx = newCtx.WithMultiStore(ms)
2018-02-28 19:17:48 -08:00
}
2019-02-07 17:52:24 -08:00
gasWanted = result.GasWanted
if abort {
return result
}
msCache.Write()
2018-01-26 05:11:01 -08:00
}
// Create a new Context based off of the existing Context with a cache-wrapped
// MultiStore in case message processing fails. At this point, the MultiStore
// is doubly cached-wrapped.
runMsgCtx, msCache := app.cacheTxContext(ctx, txBytes)
result = app.runMsgs(runMsgCtx, msgs, mode)
result.GasWanted = gasWanted
2019-07-30 09:11:09 -07:00
// Safety check: don't write the cache state unless we're in DeliverTx.
if mode != runTxModeDeliver {
return result
}
// only update state if all messages pass
if result.IsOK() {
2018-01-26 05:11:01 -08:00
msCache.Write()
}
return result
}
// runMsgs iterates through all the messages and executes them.
// nolint: gocyclo
func (app *BaseApp) runMsgs(ctx sdk.Context, msgs []sdk.Msg, mode runTxMode) (result sdk.Result) {
msgLogs := make(sdk.ABCIMessageLogs, 0, len(msgs))
var (
data []byte
code sdk.CodeType
codespace sdk.CodespaceType
)
events := sdk.EmptyEvents()
2017-12-01 09:10:17 -08:00
// NOTE: GasWanted is determined by ante handler and GasUsed by the GasMeter.
for i, msg := range msgs {
// match message route
msgRoute := msg.Route()
handler := app.router.Route(msgRoute)
if handler == nil {
return sdk.ErrUnknownRequest("unrecognized Msg type: " + msgRoute).Result()
}
var msgResult sdk.Result
// skip actual execution for CheckTx mode
if mode != runTxModeCheck {
msgResult = handler(ctx, msg)
}
// Each message result's Data must be length prefixed in order to separate
// each result.
data = append(data, msgResult.Data...)
// append events from the message's execution and a message action event
events = events.AppendEvent(sdk.NewEvent(sdk.EventTypeMessage, sdk.NewAttribute(sdk.AttributeKeyAction, msg.Type())))
events = events.AppendEvents(msgResult.Events)
// stop execution and return on first failed message
if !msgResult.IsOK() {
msgLogs = append(msgLogs, sdk.NewABCIMessageLog(uint16(i), false, msgResult.Log, events))
code = msgResult.Code
codespace = msgResult.Codespace
break
}
msgLogs = append(msgLogs, sdk.NewABCIMessageLog(uint16(i), true, msgResult.Log, events))
}
result = sdk.Result{
Code: code,
Codespace: codespace,
Data: data,
Log: strings.TrimSpace(msgLogs.String()),
GasUsed: ctx.GasMeter().GasConsumed(),
Events: events,
}
return result
}