Add in-place store migrations (#8485)

* Add 1st version of migrate

* Put migration logic into Configurator

* add test to bank store migration

* add test for configurator

* Error if no migration found

* Remove RunMigrations from Configurator interface

* Update spec

* Rename folders

* copy-paste from keys.go

* Fix nil map

* rename function

* Update simapp/app.go

Co-authored-by: Robert Zaremba <robert@zaremba.ch>

* Update simapp/app_test.go

Co-authored-by: Robert Zaremba <robert@zaremba.ch>

* Adderss reviews

* Fix tests

* Update testutil/context.go

Co-authored-by: Robert Zaremba <robert@zaremba.ch>

* Update docs for ConsensusVersion

* Rename to forVersion

* Fix tests

* Check error early

* Return 1 for intiial version

* Use MigrationKeeper

* Fix test

* Revert adding marshaler to Configurator

* Godoc updates

* Update docs

Co-authored-by: Robert Zaremba <robert@zaremba.ch>
Co-authored-by: Aaron Craelius <aaron@regen.network>
This commit is contained in:
Amaury 2021-02-10 18:49:31 +01:00 committed by GitHub
parent 9ca52f4202
commit dfc8dd813e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
34 changed files with 509 additions and 45 deletions

View File

@ -198,6 +198,9 @@ type SimApp struct {
// simulation manager
sm *module.SimulationManager
// the configurator
configurator module.Configurator
}
func init() {
@ -393,7 +396,8 @@ func NewSimApp(
app.mm.RegisterInvariants(&app.CrisisKeeper)
app.mm.RegisterRoutes(app.Router(), app.QueryRouter(), encodingConfig.Amino)
app.mm.RegisterServices(module.NewConfigurator(app.MsgServiceRouter(), app.GRPCQueryRouter()))
app.configurator = module.NewConfigurator(app.MsgServiceRouter(), app.GRPCQueryRouter())
app.mm.RegisterServices(app.configurator)
// add test gRPC service for testing gRPC queries in isolation
testdata.RegisterQueryServer(app.GRPCQueryRouter(), testdata.QueryImpl{})
@ -598,6 +602,28 @@ func (app *SimApp) RegisterTendermintService(clientCtx client.Context) {
tmservice.RegisterTendermintService(app.BaseApp.GRPCQueryRouter(), clientCtx, app.interfaceRegistry)
}
// RunMigrations performs in-place store migrations for all modules. This
// function MUST be only called by x/upgrade UpgradeHandler.
//
// `migrateFromVersions` is a map of moduleName to fromVersion (unit64), where
// fromVersion denotes the version from which we should migrate the module, the
// target version being the module's latest ConsensusVersion.
//
// Example:
// cfg := module.NewConfigurator(...)
// app.UpgradeKeeper.SetUpgradeHandler("store-migration", func(ctx sdk.Context, plan upgradetypes.Plan) {
// err := app.RunMigrations(ctx, module.MigrationMap{
// "bank": 1, // Migrate x/bank from v1 to current x/bank's ConsensusVersion
// "staking": 8, // Migrate x/staking from v8 to current x/staking's ConsensusVersion
// })
// if err != nil {
// panic(err)
// }
// })
func (app *SimApp) RunMigrations(ctx sdk.Context, migrateFromVersions module.MigrationMap) error {
return app.mm.RunMigrations(ctx, app.configurator, migrateFromVersions)
}
// RegisterSwaggerAPI registers swagger route with API Server
func RegisterSwaggerAPI(ctx client.Context, rtr *mux.Router) {
statikFS, err := fs.New()

View File

@ -6,10 +6,13 @@ import (
"testing"
"github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/libs/log"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
dbm "github.com/tendermint/tm-db"
abci "github.com/tendermint/tendermint/abci/types"
sdk "github.com/cosmos/cosmos-sdk/types"
"github.com/cosmos/cosmos-sdk/types/module"
)
func TestSimAppExportAndBlockedAddrs(t *testing.T) {
@ -45,3 +48,87 @@ func TestGetMaccPerms(t *testing.T) {
dup := GetMaccPerms()
require.Equal(t, maccPerms, dup, "duplicated module account permissions differed from actual module account permissions")
}
func TestRunMigrations(t *testing.T) {
db := dbm.NewMemDB()
encCfg := MakeTestEncodingConfig()
app := NewSimApp(log.NewTMLogger(log.NewSyncWriter(os.Stdout)), db, nil, true, map[int64]bool{}, DefaultNodeHome, 0, encCfg, EmptyAppOptions{})
// Create a new configurator for the purpose of this test.
app.configurator = module.NewConfigurator(app.MsgServiceRouter(), app.GRPCQueryRouter())
testCases := []struct {
name string
moduleName string
forVersion uint64
expRegErr bool // errors while registering migration
expRegErrMsg string
expRunErr bool // errors while running migration
expRunErrMsg string
expCalled int
}{
{
"cannot register migration for version 0",
"bank", 0,
true, "module migration versions should start at 1: invalid version", false, "", 0,
},
{
"throws error on RunMigrations if no migration registered for bank",
"", 1,
false, "", true, "no migrations found for module bank: not found", 0,
},
{
"can register and run migration handler for x/bank",
"bank", 1,
false, "", false, "", 1,
},
{
"cannot register migration handler for same module & forVersion",
"bank", 1,
true, "another migration for module bank and version 1 already exists: internal logic error", false, "", 0,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
var err error
// Since it's very hard to test actual in-place store migrations in
// tests (due to the difficulty of maintaing multiple versions of a
// module), we're just testing here that the migration logic is
// called.
called := 0
if tc.moduleName != "" {
// Register migration for module from version `forVersion` to `forVersion+1`.
err = app.configurator.RegisterMigration(tc.moduleName, tc.forVersion, func(sdk.Context) error {
called++
return nil
})
if tc.expRegErr {
require.EqualError(t, err, tc.expRegErrMsg)
return
}
}
require.NoError(t, err)
err = app.RunMigrations(
app.NewContext(true, tmproto.Header{Height: app.LastBlockHeight()}),
module.MigrationMap{
"auth": 1, "authz": 1, "bank": 1, "staking": 1, "mint": 1, "distribution": 1,
"slashing": 1, "gov": 1, "params": 1, "ibc": 1, "upgrade": 1, "vesting": 1,
"feegrant": 1, "transfer": 1, "evidence": 1, "crisis": 1, "genutil": 1, "capability": 1,
},
)
if tc.expRunErr {
require.EqualError(t, err, tc.expRunErrMsg)
} else {
require.NoError(t, err)
require.Equal(t, tc.expCalled, called)
}
})
}
}

View File

@ -492,6 +492,9 @@ func (m *MockAppModule) ExportGenesis(arg0 types0.Context, arg1 codec.JSONMarsha
return ret0
}
// ConsensusVersion mocks base method
func (m *MockAppModule) ConsensusVersion() uint64 { return 1 }
// ExportGenesis indicates an expected call of ExportGenesis
func (mr *MockAppModuleMockRecorder) ExportGenesis(arg0, arg1 interface{}) *gomock.Call {
mr.mock.ctrl.T.Helper()

25
testutil/context.go Normal file
View File

@ -0,0 +1,25 @@
package testutil
import (
"github.com/tendermint/tendermint/libs/log"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/store"
sdk "github.com/cosmos/cosmos-sdk/types"
)
// DefaultContext creates a sdk.Context with a fresh MemDB that can be used in tests.
func DefaultContext(key sdk.StoreKey, tkey sdk.StoreKey) sdk.Context {
db := dbm.NewMemDB()
cms := store.NewCommitMultiStore(db)
cms.MountStoreWithDB(key, sdk.StoreTypeIAVL, db)
cms.MountStoreWithDB(tkey, sdk.StoreTypeTransient, db)
err := cms.LoadLatestVersion()
if err != nil {
panic(err)
}
ctx := sdk.NewContext(cms, tmproto.Header{}, false, log.NewNopLogger())
return ctx
}

View File

@ -8,13 +8,11 @@ import (
"github.com/golang/mock/gomock"
"github.com/stretchr/testify/suite"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/libs/log"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/crypto/keys/secp256k1"
"github.com/cosmos/cosmos-sdk/store"
"github.com/cosmos/cosmos-sdk/tests/mocks"
"github.com/cosmos/cosmos-sdk/testutil"
"github.com/cosmos/cosmos-sdk/types"
)
@ -26,15 +24,6 @@ func TestContextTestSuite(t *testing.T) {
suite.Run(t, new(contextTestSuite))
}
func (s *contextTestSuite) defaultContext(key types.StoreKey) types.Context {
db := dbm.NewMemDB()
cms := store.NewCommitMultiStore(db)
cms.MountStoreWithDB(key, types.StoreTypeIAVL, db)
s.Require().NoError(cms.LoadLatestVersion())
ctx := types.NewContext(cms, tmproto.Header{}, false, log.NewNopLogger())
return ctx
}
func (s *contextTestSuite) TestCacheContext() {
key := types.NewKVStoreKey(s.T().Name() + "_TestCacheContext")
k1 := []byte("hello")
@ -42,7 +31,7 @@ func (s *contextTestSuite) TestCacheContext() {
k2 := []byte("key")
v2 := []byte("value")
ctx := s.defaultContext(key)
ctx := testutil.DefaultContext(key, types.NewTransientStoreKey("transient_"+s.T().Name()))
store := ctx.KVStore(key)
store.Set(k1, v1)
s.Require().Equal(v1, store.Get(k1))
@ -64,7 +53,7 @@ func (s *contextTestSuite) TestCacheContext() {
func (s *contextTestSuite) TestLogContext() {
key := types.NewKVStoreKey(s.T().Name())
ctx := s.defaultContext(key)
ctx := testutil.DefaultContext(key, types.NewTransientStoreKey("transient_"+s.T().Name()))
ctrl := gomock.NewController(s.T())
s.T().Cleanup(ctrl.Finish)

View File

@ -1,6 +1,11 @@
package module
import "github.com/gogo/protobuf/grpc"
import (
"github.com/gogo/protobuf/grpc"
sdk "github.com/cosmos/cosmos-sdk/types"
sdkerrors "github.com/cosmos/cosmos-sdk/types/errors"
)
// Configurator provides the hooks to allow modules to configure and register
// their services in the RegisterServices method. It is designed to eventually
@ -15,16 +20,34 @@ type Configurator interface {
// QueryServer returns a grpc.Server instance which allows registering services
// that will be exposed as gRPC services as well as ABCI query handlers.
QueryServer() grpc.Server
// RegisterMigration registers an in-place store migration for a module. The
// handler is a migration script to perform in-place migrations from version
// `forVersion` to version `forVersion+1`.
//
// EACH TIME a module's ConsensusVersion increments, a new migration MUST
// be registered using this function. If a migration handler is missing for
// a particular function, the upgrade logic (see RunMigrations function)
// will panic. If the ConsensusVersion bump does not introduce any store
// changes, then a no-op function must be registered here.
RegisterMigration(moduleName string, forVersion uint64, handler MigrationHandler) error
}
type configurator struct {
msgServer grpc.Server
queryServer grpc.Server
// migrations is a map of moduleName -> forVersion -> migration script handler
migrations map[string]map[uint64]MigrationHandler
}
// NewConfigurator returns a new Configurator instance
func NewConfigurator(msgServer grpc.Server, queryServer grpc.Server) Configurator {
return configurator{msgServer: msgServer, queryServer: queryServer}
return configurator{
msgServer: msgServer,
queryServer: queryServer,
migrations: map[string]map[uint64]MigrationHandler{},
}
}
var _ Configurator = configurator{}
@ -38,3 +61,51 @@ func (c configurator) MsgServer() grpc.Server {
func (c configurator) QueryServer() grpc.Server {
return c.queryServer
}
// RegisterMigration implements the Configurator.RegisterMigration method
func (c configurator) RegisterMigration(moduleName string, forVersion uint64, handler MigrationHandler) error {
if forVersion == 0 {
return sdkerrors.Wrap(sdkerrors.ErrInvalidVersion, "module migration versions should start at 1")
}
if c.migrations[moduleName] == nil {
c.migrations[moduleName] = map[uint64]MigrationHandler{}
}
if c.migrations[moduleName][forVersion] != nil {
return sdkerrors.Wrapf(sdkerrors.ErrLogic, "another migration for module %s and version %d already exists", moduleName, forVersion)
}
c.migrations[moduleName][forVersion] = handler
return nil
}
// runModuleMigrations runs all in-place store migrations for one given module from a
// version to another version.
func (c configurator) runModuleMigrations(ctx sdk.Context, moduleName string, fromVersion, toVersion uint64) error {
// No-op if toVersion is the initial version.
if toVersion <= 1 {
return nil
}
moduleMigrationsMap, found := c.migrations[moduleName]
if !found {
return sdkerrors.Wrapf(sdkerrors.ErrNotFound, "no migrations found for module %s", moduleName)
}
// Run in-place migrations for the module sequentially until toVersion.
for i := fromVersion; i < toVersion; i++ {
migrateFn, found := moduleMigrationsMap[i]
if !found {
return sdkerrors.Wrapf(sdkerrors.ErrNotFound, "no migration found for module %s from version %d to version %d", moduleName, i, i+1)
}
err := migrateFn(ctx)
if err != nil {
return err
}
}
return nil
}

View File

@ -40,6 +40,7 @@ import (
"github.com/cosmos/cosmos-sdk/codec"
codectypes "github.com/cosmos/cosmos-sdk/codec/types"
sdk "github.com/cosmos/cosmos-sdk/types"
sdkerrors "github.com/cosmos/cosmos-sdk/types/errors"
)
//__________________________________________________________________________________________
@ -174,6 +175,12 @@ type AppModule interface {
// RegisterServices allows a module to register services
RegisterServices(Configurator)
// ConsensusVersion is a sequence number for state-breaking change of the
// module. It should be incremented on each consensus-breaking change
// introduced by the module. To avoid wrong/empty versions, the initial version
// should be set to 1.
ConsensusVersion() uint64
// ABCI
BeginBlock(sdk.Context, abci.RequestBeginBlock)
EndBlock(sdk.Context, abci.RequestEndBlock) []abci.ValidatorUpdate
@ -208,6 +215,9 @@ func (gam GenesisOnlyAppModule) LegacyQuerierHandler(*codec.LegacyAmino) sdk.Que
// RegisterServices registers all services.
func (gam GenesisOnlyAppModule) RegisterServices(Configurator) {}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (gam GenesisOnlyAppModule) ConsensusVersion() uint64 { return 1 }
// BeginBlock returns an empty module begin-block
func (gam GenesisOnlyAppModule) BeginBlock(ctx sdk.Context, req abci.RequestBeginBlock) {}
@ -328,6 +338,30 @@ func (m *Manager) ExportGenesis(ctx sdk.Context, cdc codec.JSONMarshaler) map[st
return genesisData
}
// MigrationHandler is the migration function that each module registers.
type MigrationHandler func(store sdk.Context) error
// MigrationMap is a map of moduleName -> version, where version denotes the
// version from which we should perform the migration for each module.
type MigrationMap map[string]uint64
// RunMigrations performs in-place store migrations for all modules.
func (m Manager) RunMigrations(ctx sdk.Context, cfg Configurator, migrateFromVersions MigrationMap) error {
c, ok := cfg.(configurator)
if !ok {
return sdkerrors.Wrapf(sdkerrors.ErrInvalidType, "expected %T, got %T", configurator{}, cfg)
}
for moduleName, module := range m.Modules {
err := c.runModuleMigrations(ctx, moduleName, migrateFromVersions[moduleName], module.ConsensusVersion())
if err != nil {
return err
}
}
return nil
}
// BeginBlock performs begin block functionality for all modules. It creates a
// child context with an event manager to aggregate events emitted from all
// modules.

View File

@ -0,0 +1,4 @@
package v040
// AddrLen defines a valid address length
const AddrLen = 20

View File

@ -147,6 +147,9 @@ func (am AppModule) ExportGenesis(ctx sdk.Context, cdc codec.JSONMarshaler) json
return cdc.MustMarshalJSON(gs)
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }
// BeginBlock returns the begin blocker for the auth module.
func (AppModule) BeginBlock(_ sdk.Context, _ abci.RequestBeginBlock) {}

View File

@ -127,3 +127,6 @@ func (am AppModule) EndBlock(_ sdk.Context, _ abci.RequestEndBlock) []abci.Valid
func (am AppModule) ExportGenesis(_ sdk.Context, cdc codec.JSONMarshaler) json.RawMessage {
return am.DefaultGenesis(cdc)
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }

View File

@ -152,6 +152,9 @@ func (am AppModule) ExportGenesis(ctx sdk.Context, cdc codec.JSONMarshaler) json
return cdc.MustMarshalJSON(gs)
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }
func (am AppModule) BeginBlock(ctx sdk.Context, req abci.RequestBeginBlock) {}
// EndBlock does nothing

View File

@ -0,0 +1,20 @@
package keeper
import (
sdk "github.com/cosmos/cosmos-sdk/types"
v042 "github.com/cosmos/cosmos-sdk/x/bank/legacy/v042"
)
// MigrationKeeper is an interface that the keeper implements for handling
// in-place store migrations.
type MigrationKeeper interface {
// Migrate1 migrates the store from version 1 to 2.
Migrate1(ctx sdk.Context) error
}
var _ MigrationKeeper = (*BaseKeeper)(nil)
// Migrate1 implements MigrationKeeper.Migrate1 method.
func (keeper BaseKeeper) Migrate1(ctx sdk.Context) error {
return v042.MigrateStore(ctx, keeper.storeKey)
}

View File

@ -0,0 +1,47 @@
package v040
import (
"fmt"
sdk "github.com/cosmos/cosmos-sdk/types"
v040auth "github.com/cosmos/cosmos-sdk/x/auth/legacy/v040"
)
const (
// ModuleName defines the module name
ModuleName = "bank"
// StoreKey defines the primary module store key
StoreKey = ModuleName
// RouterKey defines the module's message routing key
RouterKey = ModuleName
// QuerierRoute defines the module's query routing key
QuerierRoute = ModuleName
)
// KVStore keys
var (
BalancesPrefix = []byte("balances")
SupplyKey = []byte{0x00}
DenomMetadataPrefix = []byte{0x1}
)
// DenomMetadataKey returns the denomination metadata key.
func DenomMetadataKey(denom string) []byte {
d := []byte(denom)
return append(DenomMetadataPrefix, d...)
}
// AddressFromBalancesStore returns an account address from a balances prefix
// store. The key must not contain the perfix BalancesPrefix as the prefix store
// iterator discards the actual prefix.
func AddressFromBalancesStore(key []byte) sdk.AccAddress {
addr := key[:v040auth.AddrLen]
if len(addr) != v040auth.AddrLen {
panic(fmt.Sprintf("unexpected account address key length; got: %d, expected: %d", len(addr), v040auth.AddrLen))
}
return sdk.AccAddress(addr)
}

View File

@ -1,5 +0,0 @@
package v040
const (
ModuleName = "bank"
)

View File

@ -0,0 +1,50 @@
package v042
import (
sdk "github.com/cosmos/cosmos-sdk/types"
"github.com/cosmos/cosmos-sdk/types/address"
)
const (
// ModuleName defines the module name
ModuleName = "bank"
// StoreKey defines the primary module store key
StoreKey = ModuleName
// RouterKey defines the module's message routing key
RouterKey = ModuleName
// QuerierRoute defines the module's query routing key
QuerierRoute = ModuleName
)
// KVStore keys
var (
// BalancesPrefix is the for the account balances store. We use a byte
// (instead of say `[]]byte("balances")` to save some disk space).
BalancesPrefix = []byte{0x02}
SupplyKey = []byte{0x00}
DenomMetadataPrefix = []byte{0x1}
)
// DenomMetadataKey returns the denomination metadata key.
func DenomMetadataKey(denom string) []byte {
d := []byte(denom)
return append(DenomMetadataPrefix, d...)
}
// AddressFromBalancesStore returns an account address from a balances prefix
// store. The key must not contain the perfix BalancesPrefix as the prefix store
// iterator discards the actual prefix.
func AddressFromBalancesStore(key []byte) sdk.AccAddress {
addrLen := key[0]
addr := key[1 : addrLen+1]
return sdk.AccAddress(addr)
}
// CreateAccountBalancesPrefix creates the prefix for an account's balances.
func CreateAccountBalancesPrefix(addr []byte) []byte {
return append(BalancesPrefix, address.MustLengthPrefix(addr)...)
}

View File

@ -0,0 +1,38 @@
package v042
import (
"github.com/cosmos/cosmos-sdk/store/prefix"
sdk "github.com/cosmos/cosmos-sdk/types"
v040auth "github.com/cosmos/cosmos-sdk/x/auth/legacy/v040"
v040bank "github.com/cosmos/cosmos-sdk/x/bank/legacy/v040"
)
// MigrateStore performs in-place store migrations from v0.40 to v0.42. The
// migration includes:
//
// - Change addresses to be length-prefixed.
// - Change balances prefix to 1 byte
func MigrateStore(ctx sdk.Context, storeKey sdk.StoreKey) error {
store := ctx.KVStore(storeKey)
// old key is of format:
// prefix ("balances") || addrBytes (20 bytes) || denomBytes
// new key is of format
// prefix (0x02) || addrLen (1 byte) || addrBytes || denomBytes
oldStore := prefix.NewStore(store, v040bank.BalancesPrefix)
oldStoreIter := oldStore.Iterator(nil, nil)
defer oldStoreIter.Close()
for ; oldStoreIter.Valid(); oldStoreIter.Next() {
addr := v040bank.AddressFromBalancesStore(oldStoreIter.Key())
denom := oldStoreIter.Key()[v040auth.AddrLen:]
newStoreKey := append(CreateAccountBalancesPrefix(addr), denom...)
// Set new key on store. Values don't change.
store.Set(newStoreKey, oldStoreIter.Value())
oldStore.Delete(oldStoreIter.Key())
}
return nil
}

View File

@ -0,0 +1,36 @@
package v042_test
import (
"testing"
"github.com/stretchr/testify/require"
"github.com/cosmos/cosmos-sdk/testutil"
"github.com/cosmos/cosmos-sdk/testutil/testdata"
sdk "github.com/cosmos/cosmos-sdk/types"
v040bank "github.com/cosmos/cosmos-sdk/x/bank/legacy/v040"
v042bank "github.com/cosmos/cosmos-sdk/x/bank/legacy/v042"
)
func TestStoreMigration(t *testing.T) {
bankKey := sdk.NewKVStoreKey("bank")
ctx := testutil.DefaultContext(bankKey, sdk.NewTransientStoreKey("transient_test"))
store := ctx.KVStore(bankKey)
_, _, addr := testdata.KeyTestPubAddr()
denom := []byte("foo")
value := []byte("bar")
oldKey := append(append(v040bank.BalancesPrefix, addr...), denom...)
store.Set(oldKey, value)
err := v042bank.MigrateStore(ctx, bankKey)
require.NoError(t, err)
newKey := append(v042bank.CreateAccountBalancesPrefix(addr), denom...)
// -7 because we replaced "balances" with 0x02,
// +1 because we added length-prefix to address.
require.Equal(t, len(oldKey)-7+1, len(newKey))
require.Nil(t, store.Get(oldKey))
require.Equal(t, value, store.Get(newKey))
}

View File

@ -100,6 +100,9 @@ type AppModule struct {
func (am AppModule) RegisterServices(cfg module.Configurator) {
types.RegisterMsgServer(cfg.MsgServer(), keeper.NewMsgServerImpl(am.keeper))
types.RegisterQueryServer(cfg.QueryServer(), am.keeper)
cfg.RegisterMigration(types.ModuleName, 0, func(ctx sdk.Context) error {
return am.keeper.(keeper.MigrationKeeper).Migrate1(ctx)
})
}
// NewAppModule creates a new AppModule object
@ -151,6 +154,9 @@ func (am AppModule) ExportGenesis(ctx sdk.Context, cdc codec.JSONMarshaler) json
return cdc.MustMarshalJSON(gs)
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 2 }
// BeginBlock performs a no-op.
func (AppModule) BeginBlock(_ sdk.Context, _ abci.RequestBeginBlock) {}

View File

@ -7,5 +7,5 @@ order: 1
The `x/bank` module keeps state of two primary objects, account balances and the
total supply of all balances.
- Balances: `[]byte("balances") | []byte(address) / []byte(balance.Denom) -> ProtocolBuffer(balance)`
- Supply: `0x0 -> ProtocolBuffer(Supply)`
- Balances: `0x2 | byte(address length) | []byte(address) | []byte(balance.Denom) -> ProtocolBuffer(balance)`

View File

@ -136,6 +136,9 @@ func (am AppModule) ExportGenesis(ctx sdk.Context, cdc codec.JSONMarshaler) json
return cdc.MustMarshalJSON(genState)
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }
// BeginBlock executes all ABCI BeginBlock logic respective to the capability module.
func (am AppModule) BeginBlock(_ sdk.Context, _ abci.RequestBeginBlock) {}

View File

@ -158,6 +158,9 @@ func (am AppModule) ExportGenesis(ctx sdk.Context, cdc codec.JSONMarshaler) json
return cdc.MustMarshalJSON(gs)
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }
// BeginBlock performs a no-op.
func (AppModule) BeginBlock(_ sdk.Context, _ abci.RequestBeginBlock) {}

View File

@ -161,6 +161,9 @@ func (am AppModule) ExportGenesis(ctx sdk.Context, cdc codec.JSONMarshaler) json
return cdc.MustMarshalJSON(gs)
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }
// BeginBlock returns the begin blocker for the distribution module.
func (am AppModule) BeginBlock(ctx sdk.Context, req abci.RequestBeginBlock) {
BeginBlocker(ctx, req, am.keeper)

View File

@ -175,6 +175,9 @@ func (am AppModule) ExportGenesis(ctx sdk.Context, cdc codec.JSONMarshaler) json
return cdc.MustMarshalJSON(ExportGenesis(ctx, am.keeper))
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }
// BeginBlock executes all ABCI BeginBlock logic respective to the evidence module.
func (am AppModule) BeginBlock(ctx sdk.Context, req abci.RequestBeginBlock) {
BeginBlocker(ctx, req, am.keeper)

View File

@ -167,6 +167,9 @@ func (am AppModule) ExportGenesis(ctx sdk.Context, cdc codec.JSONMarshaler) json
return cdc.MustMarshalJSON(gs)
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }
// BeginBlock returns the begin blocker for the feegrant module.
func (am AppModule) BeginBlock(_ sdk.Context, _ abci.RequestBeginBlock) {}

View File

@ -110,3 +110,6 @@ func (am AppModule) InitGenesis(ctx sdk.Context, cdc codec.JSONMarshaler, data j
func (am AppModule) ExportGenesis(_ sdk.Context, cdc codec.JSONMarshaler) json.RawMessage {
return am.DefaultGenesis(cdc)
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }

View File

@ -177,6 +177,9 @@ func (am AppModule) ExportGenesis(ctx sdk.Context, cdc codec.JSONMarshaler) json
return cdc.MustMarshalJSON(gs)
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }
// BeginBlock performs a no-op.
func (AppModule) BeginBlock(_ sdk.Context, _ abci.RequestBeginBlock) {}

View File

@ -145,6 +145,9 @@ func (am AppModule) ExportGenesis(ctx sdk.Context, cdc codec.JSONMarshaler) json
return cdc.MustMarshalJSON(gs)
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }
// BeginBlock implements the AppModule interface
func (am AppModule) BeginBlock(ctx sdk.Context, req abci.RequestBeginBlock) {
}

View File

@ -156,6 +156,9 @@ func (am AppModule) ExportGenesis(ctx sdk.Context, cdc codec.JSONMarshaler) json
return cdc.MustMarshalJSON(ExportGenesis(ctx, *am.keeper))
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }
// BeginBlock returns the begin blocker for the ibc module.
func (am AppModule) BeginBlock(ctx sdk.Context, req abci.RequestBeginBlock) {
ibcclient.BeginBlocker(ctx, am.keeper.ClientKeeper)

View File

@ -146,6 +146,9 @@ func (am AppModule) ExportGenesis(ctx sdk.Context, cdc codec.JSONMarshaler) json
return cdc.MustMarshalJSON(gs)
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }
// BeginBlock returns the begin blocker for the mint module.
func (am AppModule) BeginBlock(ctx sdk.Context, _ abci.RequestBeginBlock) {
BeginBlocker(ctx, am.keeper)

View File

@ -1,14 +1,9 @@
package keeper_test
import (
"github.com/tendermint/tendermint/libs/log"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/simapp"
"github.com/cosmos/cosmos-sdk/codec"
"github.com/cosmos/cosmos-sdk/store"
"github.com/cosmos/cosmos-sdk/simapp"
"github.com/cosmos/cosmos-sdk/testutil"
sdk "github.com/cosmos/cosmos-sdk/types"
paramskeeper "github.com/cosmos/cosmos-sdk/x/params/keeper"
)
@ -18,7 +13,7 @@ func testComponents() (*codec.LegacyAmino, sdk.Context, sdk.StoreKey, sdk.StoreK
legacyAmino := createTestCodec()
mkey := sdk.NewKVStoreKey("test")
tkey := sdk.NewTransientStoreKey("transient_test")
ctx := defaultContext(mkey, tkey)
ctx := testutil.DefaultContext(mkey, tkey)
keeper := paramskeeper.NewKeeper(marshaler, legacyAmino, mkey, tkey)
return legacyAmino, ctx, mkey, tkey, keeper
@ -37,16 +32,3 @@ func createTestCodec() *codec.LegacyAmino {
cdc.RegisterConcrete(invalid{}, "test/invalid", nil)
return cdc
}
func defaultContext(key sdk.StoreKey, tkey sdk.StoreKey) sdk.Context {
db := dbm.NewMemDB()
cms := store.NewCommitMultiStore(db)
cms.MountStoreWithDB(key, sdk.StoreTypeIAVL, db)
cms.MountStoreWithDB(tkey, sdk.StoreTypeTransient, db)
err := cms.LoadLatestVersion()
if err != nil {
panic(err)
}
ctx := sdk.NewContext(cms, tmproto.Header{}, false, log.NewNopLogger())
return ctx
}

View File

@ -139,6 +139,9 @@ func (am AppModule) ExportGenesis(_ sdk.Context, _ codec.JSONMarshaler) json.Raw
return nil
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }
// BeginBlock performs a no-op.
func (AppModule) BeginBlock(_ sdk.Context, _ abci.RequestBeginBlock) {}

View File

@ -159,6 +159,9 @@ func (am AppModule) ExportGenesis(ctx sdk.Context, cdc codec.JSONMarshaler) json
return cdc.MustMarshalJSON(gs)
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }
// BeginBlock returns the begin blocker for the slashing module.
func (am AppModule) BeginBlock(ctx sdk.Context, req abci.RequestBeginBlock) {
BeginBlocker(ctx, req, am.keeper)

View File

@ -157,6 +157,9 @@ func (am AppModule) ExportGenesis(ctx sdk.Context, cdc codec.JSONMarshaler) json
return cdc.MustMarshalJSON(gs)
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }
// BeginBlock returns the begin blocker for the staking module.
func (am AppModule) BeginBlock(ctx sdk.Context, _ abci.RequestBeginBlock) {
BeginBlocker(ctx, am.keeper)

View File

@ -120,6 +120,9 @@ func (am AppModule) ExportGenesis(_ sdk.Context, cdc codec.JSONMarshaler) json.R
return am.DefaultGenesis(cdc)
}
// ConsensusVersion implements AppModule/ConsensusVersion.
func (AppModule) ConsensusVersion() uint64 { return 1 }
// BeginBlock calls the upgrade module hooks
//
// CONTRACT: this is registered in BeginBlocker *before* all other modules' BeginBlock functions