Implement tx counter for transaction info (#621)

* Implement tx counter for transaction info

* Count tx before pre-conditions

* Doc update
This commit is contained in:
Alexander Peters
2021-09-27 21:53:01 +02:00
committed by GitHub
parent b32f53e9a0
commit 58f3776f6e
15 changed files with 267 additions and 13 deletions

View File

@@ -4,6 +4,11 @@
[Full Changelog](https://github.com/CosmWasm/wasmd/compare/v0.19.0...HEAD)
**Implemented Enhancements:**
- Add transaction index implemented as counter [\#601](https://github.com/CosmWasm/wasmd/issues/601)
## [v0.19.0](https://github.com/CosmWasm/wasmd/tree/v0.19.0) (2021-09-15)
[Full Changelog](https://github.com/CosmWasm/wasmd/compare/v0.19.0...v0.18.0)

View File

@@ -63,6 +63,10 @@ from the Cosmos SDK, and enabled them in `app.go`. If so, you can just look
at [`wasmd/app/app.go`](https://github.com/CosmWasm/wasmd/blob/master/app/app.go#)
for how to do so (just search there for lines with `wasm`).
`wasmd` also comes with a custom `ante handler` that adds the TX position in the block into the context
and passes it to the contracts. In order to support this feature you would need to add our custom
ante handler into the `ante handler chain` as in: [`app/ante.go`](https://github.com/CosmWasm/wasmd/blob/master/app/ante.go)
### Copied into your app
Sometimes, however, you will need to copy `x/wasm` into your app. This should

38
app/ante.go Normal file
View File

@@ -0,0 +1,38 @@
package app
import (
wasmkeeper "github.com/CosmWasm/wasmd/x/wasm/keeper"
sdk "github.com/cosmos/cosmos-sdk/types"
"github.com/cosmos/cosmos-sdk/x/auth/ante"
"github.com/cosmos/cosmos-sdk/x/auth/signing"
"github.com/cosmos/cosmos-sdk/x/auth/types"
)
// NewAnteHandler returns an AnteHandler that checks and increments sequence
// numbers, checks signatures & account numbers, and deducts fees from the first
// signer.
func NewAnteHandler(
ak ante.AccountKeeper, bankKeeper types.BankKeeper,
sigGasConsumer ante.SignatureVerificationGasConsumer,
signModeHandler signing.SignModeHandler,
txCounterStoreKey sdk.StoreKey,
) sdk.AnteHandler {
// copied sdk https://github.com/cosmos/cosmos-sdk/blob/v0.42.9/x/auth/ante/ante.go
return sdk.ChainAnteDecorators(
ante.NewSetUpContextDecorator(), // outermost AnteDecorator. SetUpContext must be called first
wasmkeeper.NewCountTXDecorator(txCounterStoreKey),
ante.NewRejectExtensionOptionsDecorator(),
ante.NewMempoolFeeDecorator(),
ante.NewValidateBasicDecorator(),
ante.TxTimeoutHeightDecorator{},
ante.NewValidateMemoDecorator(ak),
ante.NewConsumeGasForTxSizeDecorator(ak),
ante.NewRejectFeeGranterDecorator(),
ante.NewSetPubKeyDecorator(ak), // SetPubKeyDecorator must be called before all signature verification decorators
ante.NewValidateSigCountDecorator(ak),
ante.NewDeductFeeDecorator(ak, bankKeeper),
ante.NewSigGasConsumeDecorator(ak, sigGasConsumer),
ante.NewSigVerificationDecorator(ak, signModeHandler),
ante.NewIncrementSequenceDecorator(ak),
)
}

View File

@@ -486,9 +486,9 @@ func NewWasmApp(logger log.Logger, db dbm.DB, traceStore io.Writer, loadLatest b
app.SetInitChainer(app.InitChainer)
app.SetBeginBlocker(app.BeginBlocker)
app.SetAnteHandler(
ante.NewAnteHandler(
NewAnteHandler(
app.accountKeeper, app.bankKeeper, ante.DefaultSigVerificationGasConsumer,
encodingConfig.TxConfig.SignModeHandler(),
encodingConfig.TxConfig.SignModeHandler(), keys[wasm.StoreKey],
),
)
app.SetEndBlocker(app.EndBlocker)

View File

@@ -8,7 +8,7 @@ echo "## Add new CosmWasm contract"
RESP=$(wasmd tx wasm store "$DIR/../../x/wasm/keeper/testdata/hackatom.wasm" \
--from validator --gas 1500000 -y --chain-id=testing --node=http://localhost:26657 -b block)
CODE_ID=$(echo "$RESP" | jq -r '.logs[0].events[0].attributes[-1].value')
CODE_ID=$(echo "$RESP" | jq -r '.logs[0].events[1].attributes[-1].value')
echo "* Code id: $CODE_ID"
echo "* Download code"
TMPDIR=$(mktemp -t wasmdXXXXXX)
@@ -57,7 +57,7 @@ echo "### Upload new code"
RESP=$(wasmd tx wasm store "$DIR/../../x/wasm/keeper/testdata/burner.wasm" \
--from validator --gas 1000000 -y --chain-id=testing --node=http://localhost:26657 -b block)
BURNER_CODE_ID=$(echo "$RESP" | jq -r '.logs[0].events[0].attributes[-1].value')
BURNER_CODE_ID=$(echo "$RESP" | jq -r '.logs[0].events[1].attributes[-1].value')
echo "### Migrate to code id: $BURNER_CODE_ID"
DEST_ACCOUNT=$(wasmd keys show fred -a)

View File

@@ -74,6 +74,7 @@ var (
NewQuerier = keeper.Querier
ContractFromPortID = keeper.ContractFromPortID
WithWasmEngine = keeper.WithWasmEngine
NewCountTXDecorator = keeper.NewCountTXDecorator
// variable aliases
ModuleCdc = types.ModuleCdc

53
x/wasm/keeper/ante.go Normal file
View File

@@ -0,0 +1,53 @@
package keeper
import (
"encoding/binary"
"github.com/CosmWasm/wasmd/x/wasm/types"
sdk "github.com/cosmos/cosmos-sdk/types"
)
// CountTXDecorator ante handler to count the tx position in a block.
type CountTXDecorator struct {
storeKey sdk.StoreKey
}
// NewCountTXDecorator constructor
func NewCountTXDecorator(storeKey sdk.StoreKey) *CountTXDecorator {
return &CountTXDecorator{storeKey: storeKey}
}
// AnteHandle handler stores a tx counter with current height encoded in the store to let the app handle
// global rollback behavior instead of keeping state in the handler itself.
// The ante handler passes the counter value via sdk.Context upstream. See `types.TXCounter(ctx)` to read the value.
// Simulations don't get a tx counter value assigned.
func (a CountTXDecorator) AnteHandle(ctx sdk.Context, tx sdk.Tx, simulate bool, next sdk.AnteHandler) (sdk.Context, error) {
if simulate {
return next(ctx, tx, simulate)
}
store := ctx.KVStore(a.storeKey)
currentHeight := ctx.BlockHeight()
var txCounter uint32 = 0 // start with 0
// load counter when exists
if bz := store.Get(types.TXCounterPrefix); bz != nil {
lastHeight, val := decodeHeightCounter(bz)
if currentHeight == lastHeight {
// then use stored counter
txCounter = val
} // else use `0` from above to start with
}
// store next counter value for current height
store.Set(types.TXCounterPrefix, encodeHeightCounter(currentHeight, txCounter+1))
return next(types.WithTXCounter(ctx, txCounter), tx, simulate)
}
func encodeHeightCounter(height int64, counter uint32) []byte {
b := make([]byte, 4)
binary.BigEndian.PutUint32(b, counter)
return append(sdk.Uint64ToBigEndian(uint64(height)), b...)
}
func decodeHeightCounter(bz []byte) (int64, uint32) {
return int64(sdk.BigEndianToUint64(bz[0:8])), binary.BigEndian.Uint32(bz[8:])
}

107
x/wasm/keeper/ante_test.go Normal file
View File

@@ -0,0 +1,107 @@
package keeper
import (
"github.com/CosmWasm/wasmd/x/wasm/types"
"github.com/cosmos/cosmos-sdk/store"
sdk "github.com/cosmos/cosmos-sdk/types"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/libs/log"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
dbm "github.com/tendermint/tm-db"
"testing"
"time"
)
func TestCountTxDecorator(t *testing.T) {
keyWasm := sdk.NewKVStoreKey(types.StoreKey)
db := dbm.NewMemDB()
ms := store.NewCommitMultiStore(db)
ms.MountStoreWithDB(keyWasm, sdk.StoreTypeIAVL, db)
require.NoError(t, ms.LoadLatestVersion())
const myCurrentBlockHeight = 100
specs := map[string]struct {
setupDB func(t *testing.T, ctx sdk.Context)
simulate bool
nextAssertAnte func(ctx sdk.Context, tx sdk.Tx, simulate bool) (sdk.Context, error)
expErr bool
}{
"no initial counter set": {
setupDB: func(t *testing.T, ctx sdk.Context) {},
nextAssertAnte: func(ctx sdk.Context, tx sdk.Tx, simulate bool) (sdk.Context, error) {
gotCounter, ok := types.TXCounter(ctx)
require.True(t, ok)
assert.Equal(t, uint32(0), gotCounter)
// and stored +1
bz := ctx.MultiStore().GetKVStore(keyWasm).Get(types.TXCounterPrefix)
assert.Equal(t, []byte{0, 0, 0, 0, 0, 0, 0, myCurrentBlockHeight, 0, 0, 0, 1}, bz)
return ctx, nil
},
},
"persistent counter incremented - big endian": {
setupDB: func(t *testing.T, ctx sdk.Context) {
bz := []byte{0, 0, 0, 0, 0, 0, 0, myCurrentBlockHeight, 1, 0, 0, 2}
ctx.MultiStore().GetKVStore(keyWasm).Set(types.TXCounterPrefix, bz)
},
nextAssertAnte: func(ctx sdk.Context, tx sdk.Tx, simulate bool) (sdk.Context, error) {
gotCounter, ok := types.TXCounter(ctx)
require.True(t, ok)
assert.Equal(t, uint32(1<<24+2), gotCounter)
// and stored +1
bz := ctx.MultiStore().GetKVStore(keyWasm).Get(types.TXCounterPrefix)
assert.Equal(t, []byte{0, 0, 0, 0, 0, 0, 0, myCurrentBlockHeight, 1, 0, 0, 3}, bz)
return ctx, nil
},
},
"old height counter replaced": {
setupDB: func(t *testing.T, ctx sdk.Context) {
previousHeight := byte(myCurrentBlockHeight - 1)
bz := []byte{0, 0, 0, 0, 0, 0, 0, previousHeight, 0, 0, 0, 1}
ctx.MultiStore().GetKVStore(keyWasm).Set(types.TXCounterPrefix, bz)
},
nextAssertAnte: func(ctx sdk.Context, tx sdk.Tx, simulate bool) (sdk.Context, error) {
gotCounter, ok := types.TXCounter(ctx)
require.True(t, ok)
assert.Equal(t, uint32(0), gotCounter)
// and stored +1
bz := ctx.MultiStore().GetKVStore(keyWasm).Get(types.TXCounterPrefix)
assert.Equal(t, []byte{0, 0, 0, 0, 0, 0, 0, myCurrentBlockHeight, 0, 0, 0, 1}, bz)
return ctx, nil
},
},
"simulation not persisted": {
setupDB: func(t *testing.T, ctx sdk.Context) {
},
simulate: true,
nextAssertAnte: func(ctx sdk.Context, tx sdk.Tx, simulate bool) (sdk.Context, error) {
_, ok := types.TXCounter(ctx)
assert.False(t, ok)
require.True(t, simulate)
// and not stored
assert.False(t, ctx.MultiStore().GetKVStore(keyWasm).Has(types.TXCounterPrefix))
return ctx, nil
},
},
}
for name, spec := range specs {
t.Run(name, func(t *testing.T) {
ctx := sdk.NewContext(ms.CacheMultiStore(), tmproto.Header{
Height: myCurrentBlockHeight,
Time: time.Date(2021, time.September, 27, 12, 0, 0, 0, time.UTC),
}, false, log.NewNopLogger())
spec.setupDB(t, ctx)
var anyTx sdk.Tx
// when
ante := NewCountTXDecorator(keyWasm)
_, gotErr := ante.AnteHandle(ctx, anyTx, spec.simulate, spec.nextAssertAnte)
if spec.expErr {
require.Error(t, gotErr)
return
}
require.NoError(t, gotErr)
})
}
}

View File

@@ -307,7 +307,7 @@ func TestInstantiate(t *testing.T) {
gasAfter := ctx.GasMeter().GasConsumed()
if types.EnableGasVerification {
require.Equal(t, uint64(0x12215), gasAfter-gasBefore)
require.Equal(t, uint64(0x1221d), gasAfter-gasBefore)
}
// ensure it is stored properly
@@ -541,7 +541,7 @@ func TestExecute(t *testing.T) {
// make sure gas is properly deducted from ctx
gasAfter := ctx.GasMeter().GasConsumed()
if types.EnableGasVerification {
require.Equal(t, uint64(0x12af8), gasAfter-gasBefore)
require.Equal(t, uint64(0x12b01), gasAfter-gasBefore)
}
// ensure bob now exists and got both payments released
bobAcct = accKeeper.GetAccount(ctx, bob)

View File

@@ -52,9 +52,9 @@ func initRecurseContract(t *testing.T) (contract sdk.AccAddress, creator sdk.Acc
func TestGasCostOnQuery(t *testing.T) {
const (
GasNoWork uint64 = 44_162
GasNoWork uint64 = 44_170
// Note: about 100 SDK gas (10k wasmer gas) for each round of sha256
GasWork50 uint64 = 48_846 // this is a little shy of 50k gas - to keep an eye on the limit
GasWork50 uint64 = 48_854 // this is a little shy of 50k gas - to keep an eye on the limit
GasReturnUnhashed uint64 = 253
GasReturnHashed uint64 = 228
@@ -75,7 +75,7 @@ func TestGasCostOnQuery(t *testing.T) {
msg: Recurse{
Work: 50, // 50 rounds of sha256 inside the contract
},
expectedGas: GasWork50,
expectedGas: GasWork50 + 1,
},
"recursion 1, no work": {
gasLimit: 400_000,
@@ -216,7 +216,7 @@ func TestLimitRecursiveQueryGas(t *testing.T) {
const (
// Note: about 100 SDK gas (10k wasmer gas) for each round of sha256
GasWork2k uint64 = 233_971 // = NewContractInstanceCosts + x // we have 6x gas used in cpu than in the instance
GasWork2k uint64 = 233_979 // = NewContractInstanceCosts + x // we have 6x gas used in cpu than in the instance
// This is overhead for calling into a sub-contract
GasReturnHashed uint64 = 231
)
@@ -235,7 +235,7 @@ func TestLimitRecursiveQueryGas(t *testing.T) {
Work: 2000,
},
expectQueriesFromContract: 0,
expectedGas: GasWork2k,
expectedGas: GasWork2k + 1,
},
"recursion 5, lots of work": {
gasLimit: 4_000_000,

View File

@@ -191,6 +191,8 @@ func createTestInput(
Height: 1234567,
Time: time.Date(2020, time.April, 22, 12, 0, 0, 0, time.UTC),
}, isCheckTx, log.NewNopLogger())
ctx = types.WithTXCounter(ctx, 0)
encodingConfig := MakeEncodingConfig(t)
appCodec, legacyAmino := encodingConfig.Marshaler, encodingConfig.Amino

24
x/wasm/types/ante.go Normal file
View File

@@ -0,0 +1,24 @@
package types
import (
sdk "github.com/cosmos/cosmos-sdk/types"
)
type contextKey int
const (
// private type creates an interface key for Context that cannot be accessed by any other package
contextKeyTXCount contextKey = iota
)
// WithTXCounter stores a transaction counter value in the context
func WithTXCounter(ctx sdk.Context, counter uint32) sdk.Context {
return ctx.WithValue(contextKeyTXCount, counter)
}
// TXCounter returns the tx counter value and found bool from the context.
// The result will be (0, false) for external queries or simulations where no counter available.
func TXCounter(ctx sdk.Context) (uint32, bool) {
val, ok := ctx.Value(contextKeyTXCount).(uint32)
return val, ok
}

View File

@@ -30,6 +30,7 @@ var (
ContractCodeHistoryElementPrefix = []byte{0x05}
ContractByCodeIDAndCreatedSecondaryIndexPrefix = []byte{0x06}
PinnedCodeIndexPrefix = []byte{0x07}
TXCounterPrefix = []byte{0x08}
KeyLastCodeID = append(SequenceKeyPrefix, []byte("lastCodeId")...)
KeyLastInstanceID = append(SequenceKeyPrefix, []byte("lastContractId")...)

View File

@@ -256,6 +256,7 @@ func NewEnv(ctx sdk.Context, contractAddr sdk.AccAddress) wasmvmtypes.Env {
if nano < 1 {
panic("Block (unix) time must never be empty or negative ")
}
env := wasmvmtypes.Env{
Block: wasmvmtypes.BlockInfo{
Height: uint64(ctx.BlockHeight()),
@@ -266,6 +267,9 @@ func NewEnv(ctx sdk.Context, contractAddr sdk.AccAddress) wasmvmtypes.Env {
Address: contractAddr.String(),
},
}
if txCounter, ok := TXCounter(ctx); ok {
env.Transaction = &wasmvmtypes.TransactionInfo{Index: txCounter}
}
return env
}

View File

@@ -1,6 +1,7 @@
package types
import (
"context"
wasmvmtypes "github.com/CosmWasm/wasmvm/types"
"github.com/cosmos/cosmos-sdk/codec"
"github.com/cosmos/cosmos-sdk/codec/types"
@@ -295,8 +296,22 @@ func TestNewEnv(t *testing.T) {
srcCtx sdk.Context
exp wasmvmtypes.Env
}{
"all good": {
srcCtx: sdk.Context{}.WithBlockHeight(1).WithBlockTime(myTime).WithChainID("testing"),
"all good with tx counter": {
srcCtx: WithTXCounter(sdk.Context{}.WithBlockHeight(1).WithBlockTime(myTime).WithChainID("testing").WithContext(context.Background()), 0),
exp: wasmvmtypes.Env{
Block: wasmvmtypes.BlockInfo{
Height: 1,
Time: 1619700924259075000,
ChainID: "testing",
},
Contract: wasmvmtypes.ContractInfo{
Address: myContractAddr.String(),
},
Transaction: &wasmvmtypes.TransactionInfo{Index: 0},
},
},
"without tx counter": {
srcCtx: sdk.Context{}.WithBlockHeight(1).WithBlockTime(myTime).WithChainID("testing").WithContext(context.Background()),
exp: wasmvmtypes.Env{
Block: wasmvmtypes.BlockInfo{
Height: 1,