Revert Capabilities on Failed Tx (#5999)

Reintroduce memKVStore to keep track of fwd and reverse mappings.

On reverse mapping, rather than store a mapping to marshalled
capability; we store the index.

capability.Keeper and all scopedKeeper have access to a capability
map that maps index to the capability pointer.

This is done to make sure that all writes to memKVStore get reverted
on a fail tx, while also allowing GetCapability to retrieve the original
memory pointer from the go map.

Go map must be accessed only by first going through the
memKVStore. SInce writes to go map cannot be automatically
reverted on tx failure, it gets cleaned up on failed GetCapability calls.

Closes: #5965
This commit is contained in:
Aditya 2020-04-16 22:12:13 +05:30 committed by GitHub
parent 4da4bb64df
commit d247184157
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 274 additions and 89 deletions

View File

@ -187,6 +187,14 @@ func (app *BaseApp) MountTransientStores(keys map[string]*sdk.TransientStoreKey)
}
}
// MountMemoryStores mounts all in-memory KVStores with the BaseApp's internal
// commit multi-store.
func (app *BaseApp) MountMemoryStores(keys map[string]*sdk.MemoryStoreKey) {
for _, memKey := range keys {
app.MountStore(memKey, sdk.StoreTypeMemory)
}
}
// MountStoreWithDB mounts a store to the provided key in the BaseApp
// multistore, using a specified DB.
func (app *BaseApp) MountStoreWithDB(key sdk.StoreKey, typ sdk.StoreType, db dbm.DB) {

View File

@ -102,8 +102,9 @@ type SimApp struct {
invCheckPeriod uint
// keys to access the substores
keys map[string]*sdk.KVStoreKey
tkeys map[string]*sdk.TransientStoreKey
keys map[string]*sdk.KVStoreKey
tkeys map[string]*sdk.TransientStoreKey
memKeys map[string]*sdk.MemoryStoreKey
// subspaces
subspaces map[string]params.Subspace
@ -157,6 +158,7 @@ func NewSimApp(
evidence.StoreKey, transfer.StoreKey, capability.StoreKey,
)
tkeys := sdk.NewTransientStoreKeys(params.TStoreKey)
memKeys := sdk.NewMemoryStoreKeys(capability.MemStoreKey)
app := &SimApp{
BaseApp: bApp,
@ -164,6 +166,7 @@ func NewSimApp(
invCheckPeriod: invCheckPeriod,
keys: keys,
tkeys: tkeys,
memKeys: memKeys,
subspaces: make(map[string]params.Subspace),
}
@ -182,7 +185,7 @@ func NewSimApp(
bApp.SetParamStore(app.ParamsKeeper.Subspace(baseapp.Paramspace).WithKeyTable(std.ConsensusParamsKeyTable()))
// add capability keeper and ScopeToModule for ibc module
app.CapabilityKeeper = capability.NewKeeper(appCodec, keys[capability.StoreKey])
app.CapabilityKeeper = capability.NewKeeper(appCodec, keys[capability.StoreKey], memKeys[capability.MemStoreKey])
scopedIBCKeeper := app.CapabilityKeeper.ScopeToModule(ibc.ModuleName)
scopedTransferKeeper := app.CapabilityKeeper.ScopeToModule(transfer.ModuleName)
@ -321,6 +324,7 @@ func NewSimApp(
// initialize stores
app.MountKVStores(keys)
app.MountTransientStores(tkeys)
app.MountMemoryStores(memKeys)
// initialize BaseApp
app.SetInitChainer(app.InitChainer)
@ -418,6 +422,13 @@ func (app *SimApp) GetTKey(storeKey string) *sdk.TransientStoreKey {
return app.tkeys[storeKey]
}
// GetMemKey returns the MemStoreKey for the provided mem key.
//
// NOTE: This is solely used for testing purposes.
func (app *SimApp) GetMemKey(storeKey string) *sdk.MemoryStoreKey {
return app.memKeys[storeKey]
}
// GetSubspace returns a param subspace for a given module name.
//
// NOTE: This is solely to be used for testing purposes.

39
store/mem/mem_test.go Normal file
View File

@ -0,0 +1,39 @@
package mem_test
import (
"testing"
"github.com/stretchr/testify/require"
"github.com/cosmos/cosmos-sdk/store/mem"
"github.com/cosmos/cosmos-sdk/store/types"
)
func TestStore(t *testing.T) {
db := mem.NewStore()
key, value := []byte("key"), []byte("value")
require.Equal(t, types.StoreTypeMemory, db.GetStoreType())
require.Nil(t, db.Get(key))
db.Set(key, value)
require.Equal(t, value, db.Get(key))
newValue := []byte("newValue")
db.Set(key, newValue)
require.Equal(t, newValue, db.Get(key))
db.Delete(key)
require.Nil(t, db.Get(key))
}
func TestCommit(t *testing.T) {
db := mem.NewStore()
key, value := []byte("key"), []byte("value")
db.Set(key, value)
id := db.Commit()
require.True(t, id.IsZero())
require.True(t, db.LastCommitID().IsZero())
require.Equal(t, value, db.Get(key))
}

53
store/mem/store.go Normal file
View File

@ -0,0 +1,53 @@
package mem
import (
"io"
dbm "github.com/tendermint/tm-db"
"github.com/cosmos/cosmos-sdk/store/cachekv"
"github.com/cosmos/cosmos-sdk/store/dbadapter"
"github.com/cosmos/cosmos-sdk/store/tracekv"
"github.com/cosmos/cosmos-sdk/store/types"
)
var (
_ types.KVStore = (*Store)(nil)
_ types.Committer = (*Store)(nil)
)
// Store implements an in-memory only KVStore. Entries are persisted between
// commits and thus between blocks. State in Memory store is not committed as part of app state but maintained privately by each node
type Store struct {
dbadapter.Store
}
func NewStore() *Store {
return NewStoreWithDB(dbm.NewMemDB())
}
func NewStoreWithDB(db *dbm.MemDB) *Store { // nolint: interfacer
return &Store{Store: dbadapter.Store{DB: db}}
}
// GetStoreType returns the Store's type.
func (s Store) GetStoreType() types.StoreType {
return types.StoreTypeMemory
}
// CacheWrap cache wraps the underlying store.
func (s Store) CacheWrap() types.CacheWrap {
return cachekv.NewStore(s)
}
// CacheWrapWithTrace implements KVStore.
func (s Store) CacheWrapWithTrace(w io.Writer, tc types.TraceContext) types.CacheWrap {
return cachekv.NewStore(tracekv.NewStore(s, w, tc))
}
// Commit performs a no-op as entries are persistent between commitments.
func (s *Store) Commit() (id types.CommitID) { return }
// nolint
func (s *Store) SetPruning(pruning types.PruningOptions) {}
func (s Store) LastCommitID() (id types.CommitID) { return }

View File

@ -16,6 +16,7 @@ import (
"github.com/cosmos/cosmos-sdk/store/cachemulti"
"github.com/cosmos/cosmos-sdk/store/dbadapter"
"github.com/cosmos/cosmos-sdk/store/iavl"
"github.com/cosmos/cosmos-sdk/store/mem"
"github.com/cosmos/cosmos-sdk/store/tracekv"
"github.com/cosmos/cosmos-sdk/store/transient"
"github.com/cosmos/cosmos-sdk/store/types"
@ -528,6 +529,13 @@ func (rs *Store) loadCommitStoreFromParams(key types.StoreKey, id types.CommitID
return transient.NewStore(), nil
case types.StoreTypeMemory:
if _, ok := key.(*types.MemoryStoreKey); !ok {
return nil, fmt.Errorf("unexpected key type for a MemoryStoreKey; got: %s", key.String())
}
return mem.NewStore(), nil
default:
panic(fmt.Sprintf("unrecognized store type %v", params.typ))
}

View File

@ -273,6 +273,7 @@ const (
StoreTypeDB
StoreTypeIAVL
StoreTypeTransient
StoreTypeMemory
)
func (st StoreType) String() string {
@ -288,6 +289,9 @@ func (st StoreType) String() string {
case StoreTypeTransient:
return "StoreTypeTransient"
case StoreTypeMemory:
return "StoreTypeMemory"
}
return "unknown store type"
@ -351,6 +355,25 @@ func (key *TransientStoreKey) String() string {
return fmt.Sprintf("TransientStoreKey{%p, %s}", key, key.name)
}
// MemoryStoreKey defines a typed key to be used with an in-memory KVStore.
type MemoryStoreKey struct {
name string
}
func NewMemoryStoreKey(name string) *MemoryStoreKey {
return &MemoryStoreKey{name: name}
}
// Name returns the name of the MemoryStoreKey.
func (key *MemoryStoreKey) Name() string {
return key.name
}
// String returns a stringified representation of the MemoryStoreKey.
func (key *MemoryStoreKey) String() string {
return fmt.Sprintf("MemoryStoreKey{%p, %s}", key, key.name)
}
//----------------------------------------
// key-value result for iterator queries

View File

@ -76,6 +76,7 @@ const (
StoreTypeDB = types.StoreTypeDB
StoreTypeIAVL = types.StoreTypeIAVL
StoreTypeTransient = types.StoreTypeTransient
StoreTypeMemory = types.StoreTypeMemory
)
// nolint - reexport
@ -84,6 +85,7 @@ type (
CapabilityKey = types.CapabilityKey
KVStoreKey = types.KVStoreKey
TransientStoreKey = types.TransientStoreKey
MemoryStoreKey = types.MemoryStoreKey
)
// NewKVStoreKey returns a new pointer to a KVStoreKey.
@ -120,6 +122,17 @@ func NewTransientStoreKeys(names ...string) map[string]*TransientStoreKey {
return keys
}
// NewMemoryStoreKeys constructs a new map matching store key names to their
// respective MemoryStoreKey references.
func NewMemoryStoreKeys(names ...string) map[string]*MemoryStoreKey {
keys := make(map[string]*MemoryStoreKey)
for _, name := range names {
keys[name] = types.NewMemoryStoreKey(name)
}
return keys
}
// PrefixEndBytes returns the []byte that would end a
// range query for all []byte with a certain prefix
// Deals with last byte of prefix being FF without overflowing

View File

@ -29,7 +29,6 @@ var (
ModuleCdc = types.ModuleCdc
NewOwner = types.NewOwner
NewCapabilityOwners = types.NewCapabilityOwners
NewCapabilityStore = types.NewCapabilityStore
)
// nolint
@ -38,5 +37,4 @@ type (
ScopedKeeper = keeper.ScopedKeeper
Capability = types.Capability
CapabilityOwners = types.CapabilityOwners
CapabilityStore = types.CapabilityStore
)

View File

@ -28,7 +28,8 @@ type (
Keeper struct {
cdc codec.Marshaler
storeKey sdk.StoreKey
capStore types.CapabilityStore
memKey sdk.StoreKey
capMap map[uint64]*types.Capability
scopedModules map[string]struct{}
sealed bool
}
@ -42,16 +43,18 @@ type (
ScopedKeeper struct {
cdc codec.Marshaler
storeKey sdk.StoreKey
capStore types.CapabilityStore // shared amongst all scoped keepers
memKey sdk.StoreKey
capMap map[uint64]*types.Capability
module string
}
)
func NewKeeper(cdc codec.Marshaler, storeKey sdk.StoreKey) *Keeper {
func NewKeeper(cdc codec.Marshaler, storeKey, memKey sdk.StoreKey) *Keeper {
return &Keeper{
cdc: cdc,
storeKey: storeKey,
capStore: types.NewCapabilityStore(),
memKey: memKey,
capMap: make(map[uint64]*types.Capability),
scopedModules: make(map[string]struct{}),
sealed: false,
}
@ -74,7 +77,8 @@ func (k *Keeper) ScopeToModule(moduleName string) ScopedKeeper {
return ScopedKeeper{
cdc: k.cdc,
storeKey: k.storeKey,
capStore: k.capStore,
memKey: k.memKey,
capMap: k.capMap,
module: moduleName,
}
}
@ -88,6 +92,13 @@ func (k *Keeper) InitializeAndSeal(ctx sdk.Context) {
panic("cannot initialize and seal an already sealed capability keeper")
}
memStore := ctx.KVStore(k.memKey)
memStoreType := memStore.GetStoreType()
if memStoreType != sdk.StoreTypeMemory {
panic(fmt.Sprintf("invalid memory store type; got %s, expected: %s", memStoreType, sdk.StoreTypeMemory))
}
prefixStore := prefix.NewStore(ctx.KVStore(k.storeKey), types.KeyPrefixIndexCapability)
iterator := sdk.KVStorePrefixIterator(prefixStore, nil)
@ -102,12 +113,17 @@ func (k *Keeper) InitializeAndSeal(ctx sdk.Context) {
for _, owner := range capOwners.Owners {
// Set the forward mapping between the module and capability tuple and the
// capability name in the in-memory store.
k.capStore.SetCapabilityName(owner.Module, owner.Name, cap)
// capability name in the memKVStore
memStore.Set(types.FwdCapabilityKey(owner.Module, cap), []byte(owner.Name))
// Set the reverse mapping between the module and capability name and the
// capability in the in-memory store.
k.capStore.SetCapability(owner.Module, owner.Name, cap)
// index in the in-memory store. Since marshalling and unmarshalling into a store
// will change memory address of capability, we simply store index as value here
// and retrieve the in-memory pointer to the capability from our map
memStore.Set(types.RevCapabilityKey(owner.Module, owner.Name), sdk.Uint64ToBigEndian(index))
// Set the mapping from index from index to in-memory capability in the go map
k.capMap[index] = cap
}
}
@ -132,7 +148,7 @@ func (k Keeper) GetLatestIndex(ctx sdk.Context) uint64 {
func (sk ScopedKeeper) NewCapability(ctx sdk.Context, name string) (*types.Capability, error) {
store := ctx.KVStore(sk.storeKey)
if cap := sk.capStore.GetCapability(sk.module, name); cap != nil {
if _, ok := sk.GetCapability(ctx, name); ok {
return nil, sdkerrors.Wrapf(types.ErrCapabilityTaken, fmt.Sprintf("module: %s, name: %s", sk.module, name))
}
@ -149,13 +165,20 @@ func (sk ScopedKeeper) NewCapability(ctx sdk.Context, name string) (*types.Capab
// increment global index
store.Set(types.KeyIndex, types.IndexToKey(index+1))
memStore := ctx.KVStore(sk.memKey)
// Set the forward mapping between the module and capability tuple and the
// capability name in the in-memory store.
sk.capStore.SetCapabilityName(sk.module, name, cap)
// capability name in the memKVStore
memStore.Set(types.FwdCapabilityKey(sk.module, cap), []byte(name))
// Set the reverse mapping between the module and capability name and the
// capability in the in-memory store.
sk.capStore.SetCapability(sk.module, name, cap)
// index in the in-memory store. Since marshalling and unmarshalling into a store
// will change memory address of capability, we simply store index as value here
// and retrieve the in-memory pointer to the capability from our map
memStore.Set(types.RevCapabilityKey(sk.module, name), sdk.Uint64ToBigEndian(index))
// Set the mapping from index from index to in-memory capability in the go map
sk.capMap[index] = cap
logger(ctx).Info("created new capability", "module", sk.module, "name", name)
return cap, nil
@ -170,7 +193,7 @@ func (sk ScopedKeeper) NewCapability(ctx sdk.Context, name string) (*types.Capab
// Note, the capability's forward mapping is indexed by a string which should
// contain its unique memory reference.
func (sk ScopedKeeper) AuthenticateCapability(ctx sdk.Context, cap *types.Capability, name string) bool {
return sk.capStore.GetCapabilityName(sk.module, cap) == name
return sk.GetCapabilityName(ctx, cap) == name
}
// ClaimCapability attempts to claim a given Capability. The provided name and
@ -184,13 +207,17 @@ func (sk ScopedKeeper) ClaimCapability(ctx sdk.Context, cap *types.Capability, n
return err
}
memStore := ctx.KVStore(sk.memKey)
// Set the forward mapping between the module and capability tuple and the
// capability name in the in-memory store.
sk.capStore.SetCapabilityName(sk.module, name, cap)
// capability name in the memKVStore
memStore.Set(types.FwdCapabilityKey(sk.module, cap), []byte(name))
// Set the reverse mapping between the module and capability name and the
// capability in the in-memory store.
sk.capStore.SetCapability(sk.module, name, cap)
// index in the in-memory store. Since marshalling and unmarshalling into a store
// will change memory address of capability, we simply store index as value here
// and retrieve the in-memory pointer to the capability from our map
memStore.Set(types.RevCapabilityKey(sk.module, name), sdk.Uint64ToBigEndian(cap.GetIndex()))
logger(ctx).Info("claimed capability", "module", sk.module, "name", name, "capability", cap.GetIndex())
return nil
@ -200,18 +227,22 @@ func (sk ScopedKeeper) ClaimCapability(ctx sdk.Context, cap *types.Capability, n
// previously claimed or created. After releasing the capability, if no more
// owners exist, the capability will be globally removed.
func (sk ScopedKeeper) ReleaseCapability(ctx sdk.Context, cap *types.Capability) error {
name := sk.capStore.GetCapabilityName(sk.module, cap)
name := sk.GetCapabilityName(ctx, cap)
if len(name) == 0 {
return sdkerrors.Wrap(types.ErrCapabilityNotOwned, sk.module)
}
// Remove the forward mapping between the module and capability tuple and the
// capability name in the in-memory store.
sk.capStore.DeleteCapabilityName(sk.module, cap)
memStore := ctx.KVStore(sk.memKey)
// Remove the reverse mapping between the module and capability name and the
// capability in the in-memory store.
sk.capStore.DeleteCapability(sk.module, name)
// Set the forward mapping between the module and capability tuple and the
// capability name in the memKVStore
memStore.Delete(types.FwdCapabilityKey(sk.module, cap))
// Set the reverse mapping between the module and capability name and the
// index in the in-memory store. Since marshalling and unmarshalling into a store
// will change memory address of capability, we simply store index as value here
// and retrieve the in-memory pointer to the capability from our map
memStore.Delete(types.RevCapabilityKey(sk.module, name))
// remove owner
capOwners := sk.getOwners(ctx, cap)
@ -223,6 +254,8 @@ func (sk ScopedKeeper) ReleaseCapability(ctx sdk.Context, cap *types.Capability)
if len(capOwners.Owners) == 0 {
// remove capability owner set
prefixStore.Delete(indexKey)
// since no one ones capability, we can delete capability from map
delete(sk.capMap, cap.GetIndex())
} else {
// update capability owner set
prefixStore.Set(indexKey, sk.cdc.MustMarshalBinaryBare(capOwners))
@ -235,14 +268,38 @@ func (sk ScopedKeeper) ReleaseCapability(ctx sdk.Context, cap *types.Capability)
// by name. The module is not allowed to retrieve capabilities which it does not
// own.
func (sk ScopedKeeper) GetCapability(ctx sdk.Context, name string) (*types.Capability, bool) {
cap := sk.capStore.GetCapability(sk.module, name)
memStore := ctx.KVStore(sk.memKey)
key := types.RevCapabilityKey(sk.module, name)
indexBytes := memStore.Get(key)
index := sdk.BigEndianToUint64(indexBytes)
if len(indexBytes) == 0 {
// If a tx failed and NewCapability got reverted, it is possible
// to still have the capability in the go map since changes to
// go map do not automatically get reverted on tx failure,
// so we delete here to remove unnecessary values in map
delete(sk.capMap, index)
return nil, false
}
cap := sk.capMap[index]
if cap == nil {
// delete key from store to remove unnecessary mapping
memStore.Delete(key)
return nil, false
}
return cap, true
}
// GetCapabilityName allows a module to retrieve the name under which it stored a given
// capability given the capability
func (sk ScopedKeeper) GetCapabilityName(ctx sdk.Context, cap *types.Capability) string {
memStore := ctx.KVStore(sk.memKey)
return string(memStore.Get(types.FwdCapabilityKey(sk.module, cap)))
}
// Get all the Owners that own the capability associated with the name this ScopedKeeper uses
// to refer to the capability
func (sk ScopedKeeper) GetOwners(ctx sdk.Context, name string) (*types.CapabilityOwners, bool) {

View File

@ -30,7 +30,7 @@ func (suite *KeeperTestSuite) SetupTest() {
cdc := codec.NewHybridCodec(app.Codec())
// create new keeper so we can define custom scoping before init and seal
keeper := keeper.NewKeeper(cdc, app.GetKey(capability.StoreKey))
keeper := keeper.NewKeeper(cdc, app.GetKey(capability.StoreKey), app.GetMemKey(capability.MemStoreKey))
suite.ctx = app.BaseApp.NewContext(checkTx, abci.Header{Height: 1})
suite.keeper = keeper
@ -240,6 +240,37 @@ func (suite *KeeperTestSuite) TestReleaseCapability() {
suite.Require().Nil(got)
}
func (suite KeeperTestSuite) TestRevertCapability() {
sk := suite.keeper.ScopeToModule(bank.ModuleName)
ms := suite.ctx.MultiStore()
msCache := ms.CacheMultiStore()
cacheCtx := suite.ctx.WithMultiStore(msCache)
capName := "revert"
// Create capability on cached context
cap, err := sk.NewCapability(cacheCtx, capName)
suite.Require().NoError(err, "could not create capability")
// Check that capability written in cached context
gotCache, ok := sk.GetCapability(cacheCtx, capName)
suite.Require().True(ok, "could not retrieve capability from cached context")
suite.Require().Equal(cap, gotCache, "did not get correct capability from cached context")
// Check that capability is NOT written to original context
got, ok := sk.GetCapability(suite.ctx, capName)
suite.Require().False(ok, "retrieved capability from original context before write")
suite.Require().Nil(got, "capability not nil in original store")
// Write to underlying memKVStore
msCache.Write()
got, ok = sk.GetCapability(suite.ctx, capName)
suite.Require().True(ok, "could not retrieve capability from context")
suite.Require().Equal(cap, got, "did not get correct capability from context")
}
func TestKeeperTestSuite(t *testing.T) {
suite.Run(t, new(KeeperTestSuite))
}

View File

@ -1,56 +0,0 @@
package types
// CapabilityStore defines an ephemeral in-memory object capability store.
type CapabilityStore struct {
revMemStore map[string]*Capability
fwdMemStore map[string]string
}
func NewCapabilityStore() CapabilityStore {
return CapabilityStore{
revMemStore: make(map[string]*Capability),
fwdMemStore: make(map[string]string),
}
}
// GetCapability returns a Capability by module and name tuple. If no Capability
// exists, nil will be returned.
func (cs CapabilityStore) GetCapability(module, name string) *Capability {
key := RevCapabilityKey(module, name)
return cs.revMemStore[string(key)]
}
// GetCapabilityName returns a Capability name by module and Capability tuple. If
// no Capability name exists for the given tuple, an empty string is returned.
func (cs CapabilityStore) GetCapabilityName(module string, cap *Capability) string {
key := FwdCapabilityKey(module, cap)
return cs.fwdMemStore[string(key)]
}
// SetCapability sets the reverse mapping between the module and capability name
// and the capability in the in-memory store.
func (cs CapabilityStore) SetCapability(module, name string, cap *Capability) {
key := RevCapabilityKey(module, name)
cs.revMemStore[string(key)] = cap
}
// SetCapabilityName sets the forward mapping between the module and capability
// tuple and the capability name in the in-memory store.
func (cs CapabilityStore) SetCapabilityName(module, name string, cap *Capability) {
key := FwdCapabilityKey(module, cap)
cs.fwdMemStore[string(key)] = name
}
// DeleteCapability removes the reverse mapping between the module and capability
// name and the capability in the in-memory store.
func (cs CapabilityStore) DeleteCapability(module, name string) {
key := RevCapabilityKey(module, name)
delete(cs.revMemStore, string(key))
}
// DeleteCapabilityName removes the forward mapping between the module and capability
// tuple and the capability name in the in-memory store.
func (cs CapabilityStore) DeleteCapabilityName(module string, cap *Capability) {
key := FwdCapabilityKey(module, cap)
delete(cs.fwdMemStore, string(key))
}