Cache-wrap context during ante handler exec (#2781)

* Use cache-wrapped multi-store in ante
* Implement TestBaseAppAnteHandler
* Add reference documentation for BaseApp/CheckTx/DeliverTx
This commit is contained in:
Alexander Bezobchuk 2018-11-16 13:33:47 -05:00 committed by Jae Kwon
parent 8d6b0929fb
commit 15b6fa0959
5 changed files with 214 additions and 43 deletions

View File

@ -53,9 +53,10 @@ IMPROVEMENTS
* [\#2749](https://github.com/cosmos/cosmos-sdk/pull/2749) Add --chain-id flag to gaiad testnet * [\#2749](https://github.com/cosmos/cosmos-sdk/pull/2749) Add --chain-id flag to gaiad testnet
* Gaia * Gaia
- #2773 Require moniker to be provided on `gaiad init`. - #2772 Update BaseApp to not persist state when the ante handler fails on DeliverTx.
- #2672 [Makefile] Updated for better Windows compatibility and ledger support logic, get_tools was rewritten as a cross-compatible Makefile. - #2773 Require moniker to be provided on `gaiad init`.
- [#110](https://github.com/tendermint/devops/issues/110) Updated CircleCI job to trigger website build when cosmos docs are updated. - #2672 [Makefile] Updated for better Windows compatibility and ledger support logic, get_tools was rewritten as a cross-compatible Makefile.
- [#110](https://github.com/tendermint/devops/issues/110) Updated CircleCI job to trigger website build when cosmos docs are updated.
* SDK * SDK
- [x/mock/simulation] [\#2720] major cleanup, introduction of helper objects, reorganization - [x/mock/simulation] [\#2720] major cleanup, introduction of helper objects, reorganization

View File

@ -10,7 +10,6 @@ import (
abci "github.com/tendermint/tendermint/abci/types" abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/crypto/tmhash" "github.com/tendermint/tendermint/crypto/tmhash"
cmn "github.com/tendermint/tendermint/libs/common"
dbm "github.com/tendermint/tendermint/libs/db" dbm "github.com/tendermint/tendermint/libs/db"
"github.com/tendermint/tendermint/libs/log" "github.com/tendermint/tendermint/libs/log"
@ -505,11 +504,11 @@ func validateBasicTxMsgs(msgs []sdk.Msg) sdk.Error {
// retrieve the context for the ante handler and store the tx bytes; store // retrieve the context for the ante handler and store the tx bytes; store
// the vote infos if the tx runs within the deliverTx() state. // the vote infos if the tx runs within the deliverTx() state.
func (app *BaseApp) getContextForAnte(mode runTxMode, txBytes []byte) (ctx sdk.Context) { func (app *BaseApp) getContextForAnte(mode runTxMode, txBytes []byte) (ctx sdk.Context) {
// Get the context ctx = app.getState(mode).ctx.WithTxBytes(txBytes)
ctx = getState(app, mode).ctx.WithTxBytes(txBytes)
if mode == runTxModeDeliver { if mode == runTxModeDeliver {
ctx = ctx.WithVoteInfos(app.voteInfos) ctx = ctx.WithVoteInfos(app.voteInfos)
} }
return return
} }
@ -571,7 +570,7 @@ func (app *BaseApp) runMsgs(ctx sdk.Context, msgs []sdk.Msg, mode runTxMode) (re
// Returns the applicantion's deliverState if app is in runTxModeDeliver, // Returns the applicantion's deliverState if app is in runTxModeDeliver,
// otherwise it returns the application's checkstate. // otherwise it returns the application's checkstate.
func getState(app *BaseApp, mode runTxMode) *state { func (app *BaseApp) getState(mode runTxMode) *state {
if mode == runTxModeCheck || mode == runTxModeSimulate { if mode == runTxModeCheck || mode == runTxModeSimulate {
return app.checkState return app.checkState
} }
@ -581,20 +580,42 @@ func getState(app *BaseApp, mode runTxMode) *state {
func (app *BaseApp) initializeContext(ctx sdk.Context, mode runTxMode) sdk.Context { func (app *BaseApp) initializeContext(ctx sdk.Context, mode runTxMode) sdk.Context {
if mode == runTxModeSimulate { if mode == runTxModeSimulate {
ctx = ctx.WithMultiStore(getState(app, runTxModeSimulate).CacheMultiStore()) ctx = ctx.WithMultiStore(app.getState(runTxModeSimulate).CacheMultiStore())
} }
return ctx return ctx
} }
// cacheTxContext returns a new context based off of the provided context with a
// cache wrapped multi-store and the store itself to allow the caller to write
// changes from the cached multi-store.
func (app *BaseApp) cacheTxContext(
ctx sdk.Context, txBytes []byte, mode runTxMode,
) (sdk.Context, sdk.CacheMultiStore) {
msCache := app.getState(mode).CacheMultiStore()
if msCache.TracingEnabled() {
msCache = msCache.WithTracingContext(
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 proccessed via an // runTx processes a transaction. The transactions is proccessed via an
// anteHandler. txBytes may be nil in some cases, eg. in tests. Also, in the // anteHandler. The provided txBytes may be nil in some cases, eg. in tests. For
// future we may support "internal" transactions. // further details on transaction execution, reference the BaseApp SDK
// documentation.
func (app *BaseApp) runTx(mode runTxMode, txBytes []byte, tx sdk.Tx) (result sdk.Result) { func (app *BaseApp) runTx(mode runTxMode, txBytes []byte, tx sdk.Tx) (result sdk.Result) {
// NOTE: GasWanted should be returned by the AnteHandler. GasUsed is // NOTE: GasWanted should be returned by the AnteHandler. GasUsed is
// determined by the GasMeter. We need access to the context to get the gas // determined by the GasMeter. We need access to the context to get the gas
// meter so we initialize upfront. // meter so we initialize upfront.
var gasWanted int64 var gasWanted int64
var msCache sdk.CacheMultiStore
ctx := app.getContextForAnte(mode, txBytes) ctx := app.getContextForAnte(mode, txBytes)
ctx = app.initializeContext(ctx, mode) ctx = app.initializeContext(ctx, mode)
@ -619,16 +640,27 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte, tx sdk.Tx) (result sdk
return err.Result() return err.Result()
} }
// run the ante handler // Execute the ante handler if one is defined.
if app.anteHandler != nil { if app.anteHandler != nil {
newCtx, result, abort := app.anteHandler(ctx, tx, (mode == runTxModeSimulate)) 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.
// 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, mode)
newCtx, result, abort := app.anteHandler(anteCtx, tx, (mode == runTxModeSimulate))
if abort { if abort {
return result return result
} }
if !newCtx.IsZero() { if !newCtx.IsZero() {
ctx = newCtx ctx = newCtx
} }
msCache.Write()
gasWanted = result.GasWanted gasWanted = result.GasWanted
} }
@ -638,17 +670,10 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte, tx sdk.Tx) (result sdk
return return
} }
// Keep the state in a transient CacheWrap in case processing the messages // Create a new context based off of the existing context with a cache wrapped
// fails. // multi-store in case message processing fails.
msCache = getState(app, mode).CacheMultiStore() runMsgCtx, msCache := app.cacheTxContext(ctx, txBytes, mode)
if msCache.TracingEnabled() { result = app.runMsgs(runMsgCtx, msgs, mode)
msCache = msCache.WithTracingContext(sdk.TraceContext(
map[string]interface{}{"txHash": cmn.HexBytes(tmhash.Sum(txBytes)).String()},
)).(sdk.CacheMultiStore)
}
ctx = ctx.WithMultiStore(msCache)
result = app.runMsgs(ctx, msgs, mode)
result.GasWanted = gasWanted result.GasWanted = gasWanted
// only update state if all messages pass // only update state if all messages pass

View File

@ -282,8 +282,19 @@ func TestInitChainer(t *testing.T) {
// Simple tx with a list of Msgs. // Simple tx with a list of Msgs.
type txTest struct { type txTest struct {
Msgs []sdk.Msg Msgs []sdk.Msg
Counter int64 Counter int64
FailOnAnte bool
}
func (tx *txTest) setFailOnAnte(fail bool) {
tx.FailOnAnte = fail
}
func (tx *txTest) setFailOnHandler(fail bool) {
for i, msg := range tx.Msgs {
tx.Msgs[i] = msgCounter{msg.(msgCounter).Counter, fail}
}
} }
// Implements Tx // Implements Tx
@ -297,7 +308,8 @@ const (
// ValidateBasic() fails on negative counters. // ValidateBasic() fails on negative counters.
// Otherwise it's up to the handlers // Otherwise it's up to the handlers
type msgCounter struct { type msgCounter struct {
Counter int64 Counter int64
FailOnHandler bool
} }
// Implements Msg // Implements Msg
@ -315,9 +327,9 @@ func (msg msgCounter) ValidateBasic() sdk.Error {
func newTxCounter(txInt int64, msgInts ...int64) *txTest { func newTxCounter(txInt int64, msgInts ...int64) *txTest {
var msgs []sdk.Msg var msgs []sdk.Msg
for _, msgInt := range msgInts { for _, msgInt := range msgInts {
msgs = append(msgs, msgCounter{msgInt}) msgs = append(msgs, msgCounter{msgInt, false})
} }
return &txTest{msgs, txInt} return &txTest{msgs, txInt, false}
} }
// a msg we dont know how to route // a msg we dont know how to route
@ -369,8 +381,13 @@ func testTxDecoder(cdc *codec.Codec) sdk.TxDecoder {
func anteHandlerTxTest(t *testing.T, capKey *sdk.KVStoreKey, storeKey []byte) sdk.AnteHandler { func anteHandlerTxTest(t *testing.T, capKey *sdk.KVStoreKey, storeKey []byte) sdk.AnteHandler {
return func(ctx sdk.Context, tx sdk.Tx, simulate bool) (newCtx sdk.Context, res sdk.Result, abort bool) { return func(ctx sdk.Context, tx sdk.Tx, simulate bool) (newCtx sdk.Context, res sdk.Result, abort bool) {
store := ctx.KVStore(capKey) store := ctx.KVStore(capKey)
msgCounter := tx.(txTest).Counter txTest := tx.(txTest)
res = incrementingCounter(t, store, storeKey, msgCounter)
if txTest.FailOnAnte {
return newCtx, sdk.ErrInternal("ante handler failure").Result(), true
}
res = incrementingCounter(t, store, storeKey, txTest.Counter)
return return
} }
} }
@ -381,10 +398,15 @@ func handlerMsgCounter(t *testing.T, capKey *sdk.KVStoreKey, deliverKey []byte)
var msgCount int64 var msgCount int64
switch m := msg.(type) { switch m := msg.(type) {
case *msgCounter: case *msgCounter:
if m.FailOnHandler {
return sdk.ErrInternal("message handler failure").Result()
}
msgCount = m.Counter msgCount = m.Counter
case *msgCounter2: case *msgCounter2:
msgCount = m.Counter msgCount = m.Counter
} }
return incrementingCounter(t, store, deliverKey, msgCount) return incrementingCounter(t, store, deliverKey, msgCount)
} }
} }
@ -712,12 +734,12 @@ func TestRunInvalidTransaction(t *testing.T) {
// Transaction with no known route // Transaction with no known route
{ {
unknownRouteTx := txTest{[]sdk.Msg{msgNoRoute{}}, 0} unknownRouteTx := txTest{[]sdk.Msg{msgNoRoute{}}, 0, false}
err := app.Deliver(unknownRouteTx) err := app.Deliver(unknownRouteTx)
require.EqualValues(t, sdk.CodeUnknownRequest, err.Code) require.EqualValues(t, sdk.CodeUnknownRequest, err.Code)
require.EqualValues(t, sdk.CodespaceRoot, err.Codespace) require.EqualValues(t, sdk.CodespaceRoot, err.Codespace)
unknownRouteTx = txTest{[]sdk.Msg{msgCounter{}, msgNoRoute{}}, 0} unknownRouteTx = txTest{[]sdk.Msg{msgCounter{}, msgNoRoute{}}, 0, false}
err = app.Deliver(unknownRouteTx) err = app.Deliver(unknownRouteTx)
require.EqualValues(t, sdk.CodeUnknownRequest, err.Code) require.EqualValues(t, sdk.CodeUnknownRequest, err.Code)
require.EqualValues(t, sdk.CodespaceRoot, err.Codespace) require.EqualValues(t, sdk.CodespaceRoot, err.Codespace)
@ -829,3 +851,72 @@ func TestTxGasLimits(t *testing.T) {
} }
} }
} }
func TestBaseAppAnteHandler(t *testing.T) {
anteKey := []byte("ante-key")
anteOpt := func(bapp *BaseApp) {
bapp.SetAnteHandler(anteHandlerTxTest(t, capKey1, anteKey))
}
deliverKey := []byte("deliver-key")
routerOpt := func(bapp *BaseApp) {
bapp.Router().AddRoute(routeMsgCounter, handlerMsgCounter(t, capKey1, deliverKey))
}
cdc := codec.New()
app := setupBaseApp(t, anteOpt, routerOpt)
app.InitChain(abci.RequestInitChain{})
registerTestCodec(cdc)
app.BeginBlock(abci.RequestBeginBlock{})
// 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 (anteHandlerTxTest).
tx := newTxCounter(0, 0)
tx.setFailOnAnte(true)
txBytes, err := cdc.MarshalBinaryLengthPrefixed(tx)
require.NoError(t, err)
res := app.DeliverTx(txBytes)
require.False(t, res.IsOK(), fmt.Sprintf("%v", res))
ctx := app.getState(runTxModeDeliver).ctx
store := ctx.KVStore(capKey1)
require.Equal(t, int64(0), getIntFromStore(store, anteKey))
// execute at tx that will pass the ante handler (the checkTx state should
// mutate) but will fail the message handler
tx = newTxCounter(0, 0)
tx.setFailOnHandler(true)
txBytes, err = cdc.MarshalBinaryLengthPrefixed(tx)
require.NoError(t, err)
res = app.DeliverTx(txBytes)
require.False(t, res.IsOK(), fmt.Sprintf("%v", res))
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))
// execute a successful ante handler and message execution where state is
// implicitly checked by previous tx executions
tx = newTxCounter(1, 0)
txBytes, err = cdc.MarshalBinaryLengthPrefixed(tx)
require.NoError(t, err)
res = app.DeliverTx(txBytes)
require.True(t, res.IsOK(), fmt.Sprintf("%v", res))
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))
// commit
app.EndBlock(abci.RequestEndBlock{})
app.Commit()
}

View File

@ -537,16 +537,20 @@ func TestGaiaCLISendGenerateSignAndBroadcast(t *testing.T) {
success, stdout, _ = executeWriteRetStdStreams(t, fmt.Sprintf( success, stdout, _ = executeWriteRetStdStreams(t, fmt.Sprintf(
"gaiacli tx broadcast %v --json %v", flags, signedTxFile.Name())) "gaiacli tx broadcast %v --json %v", flags, signedTxFile.Name()))
require.True(t, success) require.True(t, success)
var result struct { var result struct {
Response abci.ResponseDeliverTx Response abci.ResponseDeliverTx
} }
require.Nil(t, app.MakeCodec().UnmarshalJSON([]byte(stdout), &result)) require.Nil(t, app.MakeCodec().UnmarshalJSON([]byte(stdout), &result))
require.Equal(t, msg.Fee.Gas, result.Response.GasUsed) require.Equal(t, msg.Fee.Gas, result.Response.GasUsed)
require.Equal(t, msg.Fee.Gas, result.Response.GasWanted) require.Equal(t, msg.Fee.Gas, result.Response.GasWanted)
tests.WaitForNextNBlocksTM(2, port) tests.WaitForNextNBlocksTM(2, port)
barAcc := executeGetAccount(t, fmt.Sprintf("gaiacli query account %s %v", barAddr, flags)) barAcc := executeGetAccount(t, fmt.Sprintf("gaiacli query account %s %v", barAddr, flags))
require.Equal(t, int64(10), barAcc.GetCoins().AmountOf(stakeTypes.DefaultBondDenom).Int64()) require.Equal(t, int64(10), barAcc.GetCoins().AmountOf(stakeTypes.DefaultBondDenom).Int64())
fooAcc = executeGetAccount(t, fmt.Sprintf("gaiacli query account %s %v", fooAddr, flags)) fooAcc = executeGetAccount(t, fmt.Sprintf("gaiacli query account %s %v", fooAddr, flags))
require.Equal(t, int64(40), fooAcc.GetCoins().AmountOf(stakeTypes.DefaultBondDenom).Int64()) require.Equal(t, int64(40), fooAcc.GetCoins().AmountOf(stakeTypes.DefaultBondDenom).Int64())
} }

View File

@ -1,13 +1,63 @@
# baseApp # BaseApp
`baseApp` requires stores to be mounted via capabilities keys - handlers can only access stores they're given the key to. The `baseApp` ensures all stores are properly loaded, cached, and committed. One mounted store is considered the "main" - it holds the latest block header, from which we can find and load the most recent state. The BaseApp defines the foundational implementation for a basic ABCI application
so that your Cosmos-SDK application can communicate with an underlying
Tendermint node.
`baseApp` distinguishes between two handler types: `AnteHandler` and `MsgHandler`. Whilst the former is a global validity check that applies to all transactions from all modules, i.e. it checks nonces and whether balances are sufficient to pay fees, validates signatures and ensures that transactions don't carry too many signatures, the latter is the full state transition function. The BaseApp is composed of many internal components. Some of the most important
During CheckTx the state transition function is only applied to the checkTxState and should return include the `CommitMultiStore` and its internal state. The internal state is
before any expensive state transitions are run (this is up to each developer). It also needs to return the estimated essentially two sub-states, both of which are used for transaction execution
gas cost. during different phases, `CheckTx` and `DeliverTx` respectively. During block
commitment, only the `DeliverTx` is persisted.
During DeliverTx the state transition function is applied to the blockchain state and the transactions The BaseApp requires stores to be mounted via capabilities keys - handlers can
need to be fully executed. only access stores they're given the key to. The `baseApp` ensures all stores are
properly loaded, cached, and committed. One mounted store is considered the
"main" - it holds the latest block header, from which we can find and load the
most recent state.
BaseApp is responsible for managing the context passed into handlers - it makes the block header available and provides the right stores for CheckTx and DeliverTx. BaseApp is completely agnostic to serialization formats. The BaseApp distinguishes between two handler types - the `AnteHandler` and the
`MsgHandler`. The former is a global validity check (checking nonces, sigs and
sufficient balances to pay fees, e.g. things that apply to all transaction from
all modules), the later is the full state transition function.
During `CheckTx` the state transition function is only applied to the `checkTxState`
and should return before any expensive state transitions are run
(this is up to each developer). It also needs to return the estimated gas cost.
During `DeliverTx` the state transition function is applied to the blockchain
state and the transactions need to be fully executed.
The BaseApp is responsible for managing the context passed into handlers -
it makes the block header available and provides the right stores for `CheckTx`
and `DeliverTx`. BaseApp is completely agnostic to serialization formats.
## Transaction Life Cycle
During the execution of a transaction, it may pass through both `CheckTx` and
`DeliverTx` as defined in the ABCI specification. `CheckTx` is executed by the
proposing validator and is used for the Tendermint mempool for all full nodes.
Both `CheckTx` and `DeliverTx` execute the application's AnteHandler (if
defined), where the AnteHandler is responsible for pre-message validation
checks such as account and signature validation, fee deduction and collection,
and incrementing sequence numbers.
### CheckTx
During the execution of `CheckTx`, only the AnteHandler is executed.
State transitions due to the AnteHandler are persisted between subsequent calls
of `CheckTx` in the check-tx state, unless the AnteHandler fails and aborts.
### DeliverTx
During the execution of `DeliverTx`, the AnteHandler and Handler is executed.
The transaction execution during `DeliverTx` operates in a similar fashion to
`CheckTx`. However, state transitions that occur during the AnteHandler are
persisted even when the following Handler processing logic fails.
It is possible that a malicious proposer may include a transaction in a block
that fails the AnteHandler. In this case, all state transitions for the
offending transaction are discarded.