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:
parent
4da4bb64df
commit
d247184157
|
@ -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) {
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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))
|
||||
}
|
|
@ -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 }
|
|
@ -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))
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
)
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
|
|
|
@ -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))
|
||||
}
|
Loading…
Reference in New Issue