From fe7ae1151d522da53569dd755ee144bc06307ba2 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Fri, 6 Jul 2018 10:47:07 -0400 Subject: [PATCH] baseapp: refactor tests * simplify mock tx type, msgs, and handlers * remove dependencies on auth and bank * dedup with setupBaseApp * lots of comments and cleanup * fixes where we weren't checking results * use some table driven tests * remove TestValidatorChange - its not testing anything since baseapp doesnt track validator changes * prepare for CheckTx only running the AnteHandler * fix runTx gas handling and add more tests * new tests for multi-msgs --- baseapp/baseapp.go | 117 +++--- baseapp/baseapp_test.go | 841 +++++++++++++++++++++++---------------- baseapp/multimsg_test.go | 352 ---------------- baseapp/router.go | 2 +- 4 files changed, 568 insertions(+), 744 deletions(-) delete mode 100644 baseapp/multimsg_test.go diff --git a/baseapp/baseapp.go b/baseapp/baseapp.go index afc060cc35..b3c8945f07 100644 --- a/baseapp/baseapp.go +++ b/baseapp/baseapp.go @@ -122,6 +122,11 @@ func (app *BaseApp) SetTxDecoder(txDecoder sdk.TxDecoder) { } // default custom logic for transaction decoding +// TODO: remove auth and wire dependencies from baseapp +// - move this to auth.DefaultTxDecoder +// - set the default here to JSON decode like docs/examples/app1 (it will fail +// for multiple messages ;)) +// - pass a TxDecoder into NewBaseApp, instead of a codec. func defaultTxDecoder(cdc *wire.Codec) sdk.TxDecoder { return func(txBytes []byte) (sdk.Tx, sdk.Error) { var tx = auth.StdTx{} @@ -370,6 +375,8 @@ func (app *BaseApp) Query(req abci.RequestQuery) (res abci.ResponseQuery) { return app.FilterPeerByAddrPort(path[3]) } if path[2] == "pubkey" { + // TODO: this should be changed to `id` + // NOTE: this changed in tendermint and we didn't notice... return app.FilterPeerByPubKey(path[3]) } } @@ -468,28 +475,38 @@ func (app *BaseApp) Deliver(tx sdk.Tx) (result sdk.Result) { // txBytes may be nil in some cases, eg. in tests. // Also, in the future we may support "internal" transactions. 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 int64 + ctx := app.getContextForAnte(mode, txBytes) + // Handle any panics. defer func() { if r := recover(); r != nil { - switch r.(type) { + switch rType := r.(type) { case sdk.ErrorOutOfGas: - log := fmt.Sprintf("out of gas in location: %v", r.(sdk.ErrorOutOfGas).Descriptor) + log := fmt.Sprintf("out of gas in location: %v", rType.Descriptor) result = sdk.ErrOutOfGas(log).Result() default: log := fmt.Sprintf("recovered: %v\nstack:\n%v", r, string(debug.Stack())) result = sdk.ErrInternal(log).Result() } } + result.GasWanted = gasWanted + result.GasUsed = ctx.GasMeter().GasConsumed() }() // Get the Msg. var msgs = tx.GetMsgs() if msgs == nil || len(msgs) == 0 { + // TODO: probably shouldn't be ErrInternal. Maybe new ErrInvalidMessage, or ? return sdk.ErrInternal("Tx.GetMsgs() must return at least one message in list").Result() } for _, msg := range msgs { - // Validate the Msg + // Validate the Msg. err := msg.ValidateBasic() if err != nil { err = err.WithDefaultCodespace(sdk.CodespaceRoot) @@ -497,29 +514,16 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte, tx sdk.Tx) (result sdk } } - // Get the context - var ctx sdk.Context - if mode == runTxModeCheck || mode == runTxModeSimulate { - ctx = app.checkState.ctx.WithTxBytes(txBytes) - } else { - ctx = app.deliverState.ctx.WithTxBytes(txBytes) - ctx = ctx.WithSigningValidators(app.signedValidators) - } - - // Simulate a DeliverTx for gas calculation - if mode == runTxModeSimulate { - ctx = ctx.WithIsCheckTx(false) - } - // Run the ante handler. if app.anteHandler != nil { - newCtx, result, abort := app.anteHandler(ctx, tx) + newCtx, anteResult, abort := app.anteHandler(ctx, tx) if abort { - return result + return anteResult } if !newCtx.IsZero() { ctx = newCtx } + gasWanted = anteResult.GasWanted } // Get the correct cache @@ -534,9 +538,12 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte, tx sdk.Tx) (result sdk ctx = ctx.WithMultiStore(msCache) } - finalResult := sdk.Result{} + // accumulate results var logs []string - for i, msg := range msgs { + var data []byte + var tags sdk.Tags + var code sdk.ABCICodeType + for msgIdx, msg := range msgs { // Match route. msgType := msg.Type() handler := app.router.Route(msgType) @@ -544,43 +551,61 @@ func (app *BaseApp) runTx(mode runTxMode, txBytes []byte, tx sdk.Tx) (result sdk return sdk.ErrUnknownRequest("Unrecognized Msg type: " + msgType).Result() } - result = handler(ctx, msg) + msgResult := handler(ctx, msg) - // Set gas utilized - finalResult.GasUsed += ctx.GasMeter().GasConsumed() - finalResult.GasWanted += result.GasWanted + // NOTE: GasWanted is determined by ante handler and + // GasUsed by the GasMeter // Append Data and Tags - finalResult.Data = append(finalResult.Data, result.Data...) - finalResult.Tags = append(finalResult.Tags, result.Tags...) - - // Construct usable logs in multi-message transactions. Messages are 1-indexed in logs. - logs = append(logs, fmt.Sprintf("Msg %d: %s", i+1, finalResult.Log)) + data = append(data, msgResult.Data...) + tags = append(tags, msgResult.Tags...) // Stop execution and return on first failed message. - if !result.IsOK() { - if len(msgs) == 1 { - return result - } - result.GasUsed = finalResult.GasUsed - if i == 0 { - result.Log = fmt.Sprintf("Msg 1 failed: %s", result.Log) - } else { - result.Log = fmt.Sprintf("Msg 1-%d Passed. Msg %d failed: %s", i, i+1, result.Log) - } - return result + if !msgResult.IsOK() { + logs = append(logs, fmt.Sprintf("Msg %d failed: %s", msgIdx, msgResult.Log)) + code = msgResult.Code + break } + + // Construct usable logs in multi-message transactions. + logs = append(logs, fmt.Sprintf("Msg %d: %s", msgIdx, msgResult.Log)) } - // If not a simulated run and result was successful, write to app.checkState.ms or app.deliverState.ms - // Only update state if all messages pass. - if mode != runTxModeSimulate && result.IsOK() { + // Set the final gas values. + result = sdk.Result{ + Code: code, + Data: data, + Log: strings.Join(logs, "\n"), + GasWanted: gasWanted, + GasUsed: ctx.GasMeter().GasConsumed(), + // TODO: FeeAmount/FeeDenom + Tags: tags, + } + + // Only update state if all messages pass and we're not in a simulation. + if result.IsOK() && mode != runTxModeSimulate { msCache.Write() } - finalResult.Log = strings.Join(logs, "\n") + return result +} - return finalResult +func (app *BaseApp) getContextForAnte(mode runTxMode, txBytes []byte) sdk.Context { + var ctx sdk.Context + + // Get the context. + if mode == runTxModeCheck || mode == runTxModeSimulate { + ctx = app.checkState.ctx.WithTxBytes(txBytes) + } else { + ctx = app.deliverState.ctx.WithTxBytes(txBytes) + ctx = ctx.WithSigningValidators(app.signedValidators) + } + + // Simulate a DeliverTx for gas calculation. + if mode == runTxModeSimulate { + ctx = ctx.WithIsCheckTx(false) + } + return ctx } // Implements ABCI diff --git a/baseapp/baseapp_test.go b/baseapp/baseapp_test.go index 2ed89c53ff..eacc673503 100644 --- a/baseapp/baseapp_test.go +++ b/baseapp/baseapp_test.go @@ -1,6 +1,8 @@ package baseapp import ( + "bytes" + "encoding/binary" "fmt" "os" "testing" @@ -9,17 +11,16 @@ import ( "github.com/stretchr/testify/require" abci "github.com/tendermint/tendermint/abci/types" - "github.com/tendermint/tendermint/crypto" - cmn "github.com/tendermint/tendermint/libs/common" dbm "github.com/tendermint/tendermint/libs/db" "github.com/tendermint/tendermint/libs/log" - tmtypes "github.com/tendermint/tendermint/types" sdk "github.com/cosmos/cosmos-sdk/types" "github.com/cosmos/cosmos-sdk/wire" - "github.com/cosmos/cosmos-sdk/x/auth" ) +//------------------------------------------------------------------------------------------ +// Helpers for setup. Most tests should be able to use setupBaseApp + func defaultLogger() log.Logger { return log.NewTMLogger(log.NewSyncWriter(os.Stdout)).With("module", "sdk/app") } @@ -28,14 +29,27 @@ func newBaseApp(name string) *BaseApp { logger := defaultLogger() db := dbm.NewMemDB() codec := wire.NewCodec() - auth.RegisterBaseAccount(codec) + registerTestCodec(codec) return NewBaseApp(name, codec, logger, db) } -func TestMountStores(t *testing.T) { - name := t.Name() - app := newBaseApp(name) - require.Equal(t, name, app.Name()) +func registerTestCodec(cdc *wire.Codec) { + // register Tx, Msg + sdk.RegisterWire(cdc) + + // register test types + cdc.RegisterConcrete(&txTest{}, "cosmos-sdk/baseapp/txTest", nil) + cdc.RegisterConcrete(&msgCounter{}, "cosmos-sdk/baseapp/msgCounter", nil) + cdc.RegisterConcrete(&msgCounter2{}, "cosmos-sdk/baseapp/msgCounter2", nil) + cdc.RegisterConcrete(&msgNoRoute{}, "cosmos-sdk/baseapp/msgNoRoute", nil) +} + +// simple one store baseapp +func setupBaseApp(t *testing.T) (*BaseApp, *sdk.KVStoreKey, *sdk.KVStoreKey) { + app := newBaseApp(t.Name()) + require.Equal(t, t.Name(), app.Name()) + + app.SetTxDecoder(testTxDecoder(app.cdc)) // make some cap keys capKey1 := sdk.NewKVStoreKey("key1") @@ -49,6 +63,14 @@ func TestMountStores(t *testing.T) { // stores are mounted err := app.LoadLatestVersion(capKey1) require.Nil(t, err) + return app, capKey1, capKey2 +} + +//------------------------------------------------------------------------------------------ +// test mounting and loading stores + +func TestMountStores(t *testing.T) { + app, capKey1, capKey2 := setupBaseApp(t) // check both stores store1 := app.cms.GetCommitKVStore(capKey1) @@ -73,16 +95,19 @@ func TestLoadVersion(t *testing.T) { emptyCommitID := sdk.CommitID{} + // fresh store has zero/empty last commit lastHeight := app.LastBlockHeight() lastID := app.LastCommitID() require.Equal(t, int64(0), lastHeight) require.Equal(t, emptyCommitID, lastID) - // execute some blocks + // execute a block, collect commit ID header := abci.Header{Height: 1} app.BeginBlock(abci.RequestBeginBlock{Header: header}) res := app.Commit() commitID1 := sdk.CommitID{1, res.Data} + + // execute a block, collect commit ID header = abci.Header{Height: 2} app.BeginBlock(abci.RequestBeginBlock{Header: header}) res = app.Commit() @@ -140,6 +165,9 @@ func testLoadVersionHelper(t *testing.T, app *BaseApp, expectedHeight int64, exp } */ +//------------------------------------------------------------------------------------------ +// test some basic abci/baseapp functionality + // Test that txs can be unmarshalled and read and that // correct error codes are returned when not func TestTxDecoder(t *testing.T) { @@ -162,26 +190,26 @@ func TestInfo(t *testing.T) { // ----- test a proper response ------- // TODO - } +//------------------------------------------------------------------------------------------ +// InitChain, BeginBlock, EndBlock + func TestInitChainer(t *testing.T) { name := t.Name() + // keep the db and logger ourselves so + // we can reload the same app later db := dbm.NewMemDB() logger := defaultLogger() app := NewBaseApp(name, nil, logger, db) - // make cap keys and mount the stores - // NOTE/TODO: mounting multiple stores is broken - // see https://github.com/cosmos/cosmos-sdk/issues/532 capKey := sdk.NewKVStoreKey("main") capKey2 := sdk.NewKVStoreKey("key2") app.MountStoresIAVL(capKey, capKey2) err := app.LoadLatestVersion(capKey) // needed to make stores non-nil require.Nil(t, err) + // set a value in the store on init chain key, value := []byte("hello"), []byte("goodbye") - - // initChainer sets a value in the store var initChainer sdk.InitChainer = func(ctx sdk.Context, req abci.RequestInitChain) abci.ResponseInitChain { store := ctx.KVStore(capKey) store.Set(key, value) @@ -231,277 +259,556 @@ func TestInitChainer(t *testing.T) { require.Equal(t, value, res.Value) } -func getStateCheckingHandler(t *testing.T, capKey *sdk.KVStoreKey, txPerHeight int, checkHeader bool) func(ctx sdk.Context, msg sdk.Msg) sdk.Result { - counter := 0 - return func(ctx sdk.Context, msg sdk.Msg) sdk.Result { - store := ctx.KVStore(capKey) - // Checking state gets updated between checkTx's / DeliverTx's - // on the store within a block. - if counter > 0 { - // check previous value in store - counterBytes := []byte{byte(counter - 1)} - prevBytes := store.Get(counterBytes) - require.Equal(t, counterBytes, prevBytes) - } +//------------------------------------------------------------------------------------------ +// Mock tx, msgs, and mapper for the baseapp tests. +// Self-contained, just uses counters. +// We don't care about signatures, coins, accounts, etc. in the baseapp. - // set the current counter in the store - counterBytes := []byte{byte(counter)} - store.Set(counterBytes, counterBytes) - - // check that we can see the current header - // wrapped in an if, so it can be reused between CheckTx and DeliverTx tests. - if checkHeader { - thisHeader := ctx.BlockHeader() - height := int64((counter / txPerHeight) + 1) - require.Equal(t, height, thisHeader.Height) - } - - counter++ - return sdk.Result{} - } +// Simple tx with a list of Msgs. +type txTest struct { + Msgs []sdk.Msg + Counter int64 } -// A mock transaction that has a validation which can fail. -type testTx struct { - positiveNum int64 +// Implements Tx +func (tx txTest) GetMsgs() []sdk.Msg { return tx.Msgs } + +const ( + typeMsgCounter = "msgCounter" + typeMsgCounter2 = "msgCounterTwo" // NOTE: no numerics (?) +) + +// ValidateBasic() fails on negative counters. +// Otherwise it's up to the handlers +type msgCounter struct { + Counter int64 } -const msgType2 = "testTx" - -func (tx testTx) Type() string { return msgType2 } -func (tx testTx) GetMemo() string { return "" } -func (tx testTx) GetMsgs() []sdk.Msg { return []sdk.Msg{tx} } -func (tx testTx) GetSignBytes() []byte { return nil } -func (tx testTx) GetSigners() []sdk.Address { return nil } -func (tx testTx) GetSignatures() []auth.StdSignature { return nil } -func (tx testTx) ValidateBasic() sdk.Error { - if tx.positiveNum >= 0 { +// Implements Msg +func (msg msgCounter) Type() string { return typeMsgCounter } +func (msg msgCounter) GetSignBytes() []byte { return nil } +func (msg msgCounter) GetSigners() []sdk.Address { return nil } +func (msg msgCounter) ValidateBasic() sdk.Error { + if msg.Counter >= 0 { return nil } - return sdk.ErrTxDecode("positiveNum should be a non-negative integer.") + return sdk.ErrInvalidSequence("counter should be a non-negative integer.") } +func newTxCounter(txInt int64, msgInts ...int64) *txTest { + var msgs []sdk.Msg + for _, msgInt := range msgInts { + msgs = append(msgs, msgCounter{msgInt}) + } + return &txTest{msgs, txInt} +} + +// a msg we dont know how to route +type msgNoRoute struct { + msgCounter +} + +func (tx msgNoRoute) Type() string { return "noroute" } + +// a msg we dont know how to decode +type msgNoDecode struct { + msgCounter +} + +func (tx msgNoDecode) Type() string { return typeMsgCounter } + +// Another counter msg. Duplicate of msgCounter +type msgCounter2 struct { + Counter int64 +} + +// Implements Msg +func (msg msgCounter2) Type() string { return typeMsgCounter2 } +func (msg msgCounter2) GetSignBytes() []byte { return nil } +func (msg msgCounter2) GetSigners() []sdk.Address { return nil } +func (msg msgCounter2) ValidateBasic() sdk.Error { + if msg.Counter >= 0 { + return nil + } + return sdk.ErrInvalidSequence("counter should be a non-negative integer.") +} + +// amino decode +func testTxDecoder(cdc *wire.Codec) sdk.TxDecoder { + return func(txBytes []byte) (sdk.Tx, sdk.Error) { + var tx txTest + if len(txBytes) == 0 { + return nil, sdk.ErrTxDecode("txBytes are empty") + } + err := cdc.UnmarshalBinary(txBytes, &tx) + if err != nil { + return nil, sdk.ErrTxDecode("").TraceSDK(err.Error()) + } + return tx, nil + } +} + +func anteHandlerTxTest(t *testing.T, capKey *sdk.KVStoreKey, storeKey []byte) sdk.AnteHandler { + return func(ctx sdk.Context, tx sdk.Tx) (newCtx sdk.Context, res sdk.Result, abort bool) { + store := ctx.KVStore(capKey) + msgCounter := tx.(txTest).Counter + res = incrementingCounter(t, store, storeKey, msgCounter) + return + } +} + +func handlerMsgCounter(t *testing.T, capKey *sdk.KVStoreKey, deliverKey []byte) sdk.Handler { + return func(ctx sdk.Context, msg sdk.Msg) sdk.Result { + store := ctx.KVStore(capKey) + var msgCount int64 + switch m := msg.(type) { + case *msgCounter: + msgCount = m.Counter + case *msgCounter2: + msgCount = m.Counter + } + return incrementingCounter(t, store, deliverKey, msgCount) + } +} + +//----------------------------------------------------------------- +// simple int mapper + +func i2b(i int64) []byte { + return []byte{byte(i)} +} + +func getIntFromStore(store sdk.KVStore, key []byte) int64 { + bz := store.Get(key) + if len(bz) == 0 { + return 0 + } + i, err := binary.ReadVarint(bytes.NewBuffer(bz)) + if err != nil { + panic(err) + } + return i +} + +func setIntOnStore(store sdk.KVStore, key []byte, i int64) { + bz := make([]byte, 8) + n := binary.PutVarint(bz, i) + store.Set(key, bz[:n]) +} + +// check counter matches what's in store. +// increment and store +func incrementingCounter(t *testing.T, store sdk.KVStore, counterKey []byte, counter int64) (res sdk.Result) { + storedCounter := getIntFromStore(store, counterKey) + require.Equal(t, storedCounter, counter) + setIntOnStore(store, counterKey, counter+1) + return +} + +//--------------------------------------------------------------------- +// Tx processing - CheckTx, DeliverTx, SimulateTx. +// These tests use the serialized tx as input, while most others will use the +// Check(), Deliver(), Simulate() methods directly. +// Ensure that Check/Deliver/Simulate work as expected with the store. + // Test that successive CheckTx can see each others' effects // on the store within a block, and that the CheckTx state -// gets reset to the latest Committed state during Commit +// gets reset to the latest committed state during Commit func TestCheckTx(t *testing.T) { - // Initialize an app for testing - app := newBaseApp(t.Name()) - // make a cap key and mount the store - capKey := sdk.NewKVStoreKey("main") - app.MountStoresIAVL(capKey) - err := app.LoadLatestVersion(capKey) // needed to make stores non-nil - require.Nil(t, err) - app.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx) (newCtx sdk.Context, res sdk.Result, abort bool) { return }) + app, capKey, _ := setupBaseApp(t) - txPerHeight := 3 - app.Router().AddRoute(msgType, getStateCheckingHandler(t, capKey, txPerHeight, false)). - AddRoute(msgType2, func(ctx sdk.Context, msg sdk.Msg) (res sdk.Result) { return }) - tx := testUpdatePowerTx{} // doesn't matter - for i := 0; i < txPerHeight; i++ { - app.Check(tx) + // This ante handler reads the key and checks that the value matches the current counter. + // This ensures changes to the kvstore persist across successive CheckTx. + counterKey := []byte("counter-key") + app.SetAnteHandler(anteHandlerTxTest(t, capKey, counterKey)) + + nTxs := int64(5) + + // TODO: can remove this once CheckTx doesnt process msgs. + app.Router().AddRoute(typeMsgCounter, func(ctx sdk.Context, msg sdk.Msg) sdk.Result { return sdk.Result{} }) + + app.InitChain(abci.RequestInitChain{}) + + for i := int64(0); i < nTxs; i++ { + tx := newTxCounter(i, 0) + txBytes, err := app.cdc.MarshalBinary(tx) + require.NoError(t, err) + r := app.CheckTx(txBytes) + assert.True(t, r.IsOK(), fmt.Sprintf("%v", r)) } - // If it gets to this point, then successive CheckTx's can see the effects of - // other CheckTx's on the block. The following checks that if another block - // is committed, the CheckTx State will reset. + + checkStateStore := app.checkState.ctx.KVStore(capKey) + storedCounter := getIntFromStore(checkStateStore, counterKey) + + // Ensure AnteHandler ran + require.Equal(t, nTxs, storedCounter) + + // If a block is committed, CheckTx state should be reset. app.BeginBlock(abci.RequestBeginBlock{}) - tx2 := testTx{} - for i := 0; i < txPerHeight; i++ { - app.Deliver(tx2) - } app.EndBlock(abci.RequestEndBlock{}) app.Commit() - checkStateStore := app.checkState.ctx.KVStore(capKey) - for i := 0; i < txPerHeight; i++ { - storedValue := checkStateStore.Get([]byte{byte(i)}) - require.Nil(t, storedValue) - } + checkStateStore = app.checkState.ctx.KVStore(capKey) + storedBytes := checkStateStore.Get(counterKey) + require.Nil(t, storedBytes) } // Test that successive DeliverTx can see each others' effects // on the store, both within and across blocks. func TestDeliverTx(t *testing.T) { - app := newBaseApp(t.Name()) + app, capKey, _ := setupBaseApp(t) - // make a cap key and mount the store - capKey := sdk.NewKVStoreKey("main") - app.MountStoresIAVL(capKey) - err := app.LoadLatestVersion(capKey) // needed to make stores non-nil - require.Nil(t, err) + // test increments in the ante + anteKey := []byte("ante-key") + app.SetAnteHandler(anteHandlerTxTest(t, capKey, anteKey)) - txPerHeight := 2 - app.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx) (newCtx sdk.Context, res sdk.Result, abort bool) { return }) - app.Router().AddRoute(msgType, getStateCheckingHandler(t, capKey, txPerHeight, true)) - - tx := testUpdatePowerTx{} // doesn't matter - header := abci.Header{AppHash: []byte("apphash")} + // test increments in the handler + deliverKey := []byte("deliver-key") + app.Router().AddRoute(typeMsgCounter, handlerMsgCounter(t, capKey, deliverKey)) nBlocks := 3 + txPerHeight := 5 for blockN := 0; blockN < nBlocks; blockN++ { - // block1 - header.Height = int64(blockN + 1) - app.BeginBlock(abci.RequestBeginBlock{Header: header}) + app.BeginBlock(abci.RequestBeginBlock{}) for i := 0; i < txPerHeight; i++ { - app.Deliver(tx) + counter := int64(blockN*txPerHeight + i) + tx := newTxCounter(counter, counter) + txBytes, err := app.cdc.MarshalBinary(tx) + require.NoError(t, err) + res := app.DeliverTx(txBytes) + require.True(t, res.IsOK(), fmt.Sprintf("%v", res)) } app.EndBlock(abci.RequestEndBlock{}) app.Commit() } } +// Number of messages doesn't matter to CheckTx. +func TestMultiMsgCheckTx(t *testing.T) { + // TODO: ensure we get the same results + // with one message or many +} + +// One call to DeliverTx should process all the messages, in order. +func TestMultiMsgDeliverTx(t *testing.T) { + app, capKey, _ := setupBaseApp(t) + + // increment the tx counter + anteKey := []byte("ante-key") + app.SetAnteHandler(anteHandlerTxTest(t, capKey, anteKey)) + + // increment the msg counter + deliverKey := []byte("deliver-key") + deliverKey2 := []byte("deliver-key2") + app.Router().AddRoute(typeMsgCounter, handlerMsgCounter(t, capKey, deliverKey)) + app.Router().AddRoute(typeMsgCounter2, handlerMsgCounter(t, capKey, deliverKey2)) + + // run a multi-msg tx + // with all msgs the same type + { + app.BeginBlock(abci.RequestBeginBlock{}) + tx := newTxCounter(0, 0, 1, 2) + txBytes, err := app.cdc.MarshalBinary(tx) + require.NoError(t, err) + res := app.DeliverTx(txBytes) + require.True(t, res.IsOK(), fmt.Sprintf("%v", res)) + + store := app.deliverState.ctx.KVStore(capKey) + + // tx counter only incremented once + txCounter := getIntFromStore(store, anteKey) + require.Equal(t, int64(1), txCounter) + + // msg counter incremented three times + msgCounter := getIntFromStore(store, deliverKey) + require.Equal(t, int64(3), msgCounter) + } + + // replace the second message with a msgCounter2 + { + tx := newTxCounter(1, 3) + tx.Msgs = append(tx.Msgs, msgCounter2{0}) + tx.Msgs = append(tx.Msgs, msgCounter2{1}) + txBytes, err := app.cdc.MarshalBinary(tx) + require.NoError(t, err) + res := app.DeliverTx(txBytes) + require.True(t, res.IsOK(), fmt.Sprintf("%v", res)) + + store := app.deliverState.ctx.KVStore(capKey) + + // tx counter only incremented once + txCounter := getIntFromStore(store, anteKey) + require.Equal(t, int64(2), txCounter) + + // original counter increments by one + // new counter increments by two + msgCounter := getIntFromStore(store, deliverKey) + require.Equal(t, int64(4), msgCounter) + msgCounter2 := getIntFromStore(store, deliverKey2) + require.Equal(t, int64(2), msgCounter2) + } +} + +// Interleave calls to Check and Deliver and ensure +// that there is no cross-talk. Check sees results of the previous Check calls +// and Deliver sees that of the previous Deliver calls, but they don't see eachother. +func TestConcurrentCheckDeliver(t *testing.T) { + // TODO +} + +// Simulate a transaction that uses gas to compute the gas. +// Simulate() and Query("/app/simulate", txBytes) should give +// the same results. func TestSimulateTx(t *testing.T) { - app := newBaseApp(t.Name()) + app, _, _ := setupBaseApp(t) - // make a cap key and mount the store - capKey := sdk.NewKVStoreKey("main") - app.MountStoresIAVL(capKey) - err := app.LoadLatestVersion(capKey) // needed to make stores non-nil - require.Nil(t, err) - - counter := 0 - app.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx) (newCtx sdk.Context, res sdk.Result, abort bool) { return }) - app.Router().AddRoute(msgType, func(ctx sdk.Context, msg sdk.Msg) sdk.Result { - ctx.GasMeter().ConsumeGas(10, "test") - store := ctx.KVStore(capKey) - // ensure store is never written - require.Nil(t, store.Get([]byte("key"))) - store.Set([]byte("key"), []byte("value")) - // check we can see the current header - thisHeader := ctx.BlockHeader() - height := int64(counter) - require.Equal(t, height, thisHeader.Height) - counter++ - return sdk.Result{} + gasConsumed := int64(5) + app.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx) (newCtx sdk.Context, res sdk.Result, abort bool) { + newCtx = ctx.WithGasMeter(sdk.NewGasMeter(gasConsumed)) + return }) - tx := testUpdatePowerTx{} // doesn't matter - header := abci.Header{AppHash: []byte("apphash")} - - app.SetTxDecoder(func(txBytes []byte) (sdk.Tx, sdk.Error) { - var ttx testUpdatePowerTx - fromJSON(txBytes, &ttx) - return ttx, nil + app.Router().AddRoute(typeMsgCounter, func(ctx sdk.Context, msg sdk.Msg) sdk.Result { + ctx.GasMeter().ConsumeGas(gasConsumed, "test") + return sdk.Result{GasUsed: ctx.GasMeter().GasConsumed()} }) - app.InitChain(abci.RequestInitChain{}) nBlocks := 3 for blockN := 0; blockN < nBlocks; blockN++ { - // block1 - header.Height = int64(blockN + 1) - app.BeginBlock(abci.RequestBeginBlock{Header: header}) + count := int64(blockN + 1) + app.BeginBlock(abci.RequestBeginBlock{}) + + tx := newTxCounter(count, count) + + // simulate a message, check gas reported result := app.Simulate(tx) - require.Equal(t, result.Code, sdk.ABCICodeOK, result.Log) - require.Equal(t, int64(80), result.GasUsed) - counter-- - encoded, err := app.cdc.MarshalJSON(tx) + require.True(t, result.IsOK(), result.Log) + require.Equal(t, int64(gasConsumed), result.GasUsed) + + // simulate again, same result + result = app.Simulate(tx) + require.True(t, result.IsOK(), result.Log) + require.Equal(t, int64(gasConsumed), result.GasUsed) + + // simulate by calling Query with encoded tx + txBytes, err := app.cdc.MarshalBinary(tx) require.Nil(t, err) query := abci.RequestQuery{ Path: "/app/simulate", - Data: encoded, + Data: txBytes, } queryResult := app.Query(query) - require.Equal(t, queryResult.Code, uint32(sdk.ABCICodeOK)) + require.True(t, queryResult.IsOK(), queryResult.Log) + var res sdk.Result app.cdc.MustUnmarshalBinary(queryResult.Value, &res) - require.Equal(t, sdk.ABCICodeOK, res.Code, res.Log) - require.Equal(t, int64(160), res.GasUsed, res.Log) + require.True(t, res.IsOK(), res.Log) + require.Equal(t, gasConsumed, res.GasUsed, res.Log) app.EndBlock(abci.RequestEndBlock{}) app.Commit() } } +//------------------------------------------------------------------------------------------- +// Tx failure cases +// TODO: add more + func TestRunInvalidTransaction(t *testing.T) { - // Initialize an app for testing - app := newBaseApp(t.Name()) - // make a cap key and mount the store - capKey := sdk.NewKVStoreKey("main") - app.MountStoresIAVL(capKey) - err := app.LoadLatestVersion(capKey) // needed to make stores non-nil - require.Nil(t, err) + app, _, _ := setupBaseApp(t) app.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx) (newCtx sdk.Context, res sdk.Result, abort bool) { return }) - app.Router().AddRoute(msgType2, func(ctx sdk.Context, msg sdk.Msg) (res sdk.Result) { return }) + app.Router().AddRoute(typeMsgCounter, func(ctx sdk.Context, msg sdk.Msg) (res sdk.Result) { return }) + app.BeginBlock(abci.RequestBeginBlock{}) - // Transaction where validate fails - invalidTx := testTx{-1} - err1 := app.Deliver(invalidTx) - require.Equal(t, sdk.ToABCICode(sdk.CodespaceRoot, sdk.CodeTxDecode), err1.Code) + + // Transaction with no messages + { + emptyTx := &txTest{} + err := app.Deliver(emptyTx) + require.Equal(t, sdk.ToABCICode(sdk.CodespaceRoot, sdk.CodeInternal), err.Code) + } + + // Transaction where ValidateBasic fails + { + testCases := []struct { + tx *txTest + fail bool + }{ + {newTxCounter(0, 0), false}, + {newTxCounter(-1, 0), false}, + {newTxCounter(100, 100), false}, + {newTxCounter(100, 5, 4, 3, 2, 1), false}, + + {newTxCounter(0, -1), true}, + {newTxCounter(0, 1, -2), true}, + {newTxCounter(0, 1, 2, -10, 5), true}, + } + + for _, testCase := range testCases { + tx := testCase.tx + res := app.Deliver(tx) + if testCase.fail { + require.Equal(t, sdk.ToABCICode(sdk.CodespaceRoot, sdk.CodeInvalidSequence), res.Code) + } else { + require.True(t, res.IsOK(), fmt.Sprintf("%v", res)) + } + } + } + // Transaction with no known route - unknownRouteTx := testUpdatePowerTx{} - err2 := app.Deliver(unknownRouteTx) - require.Equal(t, sdk.ToABCICode(sdk.CodespaceRoot, sdk.CodeUnknownRequest), err2.Code) + { + unknownRouteTx := txTest{[]sdk.Msg{msgNoRoute{}}, 0} + err := app.Deliver(unknownRouteTx) + require.Equal(t, sdk.ToABCICode(sdk.CodespaceRoot, sdk.CodeUnknownRequest), err.Code) + + unknownRouteTx = txTest{[]sdk.Msg{msgCounter{}, msgNoRoute{}}, 0} + err = app.Deliver(unknownRouteTx) + require.Equal(t, sdk.ToABCICode(sdk.CodespaceRoot, sdk.CodeUnknownRequest), err.Code) + } + + // Transaction with an unregistered message + { + tx := newTxCounter(0, 0) + tx.Msgs = append(tx.Msgs, msgNoDecode{}) + + // new codec so we can encode the tx, but we shouldn't be able to decode + newCdc := wire.NewCodec() + registerTestCodec(newCdc) + newCdc.RegisterConcrete(&msgNoDecode{}, "cosmos-sdk/baseapp/msgNoDecode", nil) + + txBytes, err := newCdc.MarshalBinary(tx) + require.NoError(t, err) + res := app.DeliverTx(txBytes) + require.EqualValues(t, sdk.ToABCICode(sdk.CodespaceRoot, sdk.CodeTxDecode), res.Code) + } } // Test that transactions exceeding gas limits fail func TestTxGasLimits(t *testing.T) { - logger := defaultLogger() - db := dbm.NewMemDB() - app := NewBaseApp(t.Name(), nil, logger, db) - - // make a cap key and mount the store - capKey := sdk.NewKVStoreKey("main") - app.MountStoresIAVL(capKey) - err := app.LoadLatestVersion(capKey) // needed to make stores non-nil - require.Nil(t, err) + app, _, _ := setupBaseApp(t) + gasGranted := int64(10) app.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx) (newCtx sdk.Context, res sdk.Result, abort bool) { - newCtx = ctx.WithGasMeter(sdk.NewGasMeter(0)) + newCtx = ctx.WithGasMeter(sdk.NewGasMeter(gasGranted)) + + // NOTE/TODO/XXX: + // AnteHandlers must have their own defer/recover in order + // for the BaseApp to know how much gas was used 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 ... + 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) + res = sdk.ErrOutOfGas(log).Result() + res.GasWanted = gasGranted + res.GasUsed = newCtx.GasMeter().GasConsumed() + default: + panic(r) + } + } + }() + + count := tx.(*txTest).Counter + newCtx.GasMeter().ConsumeGas(count, "counter-ante") + res = sdk.Result{ + GasWanted: gasGranted, + } return }) - app.Router().AddRoute(msgType, func(ctx sdk.Context, msg sdk.Msg) sdk.Result { - ctx.GasMeter().ConsumeGas(10, "counter") + app.Router().AddRoute(typeMsgCounter, func(ctx sdk.Context, msg sdk.Msg) sdk.Result { + count := msg.(msgCounter).Counter + ctx.GasMeter().ConsumeGas(count, "counter-handler") return sdk.Result{} }) - tx := testUpdatePowerTx{} // doesn't matter - header := abci.Header{AppHash: []byte("apphash")} + app.BeginBlock(abci.RequestBeginBlock{}) - app.BeginBlock(abci.RequestBeginBlock{Header: header}) - res := app.Deliver(tx) - require.Equal(t, res.Code, sdk.ToABCICode(sdk.CodespaceRoot, sdk.CodeOutOfGas), "Expected transaction to run out of gas") - app.EndBlock(abci.RequestEndBlock{}) - app.Commit() + testCases := []struct { + tx *txTest + gasUsed int64 + fail bool + }{ + {newTxCounter(0, 0), 0, false}, + {newTxCounter(1, 1), 2, false}, + {newTxCounter(9, 1), 10, false}, + {newTxCounter(1, 9), 10, false}, + {newTxCounter(10, 0), 10, false}, + {newTxCounter(0, 10), 10, false}, + {newTxCounter(0, 8, 2), 10, false}, + {newTxCounter(0, 5, 1, 1, 1, 1, 1), 10, false}, + {newTxCounter(0, 5, 1, 1, 1, 1), 9, false}, + + {newTxCounter(9, 2), 11, true}, + {newTxCounter(2, 9), 11, true}, + {newTxCounter(9, 1, 1), 11, true}, + {newTxCounter(1, 8, 1, 1), 11, true}, + {newTxCounter(11, 0), 11, true}, + {newTxCounter(0, 11), 11, true}, + {newTxCounter(0, 5, 11), 16, true}, + } + + for i, tc := range testCases { + tx := tc.tx + res := app.Deliver(tx) + + // check gas used and wanted + require.Equal(t, tc.gasUsed, res.GasUsed, fmt.Sprintf("%d: %v, %v", i, tc, res)) + + // check for out of gas + if !tc.fail { + require.True(t, res.IsOK(), fmt.Sprintf("%d: %v, %v", i, tc, res)) + } else { + require.Equal(t, res.Code, sdk.ToABCICode(sdk.CodespaceRoot, sdk.CodeOutOfGas), fmt.Sprintf("%d: %v, %v", i, tc, res)) + } + } } +//------------------------------------------------------------------------------------------- +// Queries + // Test that we can only query from the latest committed state. func TestQuery(t *testing.T) { - app := newBaseApp(t.Name()) - - // make a cap key and mount the store - capKey := sdk.NewKVStoreKey("main") - app.MountStoresIAVL(capKey) - err := app.LoadLatestVersion(capKey) // needed to make stores non-nil - require.Nil(t, err) + app, capKey, _ := setupBaseApp(t) key, value := []byte("hello"), []byte("goodbye") + app.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx) (newCtx sdk.Context, res sdk.Result, abort bool) { + store := ctx.KVStore(capKey) + store.Set(key, value) + return + }) - app.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx) (newCtx sdk.Context, res sdk.Result, abort bool) { return }) - app.Router().AddRoute(msgType, func(ctx sdk.Context, msg sdk.Msg) sdk.Result { + app.Router().AddRoute(typeMsgCounter, func(ctx sdk.Context, msg sdk.Msg) sdk.Result { store := ctx.KVStore(capKey) store.Set(key, value) return sdk.Result{} }) + app.InitChain(abci.RequestInitChain{}) + // NOTE: "/store/key1" tells us KVStore + // and the final "/key" says to use the data as the + // key in the given KVStore ... query := abci.RequestQuery{ - Path: "/store/main/key", + Path: "/store/key1/key", Data: key, } + tx := newTxCounter(0, 0) // query is empty before we do anything res := app.Query(query) require.Equal(t, 0, len(res.Value)) - tx := testUpdatePowerTx{} // doesn't matter - // query is still empty after a CheckTx - app.Check(tx) + resTx := app.Check(tx) + require.True(t, resTx.IsOK(), fmt.Sprintf("%v", resTx)) res = app.Query(query) require.Equal(t, 0, len(res.Value)) // query is still empty after a DeliverTx before we commit app.BeginBlock(abci.RequestBeginBlock{}) - app.Deliver(tx) + resTx = app.Deliver(tx) + require.True(t, resTx.IsOK(), fmt.Sprintf("%v", resTx)) res = app.Query(query) require.Equal(t, 0, len(res.Value)) @@ -513,13 +820,7 @@ func TestQuery(t *testing.T) { // Test p2p filter queries func TestP2PQuery(t *testing.T) { - app := newBaseApp(t.Name()) - - // make a cap key and mount the store - capKey := sdk.NewKVStoreKey("main") - app.MountStoresIAVL(capKey) - err := app.LoadLatestVersion(capKey) // needed to make stores non-nil - require.Nil(t, err) + app, _, _ := setupBaseApp(t) app.SetAddrPeerFilter(func(addrport string) abci.ResponseQuery { require.Equal(t, "1.1.1.1:8000", addrport) @@ -543,153 +844,3 @@ func TestP2PQuery(t *testing.T) { res = app.Query(pubkeyQuery) require.Equal(t, uint32(4), res.Code) } - -//---------------------- -// TODO: clean this up - -// A mock transaction to update a validator's voting power. -type testUpdatePowerTx struct { - Addr []byte - NewPower int64 -} - -const msgType = "testUpdatePowerTx" - -func (tx testUpdatePowerTx) Type() string { return msgType } -func (tx testUpdatePowerTx) GetMemo() string { return "" } -func (tx testUpdatePowerTx) GetMsgs() []sdk.Msg { return []sdk.Msg{tx} } -func (tx testUpdatePowerTx) GetSignBytes() []byte { return nil } -func (tx testUpdatePowerTx) ValidateBasic() sdk.Error { return nil } -func (tx testUpdatePowerTx) GetSigners() []sdk.Address { return nil } -func (tx testUpdatePowerTx) GetSignatures() []auth.StdSignature { return nil } - -func TestValidatorChange(t *testing.T) { - - // Create app. - app := newBaseApp(t.Name()) - capKey := sdk.NewKVStoreKey("key") - app.MountStoresIAVL(capKey) - app.SetTxDecoder(func(txBytes []byte) (sdk.Tx, sdk.Error) { - var ttx testUpdatePowerTx - fromJSON(txBytes, &ttx) - return ttx, nil - }) - - app.SetAnteHandler(func(ctx sdk.Context, tx sdk.Tx) (newCtx sdk.Context, res sdk.Result, abort bool) { return }) - app.Router().AddRoute(msgType, func(ctx sdk.Context, msg sdk.Msg) sdk.Result { - // TODO - return sdk.Result{} - }) - - // Load latest state, which should be empty. - err := app.LoadLatestVersion(capKey) - require.Nil(t, err) - require.Equal(t, app.LastBlockHeight(), int64(0)) - - // Create the validators - var numVals = 3 - var valSet = make([]abci.Validator, numVals) - for i := 0; i < numVals; i++ { - valSet[i] = makeVal(secret(i)) - } - - // Initialize the chain - app.InitChain(abci.RequestInitChain{ - Validators: valSet, - }) - - // Simulate the start of a block. - app.BeginBlock(abci.RequestBeginBlock{}) - - // Add 1 to each validator's voting power. - for i, val := range valSet { - tx := testUpdatePowerTx{ - Addr: makePubKey(secret(i)).Address(), - NewPower: val.Power + 1, - } - txBytes := toJSON(tx) - res := app.DeliverTx(txBytes) - require.True(t, res.IsOK(), "%#v\nABCI log: %s", res, res.Log) - } - - // Simulate the end of a block. - // Get the summary of validator updates. - res := app.EndBlock(abci.RequestEndBlock{}) - valUpdates := res.ValidatorUpdates - - // Assert that validator updates are correct. - for _, val := range valSet { - - pubkey, err := tmtypes.PB2TM.PubKey(val.PubKey) - // Sanity - require.Nil(t, err) - - // Find matching update and splice it out. - for j := 0; j < len(valUpdates); j++ { - valUpdate := valUpdates[j] - - updatePubkey, err := tmtypes.PB2TM.PubKey(valUpdate.PubKey) - require.Nil(t, err) - - // Matched. - if updatePubkey.Equals(pubkey) { - require.Equal(t, valUpdate.Power, val.Power+1) - if j < len(valUpdates)-1 { - // Splice it out. - valUpdates = append(valUpdates[:j], valUpdates[j+1:]...) - } - break - } - - // Not matched. - } - } - require.Equal(t, len(valUpdates), 0, "Some validator updates were unexpected") -} - -//---------------------------------------- - -func randPower() int64 { - return cmn.RandInt64() -} - -func makeVal(secret string) abci.Validator { - return abci.Validator{ - PubKey: tmtypes.TM2PB.PubKey(makePubKey(secret)), - Power: randPower(), - } -} - -func makePubKey(secret string) crypto.PubKey { - return makePrivKey(secret).PubKey() -} - -func makePrivKey(secret string) crypto.PrivKey { - privKey := crypto.GenPrivKeyEd25519FromSecret([]byte(secret)) - return privKey -} - -func secret(index int) string { - return fmt.Sprintf("secret%d", index) -} - -func copyVal(val abci.Validator) abci.Validator { - // val2 := *val - // return &val2 - return val -} - -func toJSON(o interface{}) []byte { - bz, err := wire.Cdc.MarshalJSON(o) - if err != nil { - panic(err) - } - return bz -} - -func fromJSON(bz []byte, ptr interface{}) { - err := wire.Cdc.UnmarshalJSON(bz, ptr) - if err != nil { - panic(err) - } -} diff --git a/baseapp/multimsg_test.go b/baseapp/multimsg_test.go deleted file mode 100644 index 581da4b176..0000000000 --- a/baseapp/multimsg_test.go +++ /dev/null @@ -1,352 +0,0 @@ -package baseapp - -import ( - "encoding/json" - "fmt" - "testing" - - sdk "github.com/cosmos/cosmos-sdk/types" - "github.com/cosmos/cosmos-sdk/wire" - "github.com/cosmos/cosmos-sdk/x/auth" - "github.com/stretchr/testify/require" - abci "github.com/tendermint/tendermint/abci/types" - "github.com/tendermint/tendermint/crypto" -) - -// tests multiple msgs of same type from same address in single tx -func TestMultipleBurn(t *testing.T) { - // Create app. - app := newTestApp(t.Name()) - capKey := sdk.NewKVStoreKey("key") - app.MountStoresIAVL(capKey) - app.SetTxDecoder(func(txBytes []byte) (sdk.Tx, sdk.Error) { - var tx auth.StdTx - fromJSON(txBytes, &tx) - return tx, nil - }) - - err := app.LoadLatestVersion(capKey) - if err != nil { - panic(err) - } - - app.accountMapper = auth.NewAccountMapper(app.cdc, capKey, &auth.BaseAccount{}) - - app.SetAnteHandler(auth.NewAnteHandler(app.accountMapper, auth.FeeCollectionKeeper{})) - - app.Router(). - AddRoute("burn", newHandleBurn(app.accountMapper)). - AddRoute("send", newHandleSpend(app.accountMapper)) - - app.InitChain(abci.RequestInitChain{}) - app.BeginBlock(abci.RequestBeginBlock{}) - - // Set chain-id - app.deliverState.ctx = app.deliverState.ctx.WithChainID(t.Name()) - - priv := makePrivKey("my secret") - addr := priv.PubKey().Address() - - addCoins(app.accountMapper, app.deliverState.ctx, addr, sdk.Coins{{"foocoin", sdk.NewInt(100)}}) - - require.Equal(t, sdk.Coins{{"foocoin", sdk.NewInt(100)}}, app.accountMapper.GetAccount(app.deliverState.ctx, addr).GetCoins(), "Balance did not update") - - msg := testBurnMsg{addr, sdk.Coins{{"foocoin", sdk.NewInt(50)}}} - tx := GenTx(t.Name(), []sdk.Msg{msg, msg}, []int64{0}, []int64{0}, priv) - - res := app.Deliver(tx) - - require.Equal(t, true, res.IsOK(), res.Log) - require.Equal(t, sdk.Coins(nil), getCoins(app.accountMapper, app.deliverState.ctx, addr), "Double burn did not work") -} - -// tests multiples msgs of same type from different addresses in single tx -func TestBurnMultipleOwners(t *testing.T) { - // Create app. - app := newTestApp(t.Name()) - capKey := sdk.NewKVStoreKey("key") - app.MountStoresIAVL(capKey) - app.SetTxDecoder(func(txBytes []byte) (sdk.Tx, sdk.Error) { - var tx auth.StdTx - fromJSON(txBytes, &tx) - return tx, nil - }) - - err := app.LoadLatestVersion(capKey) - if err != nil { - panic(err) - } - - app.accountMapper = auth.NewAccountMapper(app.cdc, capKey, &auth.BaseAccount{}) - - app.SetAnteHandler(auth.NewAnteHandler(app.accountMapper, auth.FeeCollectionKeeper{})) - - app.Router(). - AddRoute("burn", newHandleBurn(app.accountMapper)). - AddRoute("send", newHandleSpend(app.accountMapper)) - - app.InitChain(abci.RequestInitChain{}) - app.BeginBlock(abci.RequestBeginBlock{}) - - // Set chain-id - app.deliverState.ctx = app.deliverState.ctx.WithChainID(t.Name()) - - priv1 := makePrivKey("my secret 1") - addr1 := priv1.PubKey().Address() - - priv2 := makePrivKey("my secret 2") - addr2 := priv2.PubKey().Address() - - // fund accounts - addCoins(app.accountMapper, app.deliverState.ctx, addr1, sdk.Coins{{"foocoin", sdk.NewInt(100)}}) - addCoins(app.accountMapper, app.deliverState.ctx, addr2, sdk.Coins{{"foocoin", sdk.NewInt(100)}}) - - require.Equal(t, sdk.Coins{{"foocoin", sdk.NewInt(100)}}, getCoins(app.accountMapper, app.deliverState.ctx, addr1), "Balance1 did not update") - require.Equal(t, sdk.Coins{{"foocoin", sdk.NewInt(100)}}, getCoins(app.accountMapper, app.deliverState.ctx, addr2), "Balance2 did not update") - - msg1 := testBurnMsg{addr1, sdk.Coins{{"foocoin", sdk.NewInt(100)}}} - msg2 := testBurnMsg{addr2, sdk.Coins{{"foocoin", sdk.NewInt(100)}}} - - // test wrong signers: Address 1 signs both messages - tx := GenTx(t.Name(), []sdk.Msg{msg1, msg2}, []int64{0, 0}, []int64{0, 0}, priv1, priv1) - - res := app.Deliver(tx) - require.Equal(t, sdk.ABCICodeType(0x10003), res.Code, "Wrong signatures passed") - - require.Equal(t, sdk.Coins{{"foocoin", sdk.NewInt(100)}}, getCoins(app.accountMapper, app.deliverState.ctx, addr1), "Balance1 changed after invalid sig") - require.Equal(t, sdk.Coins{{"foocoin", sdk.NewInt(100)}}, getCoins(app.accountMapper, app.deliverState.ctx, addr2), "Balance2 changed after invalid sig") - - // test valid tx - tx = GenTx(t.Name(), []sdk.Msg{msg1, msg2}, []int64{0, 1}, []int64{1, 0}, priv1, priv2) - - res = app.Deliver(tx) - require.Equal(t, true, res.IsOK(), res.Log) - - require.Equal(t, sdk.Coins(nil), getCoins(app.accountMapper, app.deliverState.ctx, addr1), "Balance1 did not change after valid tx") - require.Equal(t, sdk.Coins(nil), getCoins(app.accountMapper, app.deliverState.ctx, addr2), "Balance2 did not change after valid tx") -} - -func getCoins(am auth.AccountMapper, ctx sdk.Context, addr sdk.Address) sdk.Coins { - return am.GetAccount(ctx, addr).GetCoins() -} - -func addCoins(am auth.AccountMapper, ctx sdk.Context, addr sdk.Address, coins sdk.Coins) sdk.Error { - acc := am.GetAccount(ctx, addr) - if acc == nil { - acc = am.NewAccountWithAddress(ctx, addr) - } - err := acc.SetCoins(acc.GetCoins().Plus(coins)) - if err != nil { - fmt.Println(err) - return sdk.ErrInternal(err.Error()) - } - am.SetAccount(ctx, acc) - return nil -} - -// tests different msg types in single tx with different addresses -func TestSendBurn(t *testing.T) { - // Create app. - app := newTestApp(t.Name()) - capKey := sdk.NewKVStoreKey("key") - app.MountStoresIAVL(capKey) - app.SetTxDecoder(func(txBytes []byte) (sdk.Tx, sdk.Error) { - var tx auth.StdTx - fromJSON(txBytes, &tx) - return tx, nil - }) - - err := app.LoadLatestVersion(capKey) - if err != nil { - panic(err) - } - - app.accountMapper = auth.NewAccountMapper(app.cdc, capKey, &auth.BaseAccount{}) - - app.SetAnteHandler(auth.NewAnteHandler(app.accountMapper, auth.FeeCollectionKeeper{})) - - app.Router(). - AddRoute("burn", newHandleBurn(app.accountMapper)). - AddRoute("send", newHandleSpend(app.accountMapper)) - - app.InitChain(abci.RequestInitChain{}) - app.BeginBlock(abci.RequestBeginBlock{}) - - // Set chain-id - app.deliverState.ctx = app.deliverState.ctx.WithChainID(t.Name()) - - priv1 := makePrivKey("my secret 1") - addr1 := priv1.PubKey().Address() - - priv2 := makePrivKey("my secret 2") - addr2 := priv2.PubKey().Address() - - // fund accounts - addCoins(app.accountMapper, app.deliverState.ctx, addr1, sdk.Coins{{"foocoin", sdk.NewInt(100)}}) - acc := app.accountMapper.NewAccountWithAddress(app.deliverState.ctx, addr2) - app.accountMapper.SetAccount(app.deliverState.ctx, acc) - - require.Equal(t, sdk.Coins{{"foocoin", sdk.NewInt(100)}}, getCoins(app.accountMapper, app.deliverState.ctx, addr1), "Balance1 did not update") - - sendMsg := testSendMsg{addr1, addr2, sdk.Coins{{"foocoin", sdk.NewInt(50)}}} - - msg1 := testBurnMsg{addr1, sdk.Coins{{"foocoin", sdk.NewInt(50)}}} - msg2 := testBurnMsg{addr2, sdk.Coins{{"foocoin", sdk.NewInt(50)}}} - - // send then burn - tx := GenTx(t.Name(), []sdk.Msg{sendMsg, msg2, msg1}, []int64{0, 1}, []int64{0, 0}, priv1, priv2) - - res := app.Deliver(tx) - require.Equal(t, true, res.IsOK(), res.Log) - - require.Equal(t, sdk.Coins(nil), getCoins(app.accountMapper, app.deliverState.ctx, addr1), "Balance1 did not change after valid tx") - require.Equal(t, sdk.Coins(nil), getCoins(app.accountMapper, app.deliverState.ctx, addr2), "Balance2 did not change after valid tx") - - // Check that state is only updated if all msgs in tx pass. - addCoins(app.accountMapper, app.deliverState.ctx, addr1, sdk.Coins{{"foocoin", sdk.NewInt(50)}}) - - // burn then send, with fee thats greater than individual tx, but less than combination - tx = GenTxWithFeeAmt(50000, t.Name(), []sdk.Msg{msg1, sendMsg}, []int64{0}, []int64{1}, priv1) - - res = app.Deliver(tx) - require.Equal(t, sdk.ABCICodeType(0x1000c), res.Code, "Allowed tx to pass with insufficient funds") - - // Double check that state is correct after Commit. - app.EndBlock(abci.RequestEndBlock{}) - app.Commit() - - app.BeginBlock(abci.RequestBeginBlock{}) - app.deliverState.ctx = app.deliverState.ctx.WithChainID(t.Name()) - - require.Equal(t, sdk.Coins{{"foocoin", sdk.NewInt(50)}}, getCoins(app.accountMapper, app.deliverState.ctx, addr1), "Allowed valid msg to pass in invalid tx") - require.Equal(t, sdk.Coins(nil), getCoins(app.accountMapper, app.deliverState.ctx, addr2), "Balance2 changed after invalid tx") -} - -// Use burn and send msg types to test multiple msgs in one tx -type testBurnMsg struct { - Addr sdk.Address - Amount sdk.Coins -} - -const msgType3 = "burn" - -func (msg testBurnMsg) Type() string { return msgType3 } -func (msg testBurnMsg) GetSignBytes() []byte { - bz, _ := json.Marshal(msg) - return sdk.MustSortJSON(bz) -} -func (msg testBurnMsg) ValidateBasic() sdk.Error { - if msg.Addr == nil { - return sdk.ErrInvalidAddress("Cannot use nil as Address") - } - return nil -} -func (msg testBurnMsg) GetSigners() []sdk.Address { - return []sdk.Address{msg.Addr} -} - -type testSendMsg struct { - Sender sdk.Address - Receiver sdk.Address - Amount sdk.Coins -} - -const msgType4 = "send" - -func (msg testSendMsg) Type() string { return msgType4 } -func (msg testSendMsg) GetSignBytes() []byte { - bz, _ := json.Marshal(msg) - return sdk.MustSortJSON(bz) -} -func (msg testSendMsg) ValidateBasic() sdk.Error { - if msg.Sender == nil || msg.Receiver == nil { - return sdk.ErrInvalidAddress("Cannot use nil as Address") - } - return nil -} -func (msg testSendMsg) GetSigners() []sdk.Address { - return []sdk.Address{msg.Sender} -} - -// Simple Handlers for burn and send - -func newHandleBurn(am auth.AccountMapper) sdk.Handler { - return func(ctx sdk.Context, msg sdk.Msg) sdk.Result { - ctx.GasMeter().ConsumeGas(20000, "burning coins") - burnMsg := msg.(testBurnMsg) - err := addCoins(am, ctx, burnMsg.Addr, burnMsg.Amount.Negative()) - if err != nil { - return err.Result() - } - return sdk.Result{} - } -} - -func newHandleSpend(am auth.AccountMapper) sdk.Handler { - return func(ctx sdk.Context, msg sdk.Msg) sdk.Result { - ctx.GasMeter().ConsumeGas(40000, "spending coins") - spendMsg := msg.(testSendMsg) - err := addCoins(am, ctx, spendMsg.Sender, spendMsg.Amount.Negative()) - if err != nil { - return err.Result() - } - - err = addCoins(am, ctx, spendMsg.Receiver, spendMsg.Amount) - if err != nil { - return err.Result() - } - return sdk.Result{} - } -} - -// generate a signed transaction -func GenTx(chainID string, msgs []sdk.Msg, accnums []int64, seq []int64, priv ...crypto.PrivKey) auth.StdTx { - return GenTxWithFeeAmt(100000, chainID, msgs, accnums, seq, priv...) -} - -// generate a signed transaction with the given fee amount -func GenTxWithFeeAmt(feeAmt int64, chainID string, msgs []sdk.Msg, accnums []int64, seq []int64, priv ...crypto.PrivKey) auth.StdTx { - // make the transaction free - fee := auth.StdFee{ - sdk.Coins{{"foocoin", sdk.NewInt(0)}}, - feeAmt, - } - - sigs := make([]auth.StdSignature, len(priv)) - for i, p := range priv { - sig, err := p.Sign(auth.StdSignBytes(chainID, accnums[i], seq[i], fee, msgs, "")) - // TODO: replace with proper error handling: - if err != nil { - panic(err) - } - sigs[i] = auth.StdSignature{ - PubKey: p.PubKey(), - Signature: sig, - AccountNumber: accnums[i], - Sequence: seq[i], - } - } - return auth.NewStdTx(msgs, fee, sigs, "") -} - -// spin up simple app for testing -type testApp struct { - *BaseApp - accountMapper auth.AccountMapper -} - -func newTestApp(name string) testApp { - return testApp{ - BaseApp: newBaseApp(name), - } -} - -func MakeCodec() *wire.Codec { - cdc := wire.NewCodec() - cdc.RegisterInterface((*sdk.Msg)(nil), nil) - crypto.RegisterAmino(cdc) - cdc.RegisterInterface((*auth.Account)(nil), nil) - cdc.RegisterConcrete(&auth.BaseAccount{}, "cosmos-sdk/BaseAccount", nil) - cdc.Seal() - return cdc -} diff --git a/baseapp/router.go b/baseapp/router.go index 83efe5dad1..abbbf9e121 100644 --- a/baseapp/router.go +++ b/baseapp/router.go @@ -36,7 +36,7 @@ var isAlpha = regexp.MustCompile(`^[a-zA-Z]+$`).MatchString // AddRoute - TODO add description func (rtr *router) AddRoute(r string, h sdk.Handler) Router { if !isAlpha(r) { - panic("route expressions can only contain alphanumeric characters") + panic("route expressions can only contain alphabet characters") } rtr.routes = append(rtr.routes, route{r, h})