From 0b732a44abe67f1eb4c606aa01cf524e52c6fa95 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Mon, 30 Aug 2021 23:34:58 +0800 Subject: [PATCH 01/47] add store/v2 types, RootStore --- store/v2/types.go | 80 +++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 80 insertions(+) create mode 100644 store/v2/types.go diff --git a/store/v2/types.go b/store/v2/types.go new file mode 100644 index 000000000000..ba1f4af8fcb8 --- /dev/null +++ b/store/v2/types.go @@ -0,0 +1,80 @@ +package types + +import ( + "io" + + snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" + v1 "github.com/cosmos/cosmos-sdk/store/types" +) + +type StoreKey = v1.StoreKey +type CommitID = v1.CommitID +type StoreUpgrades = v1.StoreUpgrades +type Iterator = v1.Iterator +type PruningOptions = v1.PruningOptions + +type TraceContext = v1.TraceContext +type WriteListener = v1.WriteListener + +type BasicKVStore = v1.BasicKVStore +type KVStore = v1.KVStore +type Committer = v1.Committer +type CommitKVStore = v1.CommitKVStore +type CacheKVStore = v1.CacheKVStore +type Queryable = v1.Queryable +type CacheWrap = v1.CacheWrap + +var ( + PruneDefault = v1.PruneDefault + PruneEverything = v1.PruneEverything + PruneNothing = v1.PruneNothing +) + +type BasicRootStore interface { + GetKVStore(StoreKey) KVStore + CacheRootStore() CacheRootStore +} + +type rootStoreTraceListen interface { + TracingEnabled() bool + SetTracer(w io.Writer) + SetTraceContext(TraceContext) + ListeningEnabled(key StoreKey) bool + AddListeners(key StoreKey, listeners []WriteListener) +} + +type RootStore interface { + BasicRootStore + rootStoreTraceListen + + GetVersion(uint64) (BasicRootStore, error) +} + +type CommitRootStore interface { + RootStore + Committer + snapshottypes.Snapshotter // or PortableStore + SetInitialVersion(uint64) error +} + +type CacheRootStore interface { + BasicRootStore + rootStoreTraceListen + Write() +} + +// provides inter-block (persistent) caching capabilities for a CommitRootStore +// TODO +type RootStorePersistentCache = v1.MultiStorePersistentCache + +//---------------------------------------- +// Store types + +type StoreType = v1.StoreType + +// Valid types +const StoreTypeMemory = v1.StoreTypeMemory +const StoreTypeTransient = v1.StoreTypeTransient +const StoreTypeDecoupled = v1.StoreTypeDecoupled +const StoreTypeSMT = v1.StoreTypeSMT +const StoreTypePersistent = StoreTypeDecoupled From 2ffea145005c310bb1f0c799995f5a33e98efa8c Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Mon, 6 Sep 2021 11:12:33 +0800 Subject: [PATCH 02/47] store/v2 dbadapter --- store/v2/dbadapter/store.go | 95 ++++++++++++++++++++++++++++++++ store/v2/dbadapter/store_test.go | 90 ++++++++++++++++++++++++++++++ 2 files changed, 185 insertions(+) create mode 100644 store/v2/dbadapter/store.go create mode 100644 store/v2/dbadapter/store_test.go diff --git a/store/v2/dbadapter/store.go b/store/v2/dbadapter/store.go new file mode 100644 index 000000000000..2f0ceb5df54a --- /dev/null +++ b/store/v2/dbadapter/store.go @@ -0,0 +1,95 @@ +package dbadapter + +import ( + "io" + + dbm "github.com/tendermint/tm-db" + + "github.com/cosmos/cosmos-sdk/store/cachekv" + "github.com/cosmos/cosmos-sdk/store/listenkv" + "github.com/cosmos/cosmos-sdk/store/tracekv" + "github.com/cosmos/cosmos-sdk/store/types" +) + +// Wrapper type for dbm.Db with implementation of KVStore +type Store struct { + dbm.DB +} + +// Get wraps the underlying DB's Get method panicing on error. +func (dsa Store) Get(key []byte) []byte { + v, err := dsa.DB.Get(key) + if err != nil { + panic(err) + } + + return v +} + +// Has wraps the underlying DB's Has method panicing on error. +func (dsa Store) Has(key []byte) bool { + ok, err := dsa.DB.Has(key) + if err != nil { + panic(err) + } + + return ok +} + +// Set wraps the underlying DB's Set method panicing on error. +func (dsa Store) Set(key, value []byte) { + types.AssertValidKey(key) + if err := dsa.DB.Set(key, value); err != nil { + panic(err) + } +} + +// Delete wraps the underlying DB's Delete method panicing on error. +func (dsa Store) Delete(key []byte) { + if err := dsa.DB.Delete(key); err != nil { + panic(err) + } +} + +// Iterator wraps the underlying DB's Iterator method panicing on error. +func (dsa Store) Iterator(start, end []byte) types.Iterator { + iter, err := dsa.DB.Iterator(start, end) + if err != nil { + panic(err) + } + + return iter +} + +// ReverseIterator wraps the underlying DB's ReverseIterator method panicing on error. +func (dsa Store) ReverseIterator(start, end []byte) types.Iterator { + iter, err := dsa.DB.ReverseIterator(start, end) + if err != nil { + panic(err) + } + + return iter +} + +// GetStoreType returns the type of the store. +func (Store) GetStoreType() types.StoreType { + return types.StoreTypeDB +} + +// CacheWrap branches the underlying store. +func (dsa Store) CacheWrap() types.CacheWrap { + return cachekv.NewStore(dsa) +} + +// CacheWrapWithTrace implements KVStore. +func (dsa Store) CacheWrapWithTrace(w io.Writer, tc types.TraceContext) types.CacheWrap { + return cachekv.NewStore(tracekv.NewStore(dsa, w, tc)) +} + +// CacheWrapWithListeners implements the CacheWrapper interface. +func (dsa Store) CacheWrapWithListeners(storeKey types.StoreKey, listeners []types.WriteListener) types.CacheWrap { + return cachekv.NewStore(listenkv.NewStore(dsa, storeKey, listeners)) +} + +// dbm.DB implements KVStore so we can CacheKVStore it. +var _ types.KVStore = Store{} diff --git a/store/v2/dbadapter/store_test.go b/store/v2/dbadapter/store_test.go new file mode 100644 index 000000000000..9f8ac71b25cf --- /dev/null +++ b/store/v2/dbadapter/store_test.go @@ -0,0 +1,90 @@ +package dbadapter_test + +import ( + "bytes" + "errors" + "testing" + + "github.com/cosmos/cosmos-sdk/store/cachekv" + + "github.com/golang/mock/gomock" + "github.com/stretchr/testify/require" + + "github.com/cosmos/cosmos-sdk/store/dbadapter" + "github.com/cosmos/cosmos-sdk/store/types" + "github.com/cosmos/cosmos-sdk/tests/mocks" +) + +var errFoo = errors.New("dummy") + +func TestAccessors(t *testing.T) { + mockCtrl := gomock.NewController(t) + defer mockCtrl.Finish() + + mockDB := mocks.NewMockDB(mockCtrl) + store := dbadapter.Store{mockDB} + key := []byte("test") + value := []byte("testvalue") + + require.Panics(t, func() { store.Set(nil, []byte("value")) }, "setting a nil key should panic") + require.Panics(t, func() { store.Set([]byte(""), []byte("value")) }, "setting an empty key should panic") + + require.Equal(t, types.StoreTypeDB, store.GetStoreType()) + store.GetStoreType() + + retFoo := []byte("xxx") + mockDB.EXPECT().Get(gomock.Eq(key)).Times(1).Return(retFoo, nil) + require.True(t, bytes.Equal(retFoo, store.Get(key))) + + mockDB.EXPECT().Get(gomock.Eq(key)).Times(1).Return(nil, errFoo) + require.Panics(t, func() { store.Get(key) }) + + mockDB.EXPECT().Has(gomock.Eq(key)).Times(1).Return(true, nil) + require.True(t, store.Has(key)) + + mockDB.EXPECT().Has(gomock.Eq(key)).Times(1).Return(false, nil) + require.False(t, store.Has(key)) + + mockDB.EXPECT().Has(gomock.Eq(key)).Times(1).Return(false, errFoo) + require.Panics(t, func() { store.Has(key) }) + + mockDB.EXPECT().Set(gomock.Eq(key), gomock.Eq(value)).Times(1).Return(nil) + require.NotPanics(t, func() { store.Set(key, value) }) + + mockDB.EXPECT().Set(gomock.Eq(key), gomock.Eq(value)).Times(1).Return(errFoo) + require.Panics(t, func() { store.Set(key, value) }) + + mockDB.EXPECT().Delete(gomock.Eq(key)).Times(1).Return(nil) + require.NotPanics(t, func() { store.Delete(key) }) + + mockDB.EXPECT().Delete(gomock.Eq(key)).Times(1).Return(errFoo) + require.Panics(t, func() { store.Delete(key) }) + + start, end := []byte("start"), []byte("end") + mockDB.EXPECT().Iterator(gomock.Eq(start), gomock.Eq(end)).Times(1).Return(nil, nil) + require.NotPanics(t, func() { store.Iterator(start, end) }) + + mockDB.EXPECT().Iterator(gomock.Eq(start), gomock.Eq(end)).Times(1).Return(nil, errFoo) + require.Panics(t, func() { store.Iterator(start, end) }) + + mockDB.EXPECT().ReverseIterator(gomock.Eq(start), gomock.Eq(end)).Times(1).Return(nil, nil) + require.NotPanics(t, func() { store.ReverseIterator(start, end) }) + + mockDB.EXPECT().ReverseIterator(gomock.Eq(start), gomock.Eq(end)).Times(1).Return(nil, errFoo) + require.Panics(t, func() { store.ReverseIterator(start, end) }) +} + +func TestCacheWraps(t *testing.T) { + mockCtrl := gomock.NewController(t) + mockDB := mocks.NewMockDB(mockCtrl) + store := dbadapter.Store{mockDB} + + cacheWrapper := store.CacheWrap() + require.IsType(t, &cachekv.Store{}, cacheWrapper) + + cacheWrappedWithTrace := store.CacheWrapWithTrace(nil, nil) + require.IsType(t, &cachekv.Store{}, cacheWrappedWithTrace) + + cacheWrappedWithListeners := store.CacheWrapWithListeners(nil, nil) + require.IsType(t, &cachekv.Store{}, cacheWrappedWithListeners) +} From 66aec814e8617b47e9da7414cac9d70a427e3abb Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 15 Sep 2021 00:55:28 +0800 Subject: [PATCH 03/47] store/v2 mem & transient stores --- store/v2/mem/store.go | 42 ++++++++++++++++++++++++++++++ store/v2/transient/store.go | 44 ++++++++++++++++++++++++++++++++ store/v2/transient/store_test.go | 28 ++++++++++++++++++++ 3 files changed, 114 insertions(+) create mode 100644 store/v2/mem/store.go create mode 100644 store/v2/transient/store.go create mode 100644 store/v2/transient/store_test.go diff --git a/store/v2/mem/store.go b/store/v2/mem/store.go new file mode 100644 index 000000000000..6b2665839c36 --- /dev/null +++ b/store/v2/mem/store.go @@ -0,0 +1,42 @@ +package mem + +import ( + dbm "github.com/cosmos/cosmos-sdk/db" + "github.com/cosmos/cosmos-sdk/store/types" + "github.com/cosmos/cosmos-sdk/store/v2/dbadapter" +) + +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 + conn dbm.DBConnection +} + +// NewStore constructs a new in-memory store. +func NewStore(db dbm.DBConnection) *Store { + return &Store{ + Store: dbadapter.Store{DB: db.ReadWriter()}, + conn: db, + } +} + +// GetStoreType returns the Store's type. +func (s Store) GetStoreType() types.StoreType { + return types.StoreTypeMemory +} + +// Commit commits to the underlying DB. +func (s *Store) Commit() (id types.CommitID) { + return +} + +func (s *Store) SetPruning(pruning types.PruningOptions) {} +func (s *Store) GetPruning() types.PruningOptions { return types.PruningOptions{} } + +func (s Store) LastCommitID() (id types.CommitID) { return } diff --git a/store/v2/transient/store.go b/store/v2/transient/store.go new file mode 100644 index 000000000000..51ab85cff011 --- /dev/null +++ b/store/v2/transient/store.go @@ -0,0 +1,44 @@ +package transient + +import ( + dbm "github.com/cosmos/cosmos-sdk/db" + "github.com/cosmos/cosmos-sdk/store/types" + "github.com/cosmos/cosmos-sdk/store/v2/dbadapter" +) + +var ( + _ types.KVStore = (*Store)(nil) + _ types.Committer = (*Store)(nil) +) + +// Store is a wrapper for a memory store which does not persist data. +type Store struct { + dbadapter.Store + conn dbm.DBConnection +} + +// NewStore constructs a new transient store. +func NewStore(db dbm.DBConnection) *Store { + return &Store{ + Store: dbadapter.Store{DB: db.ReadWriter()}, + conn: db, + } +} + +// Implements Store. +func (ts *Store) GetStoreType() types.StoreType { + return types.StoreTypeTransient +} + +// Implements CommitStore +// Commit cleans up Store. +func (ts *Store) Commit() (id types.CommitID) { + ts.DB.Discard() + ts.Store = dbadapter.Store{DB: ts.conn.ReadWriter()} + return +} + +func (ts *Store) SetPruning(types.PruningOptions) {} +func (ts *Store) GetPruning() types.PruningOptions { return types.PruningOptions{} } + +func (ts *Store) LastCommitID() (id types.CommitID) { return } diff --git a/store/v2/transient/store_test.go b/store/v2/transient/store_test.go new file mode 100644 index 000000000000..e46ce085825f --- /dev/null +++ b/store/v2/transient/store_test.go @@ -0,0 +1,28 @@ +package transient_test + +import ( + "bytes" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/cosmos/cosmos-sdk/db/memdb" + "github.com/cosmos/cosmos-sdk/store/types" + "github.com/cosmos/cosmos-sdk/store/v2/transient" +) + +var k, v = []byte("hello"), []byte("world") + +func TestTransientStore(t *testing.T) { + tstore := transient.NewStore(memdb.NewDB()) + require.Nil(t, tstore.Get(k)) + tstore.Set(k, v) + require.Equal(t, v, tstore.Get(k)) + tstore.Commit() + require.Nil(t, tstore.Get(k)) + + emptyCommitID := tstore.LastCommitID() + require.Equal(t, emptyCommitID.Version, int64(0)) + require.True(t, bytes.Equal(emptyCommitID.Hash, nil)) + require.Equal(t, types.StoreTypeTransient, tstore.GetStoreType()) +} From 4d1735d60d0e036a119c3d0d6bafd2012ed4f180 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Mon, 6 Sep 2021 10:07:07 +0800 Subject: [PATCH 04/47] db iterator adapter --- internal/db/iterator_adapter.go | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) create mode 100644 internal/db/iterator_adapter.go diff --git a/internal/db/iterator_adapter.go b/internal/db/iterator_adapter.go new file mode 100644 index 000000000000..e7ac348b9223 --- /dev/null +++ b/internal/db/iterator_adapter.go @@ -0,0 +1,24 @@ +package db + +import ( + dbm "github.com/cosmos/cosmos-sdk/db" + storetypes "github.com/cosmos/cosmos-sdk/store/types" +) + +var _ = (*storetypes.Iterator)(nil) + +type dbAsStoreIter struct { + dbm.Iterator + valid bool +} + +// DBToStoreIterator returns an iterator wrapping the given iterator so that it satisfies the +// store/types.Iterator interface. +func DBToStoreIterator(source dbm.Iterator) *dbAsStoreIter { + ret := &dbAsStoreIter{Iterator: source} + ret.Next() + return ret +} + +func (it *dbAsStoreIter) Next() { it.valid = it.Iterator.Next() } +func (it *dbAsStoreIter) Valid() bool { return it.valid } From 5f8d49452f8fdb55fe686f56a0d7c36b13849f5f Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 14 Oct 2021 18:02:19 +0800 Subject: [PATCH 05/47] flat store additions GetVersion rename to databucket, indexbucket --- store/v2/flat/store.go | 34 +++++----- store/v2/flat/view_store.go | 126 ++++++++++++++++++++++++++++++++++++ 2 files changed, 143 insertions(+), 17 deletions(-) create mode 100644 store/v2/flat/view_store.go diff --git a/store/v2/flat/store.go b/store/v2/flat/store.go index 6cd06b6a2aa7..b00ff6849392 100644 --- a/store/v2/flat/store.go +++ b/store/v2/flat/store.go @@ -54,11 +54,11 @@ type StoreConfig struct { // optionally using separate backing key-value DBs for each. // Allows synchronized R/W access by locking. type Store struct { - stateDB dbm.DBConnection - stateTxn dbm.DBReadWriter - dataTxn dbm.DBReadWriter - merkleTxn dbm.DBReadWriter - indexTxn dbm.DBReadWriter + stateDB dbm.DBConnection + stateTxn dbm.DBReadWriter + dataBucket dbm.DBReadWriter + indexBucket dbm.DBReadWriter + merkleTxn dbm.DBReadWriter // State commitment (SC) KV store for current version merkleStore *smt.Store @@ -132,8 +132,8 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { return &Store{ stateDB: db, stateTxn: stateTxn, - dataTxn: prefix.NewPrefixReadWriter(stateTxn, dataPrefix), - indexTxn: prefix.NewPrefixReadWriter(stateTxn, indexPrefix), + dataBucket: prefix.NewPrefixReadWriter(stateTxn, dataPrefix), + indexBucket: prefix.NewPrefixReadWriter(stateTxn, indexPrefix), merkleTxn: merkleTxn, merkleStore: merkleStore, opts: opts, @@ -153,7 +153,7 @@ func (s *Store) Get(key []byte) []byte { s.mtx.RLock() defer s.mtx.RUnlock() - val, err := s.dataTxn.Get(key) + val, err := s.dataBucket.Get(key) if err != nil { panic(err) } @@ -165,7 +165,7 @@ func (s *Store) Has(key []byte) bool { s.mtx.RLock() defer s.mtx.RUnlock() - has, err := s.dataTxn.Has(key) + has, err := s.dataBucket.Has(key) if err != nil { panic(err) } @@ -177,13 +177,13 @@ func (s *Store) Set(key, value []byte) { s.mtx.Lock() defer s.mtx.Unlock() - err := s.dataTxn.Set(key, value) + err := s.dataBucket.Set(key, value) if err != nil { panic(err) } s.merkleStore.Set(key, value) khash := sha256.Sum256(key) - err = s.indexTxn.Set(khash[:], key) + err = s.indexBucket.Set(khash[:], key) if err != nil { panic(err) } @@ -196,8 +196,8 @@ func (s *Store) Delete(key []byte) { defer s.mtx.Unlock() s.merkleStore.Delete(key) - _ = s.indexTxn.Delete(khash[:]) - _ = s.dataTxn.Delete(key) + _ = s.indexBucket.Delete(khash[:]) + _ = s.dataBucket.Delete(key) } type contentsIterator struct { @@ -216,7 +216,7 @@ func (it *contentsIterator) Valid() bool { return it.valid } // Iterator implements KVStore. func (s *Store) Iterator(start, end []byte) types.Iterator { - iter, err := s.dataTxn.Iterator(start, end) + iter, err := s.dataBucket.Iterator(start, end) if err != nil { panic(err) } @@ -225,7 +225,7 @@ func (s *Store) Iterator(start, end []byte) types.Iterator { // ReverseIterator implements KVStore. func (s *Store) ReverseIterator(start, end []byte) types.Iterator { - iter, err := s.dataTxn.ReverseIterator(start, end) + iter, err := s.dataBucket.ReverseIterator(start, end) if err != nil { panic(err) } @@ -329,8 +329,8 @@ func (s *Store) commit(target uint64) (id *types.CommitID, err error) { } s.stateTxn = stateTxn - s.dataTxn = prefix.NewPrefixReadWriter(stateTxn, dataPrefix) - s.indexTxn = prefix.NewPrefixReadWriter(stateTxn, indexPrefix) + s.dataBucket = prefix.NewPrefixReadWriter(stateTxn, dataPrefix) + s.indexBucket = prefix.NewPrefixReadWriter(stateTxn, indexPrefix) s.merkleTxn = merkleTxn s.merkleStore = loadSMT(merkleTxn, root) diff --git a/store/v2/flat/view_store.go b/store/v2/flat/view_store.go new file mode 100644 index 000000000000..6206a3dc6f9c --- /dev/null +++ b/store/v2/flat/view_store.go @@ -0,0 +1,126 @@ +package flat + +import ( + "errors" + "io" + + dbm "github.com/cosmos/cosmos-sdk/db" + "github.com/cosmos/cosmos-sdk/db/prefix" + + util "github.com/cosmos/cosmos-sdk/internal" + "github.com/cosmos/cosmos-sdk/store/cachekv" + "github.com/cosmos/cosmos-sdk/store/listenkv" + "github.com/cosmos/cosmos-sdk/store/tracekv" + types "github.com/cosmos/cosmos-sdk/store/v2" + "github.com/cosmos/cosmos-sdk/store/v2/smt" +) + +var ErrReadOnly = errors.New("cannot modify read-only store") + +// Represents a read-only view of a store's contents at a given version. +type viewStore struct { + stateView dbm.DBReader + dataBucket dbm.DBReader + indexBucket dbm.DBReader + merkleView dbm.DBReader + merkleStore *smt.Store +} + +func (s *Store) GetVersion(ver int64) (ret *viewStore, err error) { + stateView, err := s.stateDB.ReaderAt(uint64(ver)) + if err != nil { + return + } + defer func() { + if err != nil { + err = util.CombineErrors(err, stateView.Discard(), "stateView.Discard also failed") + } + }() + + merkleView := stateView + if s.opts.MerkleDB != nil { + merkleView, err = s.opts.MerkleDB.ReaderAt(uint64(ver)) + if err != nil { + return + } + defer func() { + if err != nil { + err = util.CombineErrors(err, merkleView.Discard(), "merkleView.Discard also failed") + } + }() + } + root, err := stateView.Get(merkleRootKey) + if err != nil { + return + } + return &viewStore{ + stateView: stateView, + dataBucket: prefix.NewPrefixReader(stateView, dataPrefix), + merkleView: merkleView, + indexBucket: prefix.NewPrefixReader(stateView, indexPrefix), + merkleStore: loadSMT(dbm.ReaderAsReadWriter(merkleView), root), + }, nil +} + +// Get implements KVStore. +func (s *viewStore) Get(key []byte) []byte { + val, err := s.dataBucket.Get(key) + if err != nil { + panic(err) + } + return val +} + +// Has implements KVStore. +func (s *viewStore) Has(key []byte) bool { + has, err := s.dataBucket.Has(key) + if err != nil { + panic(err) + } + return has +} + +// Set implements KVStore. +func (s *viewStore) Set(key []byte, value []byte) { + panic(ErrReadOnly) +} + +// Delete implements KVStore. +func (s *viewStore) Delete(key []byte) { + panic(ErrReadOnly) +} + +// Iterator implements KVStore. +func (s *viewStore) Iterator(start, end []byte) types.Iterator { + iter, err := s.dataBucket.Iterator(start, end) + if err != nil { + panic(err) + } + return newIterator(iter) +} + +// ReverseIterator implements KVStore. +func (s *viewStore) ReverseIterator(start, end []byte) types.Iterator { + iter, err := s.dataBucket.ReverseIterator(start, end) + if err != nil { + panic(err) + } + return newIterator(iter) +} + +// GetStoreType implements Store. +func (s *viewStore) GetStoreType() types.StoreType { + return types.StoreTypeDecoupled +} + +func (st *viewStore) CacheWrap() types.CacheWrap { + return cachekv.NewStore(st) +} + +func (st *viewStore) CacheWrapWithTrace(w io.Writer, tc types.TraceContext) types.CacheWrap { + return cachekv.NewStore(tracekv.NewStore(st, w, tc)) +} + +func (st *viewStore) CacheWrapWithListeners(storeKey types.StoreKey, listeners []types.WriteListener) types.CacheWrap { + return cachekv.NewStore(listenkv.NewStore(st, storeKey, listeners)) +} From 4afd095498b8c2e0e9e177d5bf2ad5b0da25f332 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Mon, 25 Oct 2021 16:07:57 +0800 Subject: [PATCH 06/47] add RootStore implementation - GetVersion - migration (StoreUpgrades) - flat.Store query uses getversion --- db/prefix/prefix.go | 35 + store/v2/dbadapter/store.go | 18 +- store/v2/flat/root_store.go | 637 ++++++++++++++++++ store/v2/flat/root_store_test.go | 242 +++++++ store/v2/flat/store.go | 40 +- store/v2/flat/store_test.go | 2 +- .../v2/flat/{view_store.go => store_view.go} | 36 +- store/v2/types.go | 26 +- 8 files changed, 974 insertions(+), 62 deletions(-) create mode 100644 store/v2/flat/root_store.go create mode 100644 store/v2/flat/root_store_test.go rename store/v2/flat/{view_store.go => store_view.go} (73%) diff --git a/db/prefix/prefix.go b/db/prefix/prefix.go index 6116c79203df..3f97a91fa92c 100644 --- a/db/prefix/prefix.go +++ b/db/prefix/prefix.go @@ -15,8 +15,14 @@ type prefixRW struct { prefix []byte } +type prefixW struct { + db dbm.DBWriter + prefix []byte +} + var _ dbm.DBReader = (*prefixR)(nil) var _ dbm.DBReadWriter = (*prefixRW)(nil) +var _ dbm.DBWriter = (*prefixW)(nil) func NewPrefixReader(db dbm.DBReader, prefix []byte) prefixR { return prefixR{ @@ -32,6 +38,13 @@ func NewPrefixReadWriter(db dbm.DBReadWriter, prefix []byte) prefixRW { } } +func NewPrefixWriter(db dbm.DBWriter, prefix []byte) prefixW { + return prefixW{ + prefix: prefix, + db: db, + } +} + func prefixed(prefix, key []byte) []byte { return append(prefix, key...) } @@ -135,6 +148,28 @@ func (pdb prefixRW) Commit() error { return pdb.db.Commit() } // Discard implements DBReadWriter. func (pdb prefixRW) Discard() error { return pdb.db.Discard() } +// Set implements DBReadWriter. +func (pdb prefixW) Set(key []byte, value []byte) error { + if len(key) == 0 { + return dbm.ErrKeyEmpty + } + return pdb.db.Set(prefixed(pdb.prefix, key), value) +} + +// Delete implements DBReadWriter. +func (pdb prefixW) Delete(key []byte) error { + if len(key) == 0 { + return dbm.ErrKeyEmpty + } + return pdb.db.Delete(prefixed(pdb.prefix, key)) +} + +// Close implements DBReadWriter. +func (pdb prefixW) Commit() error { return pdb.db.Commit() } + +// Discard implements DBReadWriter. +func (pdb prefixW) Discard() error { return pdb.db.Discard() } + // Returns a slice of the same length (big endian), but incremented by one. // Returns nil on overflow (e.g. if bz bytes are all 0xFF) // CONTRACT: len(bz) > 0 diff --git a/store/v2/dbadapter/store.go b/store/v2/dbadapter/store.go index 2f0ceb5df54a..152314de8110 100644 --- a/store/v2/dbadapter/store.go +++ b/store/v2/dbadapter/store.go @@ -3,17 +3,22 @@ package dbadapter import ( "io" - dbm "github.com/tendermint/tm-db" + // dbm "github.com/tendermint/tm-db" + dbm "github.com/cosmos/cosmos-sdk/db" + dbutil "github.com/cosmos/cosmos-sdk/internal/db" "github.com/cosmos/cosmos-sdk/store/cachekv" "github.com/cosmos/cosmos-sdk/store/listenkv" "github.com/cosmos/cosmos-sdk/store/tracekv" "github.com/cosmos/cosmos-sdk/store/types" ) +// dbm.DB implements KVStore so we can CacheKVStore it. +var _ types.KVStore = Store{} + // Wrapper type for dbm.Db with implementation of KVStore type Store struct { - dbm.DB + DB dbm.DBReadWriter } // Get wraps the underlying DB's Get method panicing on error. @@ -58,7 +63,7 @@ func (dsa Store) Iterator(start, end []byte) types.Iterator { panic(err) } - return iter + return dbutil.DBToStoreIterator(iter) } // ReverseIterator wraps the underlying DB's ReverseIterator method panicing on error. @@ -68,11 +73,11 @@ func (dsa Store) ReverseIterator(start, end []byte) types.Iterator { panic(err) } - return iter + return dbutil.DBToStoreIterator(iter) } // GetStoreType returns the type of the store. -func (Store) GetStoreType() types.StoreType { +func (dsa Store) GetStoreType() types.StoreType { return types.StoreTypeDB } @@ -90,6 +95,3 @@ func (dsa Store) CacheWrapWithTrace(w io.Writer, tc types.TraceContext) types.Ca func (dsa Store) CacheWrapWithListeners(storeKey types.StoreKey, listeners []types.WriteListener) types.CacheWrap { return cachekv.NewStore(listenkv.NewStore(dsa, storeKey, listeners)) } - -// dbm.DB implements KVStore so we can CacheKVStore it. -var _ types.KVStore = Store{} diff --git a/store/v2/flat/root_store.go b/store/v2/flat/root_store.go new file mode 100644 index 000000000000..75f0f490aee8 --- /dev/null +++ b/store/v2/flat/root_store.go @@ -0,0 +1,637 @@ +package flat + +import ( + "errors" + "fmt" + "io" + "strings" + + abci "github.com/tendermint/tendermint/abci/types" + + dbm "github.com/cosmos/cosmos-sdk/db" + "github.com/cosmos/cosmos-sdk/db/memdb" + prefixdb "github.com/cosmos/cosmos-sdk/db/prefix" + util "github.com/cosmos/cosmos-sdk/internal" + "github.com/cosmos/cosmos-sdk/store/cachekv" + "github.com/cosmos/cosmos-sdk/store/prefix" + types "github.com/cosmos/cosmos-sdk/store/v2" + "github.com/cosmos/cosmos-sdk/store/v2/mem" + transkv "github.com/cosmos/cosmos-sdk/store/v2/transient" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" + "github.com/cosmos/cosmos-sdk/types/kv" +) + +var ( + _ types.CommitRootStore = (*rootStore)(nil) + _ types.CacheRootStore = (*rootCache)(nil) + _ types.BasicRootStore = (*rootView)(nil) +) + +var ( + schemaPrefix = []byte{5} // Prefix for store keys (prefixes) +) + +// RootStoreConfig is used to define a schema and pass options to the RootStore constructor. +type RootStoreConfig struct { + StoreConfig + PersistentCache types.RootStorePersistentCache + Upgrades []types.StoreUpgrades + prefixRegistry + *listenerMixin + *traceMixin +} + +// Represents the valid store types for a RootStore schema, a subset of the StoreType values +type subStoreType byte + +const ( + subStorePersistent subStoreType = iota + subStoreMemory + subStoreTransient +) + +// A loaded mapping of store names to types +type storeSchema map[string]subStoreType + +// Builder type used to create a valid schema with no prefix conflicts +type prefixRegistry struct { + storeSchema + reserved []string +} + +// Mixin types that will be composed into each distinct root store variant type +type listenerMixin struct { + listeners map[types.StoreKey][]types.WriteListener +} + +type traceMixin struct { + TraceWriter io.Writer + TraceContext types.TraceContext +} + +type storeMixin struct { + schema storeSchema + *listenerMixin + *traceMixin +} + +// Main persistent store type +type rootStore struct { + *Store + mem *mem.Store + tran *transkv.Store + storeMixin +} + +// Branched state +type rootCache struct { + types.CacheKVStore + mem, tran types.CacheKVStore + storeMixin +} + +// Read-only store for querying +type rootView struct { + *storeView + schema storeSchema + // storeMixin //? +} + +// Auxiliary type used only to avoid repetitive method implementations +type rootGeneric struct { + schema storeSchema + persist, mem, tran types.KVStore +} + +// DefaultRootStoreConfig returns a RootStore config with an empty schema, a single backing DB, +// pruning with PruneDefault, no listeners and no tracer. +func DefaultRootStoreConfig() RootStoreConfig { + return RootStoreConfig{ + StoreConfig: StoreConfig{Pruning: types.PruneDefault}, + prefixRegistry: prefixRegistry{ + storeSchema: storeSchema{}, + }, + listenerMixin: &listenerMixin{ + listeners: map[types.StoreKey][]types.WriteListener{}, + }, + traceMixin: &traceMixin{ + TraceWriter: nil, + TraceContext: nil, + }, + } +} + +func validSubStoreType(sst subStoreType) bool { + return byte(sst) <= byte(subStoreTransient) +} + +// Returns true iff both schema maps match exactly (including mem/tran stores) +func (this storeSchema) equal(that storeSchema) bool { + if len(this) != len(that) { + return false + } + for key, val := range that { + myval, has := this[key] + if !has { + return false + } + if val != myval { + return false + } + } + return true +} + +// Parses a schema from the DB +func readSchema(bucket dbm.DBReader) (*prefixRegistry, error) { + ret := prefixRegistry{storeSchema: storeSchema{}} + it, err := bucket.Iterator(nil, nil) + if err != nil { + return nil, err + } + for it.Next() { + value := it.Value() + if len(value) != 1 || !validSubStoreType(subStoreType(value[0])) { + return nil, fmt.Errorf("invalid mapping for store key: %v => %v", it.Key(), value) + } + ret.storeSchema[string(it.Key())] = subStoreType(value[0]) + ret.reserved = append(ret.reserved, string(it.Key())) // assume iter yields keys sorted + } + it.Close() + return &ret, nil +} + +// NewRootStore constructs a RootStore directly from a DB connection and options. +func NewRootStore(db dbm.DBConnection, opts RootStoreConfig) (*rootStore, error) { + base, err := NewStore(db, opts.StoreConfig) + if err != nil { + return nil, err + } + return makeRootStore(base, opts) +} + +// TODO: +// should config contain the pre- or post-migration schema? - currently pre +func makeRootStore(base *Store, opts RootStoreConfig) (ret *rootStore, err error) { + schemaView := prefixdb.NewPrefixReader(base.stateDB.Reader(), schemaPrefix) + defer func() { + if err != nil { + err = util.CombineErrors(err, schemaView.Discard(), "schemaView.Discard also failed") + err = util.CombineErrors(err, base.Close(), "base.Close also failed") + } + }() + pr, err := readSchema(schemaView) + if err != nil { + return + } + // If the loaded schema is empty, just copy the config schema; + // Otherwise, verify it is identical to the config schema + if len(pr.storeSchema) == 0 { + for k, v := range opts.storeSchema { + pr.storeSchema[k] = v + } + pr.reserved = make([]string, len(opts.reserved)) + copy(pr.reserved, opts.reserved) + } else { + if !pr.equal(opts.storeSchema) { + err = errors.New("loaded schema does not match configured schema") + return + } + } + // Apply migrations, then clear old schema and write the new one + for _, upgrades := range opts.Upgrades { + err = pr.migrate(base, upgrades) + if err != nil { + return + } + } + schemaWriter := prefixdb.NewPrefixWriter(base.stateTxn, schemaPrefix) + it, err := schemaView.Iterator(nil, nil) + if err != nil { + return + } + for it.Next() { + err = schemaWriter.Delete(it.Key()) + if err != nil { + return + } + } + err = it.Close() + if err != nil { + return + } + err = schemaView.Discard() + if err != nil { + return + } + for skey, typ := range pr.storeSchema { + err = schemaWriter.Set([]byte(skey), []byte{byte(typ)}) + if err != nil { + return + } + } + // The migrated contents and schema are not committed until the next store.Commit + ret = &rootStore{ + Store: base, + mem: mem.NewStore(memdb.NewDB()), + tran: transkv.NewStore(memdb.NewDB()), + storeMixin: storeMixin{ + schema: pr.storeSchema, + listenerMixin: opts.listenerMixin, + traceMixin: opts.traceMixin, + }, + } + return +} + +// Applies store upgrades to the DB contents. +func (pr *prefixRegistry) migrate(store *Store, upgrades types.StoreUpgrades) error { + // branch state to allow mutation while iterating + branch := cachekv.NewStore(store) + + for _, key := range upgrades.Deleted { + sst, ix, err := pr.storeInfo(key) + if err != nil { + return err + } + if sst != subStorePersistent { + return fmt.Errorf("prefix is for non-persistent substore: %v (%v)", key, sst) + } + pr.reserved = append(pr.reserved[:ix], pr.reserved[ix+1:]...) + delete(pr.storeSchema, key) + + sub := prefix.NewStore(store, []byte(key)) + subbranch := prefix.NewStore(branch, []byte(key)) + it := sub.Iterator(nil, nil) + for ; it.Valid(); it.Next() { + subbranch.Delete(it.Key()) + } + it.Close() + } + for _, rename := range upgrades.Renamed { + sst, ix, err := pr.storeInfo(rename.OldKey) + if err != nil { + return err + } + if sst != subStorePersistent { + return fmt.Errorf("prefix is for non-persistent substore: %v (%v)", rename.OldKey, sst) + } + pr.reserved = append(pr.reserved[:ix], pr.reserved[ix+1:]...) + delete(pr.storeSchema, rename.OldKey) + err = pr.ReservePrefix(types.NewKVStoreKey(rename.NewKey), types.StoreTypePersistent) + if err != nil { + return err + } + + sub := prefix.NewStore(store, []byte(rename.OldKey)) + subbranch := prefix.NewStore(branch, []byte(rename.NewKey)) + it := sub.Iterator(nil, nil) + for ; it.Valid(); it.Next() { + subbranch.Set(it.Key(), it.Value()) + } + it.Close() + } + branch.Write() + + for _, key := range upgrades.Added { + err := pr.ReservePrefix(types.NewKVStoreKey(key), types.StoreTypePersistent) + if err != nil { + return err + } + } + return nil +} + +func (rs *rootStore) GetKVStore(key types.StoreKey) types.KVStore { + return rs.generic().getStore(key.Name()) +} + +func (rs *rootStore) Commit() types.CommitID { + id := rs.Store.Commit() + rs.tran.Commit() + return id +} + +func (rs *rootStore) Close() error { return rs.Store.Close() } + +func (rs *Store) SetInitialVersion(version uint64) error { + rs.opts.InitialVersion = uint64(version) + return nil +} + +func (rs *rootStore) GetVersion(version int64) (types.BasicRootStore, error) { + return rs.getView(version) +} + +func (rs *rootStore) getView(version int64) (*rootView, error) { + view, err := rs.Store.GetVersion(version) + if err != nil { + return nil, err + } + return rs.makeRootView(view) +} + +func (rs *rootStore) makeRootView(view *storeView) (ret *rootView, err error) { + schemaView := prefixdb.NewPrefixReader(view.stateView, schemaPrefix) + defer func() { + if err != nil { + err = util.CombineErrors(err, schemaView.Discard(), "schemaView.Discard also failed") + } + }() + pr, err := readSchema(schemaView) + if err != nil { + return + } + // The migrated contents and schema are not committed until the next store.Commit + return &rootView{ + storeView: view, + schema: pr.storeSchema, + }, nil +} + +// if the schema indicates a mem/tran store, it's ignored +func (rv *rootView) generic() rootGeneric { return rootGeneric{rv.schema, rv, nil, nil} } + +func (rv *rootView) GetKVStore(key types.StoreKey) types.KVStore { + return rv.generic().getStore(key.Name()) +} + +// Copies only the schema +func newStoreMixin(schema storeSchema) storeMixin { + return storeMixin{ + schema: schema, + listenerMixin: &listenerMixin{}, + traceMixin: &traceMixin{}, + } +} + +func (rv *rootView) CacheRootStore() types.CacheRootStore { + return &rootCache{ + CacheKVStore: cachekv.NewStore(rv), + mem: cachekv.NewStore(mem.NewStore(memdb.NewDB())), + tran: cachekv.NewStore(transkv.NewStore(memdb.NewDB())), + storeMixin: newStoreMixin(rv.schema), + } +} + +func (rs *rootStore) CacheRootStore() types.CacheRootStore { + return &rootCache{ + CacheKVStore: cachekv.NewStore(rs), + mem: cachekv.NewStore(rs.mem), + tran: cachekv.NewStore(rs.tran), + storeMixin: newStoreMixin(rs.schema), + } +} + +// parsePath expects a format like /[/] +// Must start with /, subpath may be empty +// Returns error if it doesn't start with / +func parsePath(path string) (storeName string, subpath string, err error) { + if !strings.HasPrefix(path, "/") { + return storeName, subpath, sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "invalid path: %s", path) + } + + paths := strings.SplitN(path[1:], "/", 2) + storeName = paths[0] + + if len(paths) == 2 { + subpath = "/" + paths[1] + } + + return storeName, subpath, nil +} + +// Query implements ABCI interface, allows queries. +// +// by default we will return from (latest height -1), +// as we will have merkle proofs immediately (header height = data height + 1) +// If latest-1 is not present, use latest (which must be present) +// if you care to have the latest data to see a tx results, you must +// explicitly set the height you want to see +func (rs *rootStore) Query(req abci.RequestQuery) (res abci.ResponseQuery) { + if len(req.Data) == 0 { + return sdkerrors.QueryResult(sdkerrors.Wrap(sdkerrors.ErrTxDecode, "query cannot be zero length"), false) + } + + // if height is 0, use the latest height + height := req.Height + if height == 0 { + versions, err := rs.stateDB.Versions() + if err != nil { + return sdkerrors.QueryResult(errors.New("failed to get version info"), false) + } + latest := versions.Last() + if versions.Exists(latest - 1) { + height = int64(latest - 1) + } else { + height = int64(latest) + } + } + if height < 0 { + return sdkerrors.QueryResult(fmt.Errorf("height overflow: %v", height), false) + } + res.Height = height + + storeName, subpath, err := parsePath(req.Path) + if err != nil { + return sdkerrors.QueryResult(err, false) + } + + // trim the path and make the query + // req.Path = subpath + // res := rs.Store.Query(req) + + switch subpath { + case "/key": + var err error + res.Key = req.Data // data holds the key bytes + + view, err := rs.getView(height) + if err != nil { + if errors.Is(err, dbm.ErrVersionDoesNotExist) { + err = sdkerrors.ErrInvalidHeight + } + return sdkerrors.QueryResult(err, false) + } + + substore := view.generic().getStore(storeName) + if substore == nil { + return sdkerrors.QueryResult(sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "no such store: %s", storeName), false) + } + res.Value = substore.Get(res.Key) + if !req.Prove { + break + } + // res.ProofOps, err = view.prove(storeName, res.Key) + fullkey := storeName + string(res.Key) + res.ProofOps, err = view.merkleStore.GetProof([]byte(fullkey)) + if err != nil { + return sdkerrors.QueryResult(fmt.Errorf("Merkle proof creation failed for key: %v", res.Key), false) + } + + case "/subspace": + pairs := kv.Pairs{ + Pairs: make([]kv.Pair, 0), + } + + subspace := req.Data + res.Key = subspace + + iterator := rs.Iterator(subspace, types.PrefixEndBytes(subspace)) + for ; iterator.Valid(); iterator.Next() { + pairs.Pairs = append(pairs.Pairs, kv.Pair{Key: iterator.Key(), Value: iterator.Value()}) + } + iterator.Close() + + bz, err := pairs.Marshal() + if err != nil { + panic(fmt.Errorf("failed to marshal KV pairs: %w", err)) + } + + res.Value = bz + + default: + return sdkerrors.QueryResult(sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "unexpected query path: %v", req.Path), false) + } + + return res +} + +func (rs *rootStore) generic() rootGeneric { return rootGeneric{rs.schema, rs, rs.mem, rs.tran} } + +func (store rootGeneric) getStore(key string) types.KVStore { + var sub types.KVStore + if typ, ok := store.schema[key]; ok { + switch typ { + case subStorePersistent: + sub = store.persist + case subStoreMemory: + sub = store.mem + case subStoreTransient: + sub = store.tran + } + } + if sub == nil { + panic(fmt.Sprintf("store does not exist for key: %s", key)) + } + return prefix.NewStore(sub, []byte(key)) +} + +func (rs *rootCache) GetKVStore(key types.StoreKey) types.KVStore { + return rs.generic().getStore(key.Name()) +} + +func (rs *rootCache) Write() { + rs.CacheKVStore.Write() + rs.mem.Write() + rs.tran.Write() +} + +// Recursively wraps the CacheRootStore in another cache store. +func (rs *rootCache) CacheRootStore() types.CacheRootStore { + return &rootCache{ + CacheKVStore: cachekv.NewStore(rs), + mem: cachekv.NewStore(rs.mem), + tran: cachekv.NewStore(rs.tran), + storeMixin: newStoreMixin(rs.schema), + } +} + +func (rs *rootCache) generic() rootGeneric { return rootGeneric{rs.schema, rs, rs.mem, rs.tran} } + +// Returns closest index and whether it's a match +func binarySearch(hay []string, ndl string) (int, bool) { + var mid int + from, to := 0, len(hay)-1 + for from <= to { + mid = (from + to) / 2 + switch strings.Compare(hay[mid], ndl) { + case -1: + from = mid + 1 + case 1: + to = mid - 1 + default: + return mid, true + } + } + return from, false +} + +func (pr *prefixRegistry) storeInfo(key string) (sst subStoreType, ix int, err error) { + ix, has := binarySearch(pr.reserved, key) + if !has { + err = fmt.Errorf("prefix does not exist: %v", key) + return + } + sst, has = pr.storeSchema[key] + if !has { + err = fmt.Errorf("prefix is registered but not in schema: %v", key) + } + + return +} + +func (pr *prefixRegistry) ReservePrefix(key types.StoreKey, typ types.StoreType) error { + // Find the neighboring reserved prefix, and check for duplicates and conflicts + i, has := binarySearch(pr.reserved, key.Name()) + if has { + return fmt.Errorf("prefix already exists: %v", key) + } + if i > 0 && strings.HasPrefix(key.Name(), pr.reserved[i-1]) { + return fmt.Errorf("prefix conflict: '%v' exists, cannot add '%v'", pr.reserved[i-1], key.Name()) + } + if i < len(pr.reserved) && strings.HasPrefix(pr.reserved[i], key.Name()) { + return fmt.Errorf("prefix conflict: '%v' exists, cannot add '%v'", pr.reserved[i], key.Name()) + } + reserved := pr.reserved[:i] + reserved = append(reserved, key.Name()) + pr.reserved = append(reserved, pr.reserved[i:]...) + + var sstype subStoreType + switch typ { + case types.StoreTypeDecoupled: + sstype = subStorePersistent + case types.StoreTypeMemory: + sstype = subStoreMemory + case types.StoreTypeTransient: + sstype = subStoreTransient + // case types.StoreTypeSMT: // could be used for external storage + default: + return fmt.Errorf("StoreType not supported: %v", typ) + } + pr.storeSchema[key.Name()] = sstype + return nil +} + +func (lreg *listenerMixin) AddListeners(key types.StoreKey, listeners []types.WriteListener) { + if ls, ok := lreg.listeners[key]; ok { + lreg.listeners[key] = append(ls, listeners...) + } else { + lreg.listeners[key] = listeners + } +} + +// ListeningEnabled returns if listening is enabled for a specific KVStore +func (lreg *listenerMixin) ListeningEnabled(key types.StoreKey) bool { + if ls, ok := lreg.listeners[key]; ok { + return len(ls) != 0 + } + return false +} + +func (treg *traceMixin) TracingEnabled() bool { + return treg.TraceWriter != nil +} +func (treg *traceMixin) SetTracer(w io.Writer) { + treg.TraceWriter = w +} +func (treg *traceMixin) SetTraceContext(tc types.TraceContext) { + treg.TraceContext = tc +} + +func (rs *rootStore) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser, ready chan<- struct{}) error { + return nil +} +func (rs *rootStore) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) { + return nil, nil +} diff --git a/store/v2/flat/root_store_test.go b/store/v2/flat/root_store_test.go new file mode 100644 index 000000000000..de26f9fdd143 --- /dev/null +++ b/store/v2/flat/root_store_test.go @@ -0,0 +1,242 @@ +package flat + +import ( + "testing" + + "github.com/stretchr/testify/require" + + "github.com/cosmos/cosmos-sdk/db/memdb" + types "github.com/cosmos/cosmos-sdk/store/v2" +) + +var ( + skey_1 = types.NewKVStoreKey("store1") + skey_2 = types.NewKVStoreKey("store2") + skey_3 = types.NewKVStoreKey("store3") + skey_1b = types.NewKVStoreKey("store1b") + skey_2b = types.NewKVStoreKey("store2b") + skey_3b = types.NewKVStoreKey("store3b") +) + +func storeConfig123(t *testing.T) RootStoreConfig { + opts := DefaultRootStoreConfig() + opts.Pruning = types.PruneNothing + err := opts.ReservePrefix(skey_1, types.StoreTypePersistent) + require.NoError(t, err) + err = opts.ReservePrefix(skey_2, types.StoreTypePersistent) + require.NoError(t, err) + err = opts.ReservePrefix(skey_3, types.StoreTypePersistent) + require.NoError(t, err) + return opts +} + +func TestRootStoreConfig(t *testing.T) { + opts := DefaultRootStoreConfig() + // Ensure that no prefix conflicts are allowed + err := opts.ReservePrefix(skey_1, types.StoreTypePersistent) + require.NoError(t, err) + err = opts.ReservePrefix(skey_2, types.StoreTypePersistent) + require.NoError(t, err) + err = opts.ReservePrefix(skey_3b, types.StoreTypePersistent) + require.NoError(t, err) + err = opts.ReservePrefix(skey_1b, types.StoreTypePersistent) + require.Error(t, err) + err = opts.ReservePrefix(skey_2b, types.StoreTypePersistent) + require.Error(t, err) + err = opts.ReservePrefix(skey_3, types.StoreTypePersistent) + require.Error(t, err) +} + +func TestRootStoreBasic(t *testing.T) { + opts := DefaultRootStoreConfig() + err := opts.ReservePrefix(skey_1, types.StoreTypePersistent) + require.NoError(t, err) + db := memdb.NewDB() + store, err := NewRootStore(db, opts) + require.NoError(t, err) + + store_1 := store.GetKVStore(skey_1) + require.NotNil(t, store_1) + store_1.Set([]byte{0}, []byte{0}) + val := store_1.Get([]byte{0}) + require.Equal(t, []byte{0}, val) + store_1.Delete([]byte{0}) + val = store_1.Get([]byte{0}) + require.Equal(t, []byte(nil), val) +} + +func TestGetVersion(t *testing.T) { + db := memdb.NewDB() + opts := storeConfig123(t) + store, err := NewRootStore(db, opts) + require.NoError(t, err) + + cid := store.Commit() + // opts := DefaultRootStoreConfig() + + view, err := store.GetVersion(cid.Version) + require.NoError(t, err) + subview := view.GetKVStore(skey_1) + require.NotNil(t, subview) + + // version view should be read-only + require.Panics(t, func() { subview.Set([]byte{1}, []byte{1}) }) + require.Panics(t, func() { subview.Delete([]byte{0}) }) + // nonexistent version shouldn't be accessible + view, err = store.GetVersion(cid.Version + 1) + require.Equal(t, ErrVersionDoesNotExist, err) + + substore := store.GetKVStore(skey_1) + require.NotNil(t, substore) + substore.Set([]byte{0}, []byte{0}) + // setting a new value shouldn't affect old version + require.False(t, subview.Has([]byte{0})) + + cid = store.Commit() + view, err = store.GetVersion(cid.Version) + require.NoError(t, err) + subview = view.GetKVStore(skey_1) + require.NotNil(t, subview) + + store.Delete([]byte{0}) + require.Equal(t, []byte{0}, subview.Get([]byte{0})) +} + +func TestRootStoreMigration(t *testing.T) { + skey_2b := types.NewKVStoreKey("store2b") + skey_4 := types.NewKVStoreKey("store4") + + db := memdb.NewDB() + opts := storeConfig123(t) + store, err := NewRootStore(db, opts) + require.NoError(t, err) + + // write some data in all stores + k1, v1 := []byte("first"), []byte("store") + s1 := store.GetKVStore(skey_1) + require.NotNil(t, s1) + s1.Set(k1, v1) + + k2, v2 := []byte("second"), []byte("restore") + s2 := store.GetKVStore(skey_2) + require.NotNil(t, s2) + s2.Set(k2, v2) + + k3, v3 := []byte("third"), []byte("dropped") + s3 := store.GetKVStore(skey_3) + require.NotNil(t, s3) + s3.Set(k3, v3) + + require.Panics(t, func() { store.GetKVStore(skey_4) }) + + cid := store.Commit() + require.NoError(t, store.Close()) + + // Load without changes and make sure it is sensible + store, err = NewRootStore(db, opts) + require.NoError(t, err) + + // let's query data to see it was saved properly + s2 = store.GetKVStore(skey_2) + require.NotNil(t, s2) + require.Equal(t, v2, s2.Get(k2)) + require.NoError(t, store.Close()) + + // now, let's load with upgrades... + opts.Upgrades = []types.StoreUpgrades{ + types.StoreUpgrades{ + Added: []string{skey_4.Name()}, + Renamed: []types.StoreRename{{ + OldKey: skey_2.Name(), + NewKey: skey_2b.Name(), + }}, + Deleted: []string{skey_3.Name()}, + }, + } + restore, err := NewRootStore(db, opts) + require.Nil(t, err) + + // s1 was not changed + s1 = restore.GetKVStore(skey_1) + require.NotNil(t, s1) + require.Equal(t, v1, s1.Get(k1)) + + // store3 is gone + require.Panics(t, func() { s3 = restore.GetKVStore(skey_3) }) + + // store4 is mounted, with empty data + s4 := restore.GetKVStore(skey_4) + require.NotNil(t, s4) + + values := 0 + it := s4.Iterator(nil, nil) + for ; it.Valid(); it.Next() { + values += 1 + } + require.Zero(t, values) + require.NoError(t, it.Close()) + + // write something inside store4 + k4, v4 := []byte("fourth"), []byte("created") + s4.Set(k4, v4) + + // store2 is no longer mounted + require.Panics(t, func() { restore.GetKVStore(skey_2) }) + + // restore2 has the old data + rs2 := restore.GetKVStore(skey_2b) + require.NotNil(t, rs2) + require.Equal(t, v2, rs2.Get(k2)) + + // store this migrated data, and load it again without migrations + migratedID := restore.Commit() + require.Equal(t, migratedID.Version, int64(2)) + require.NoError(t, restore.Close()) + + // fail to load the migrated store with the old schema + reload, err := NewRootStore(db, storeConfig123(t)) + require.Error(t, err) + + // pass in a schema reflecting the migrations + migratedOpts := DefaultRootStoreConfig() + err = migratedOpts.ReservePrefix(skey_1, types.StoreTypePersistent) + require.NoError(t, err) + err = migratedOpts.ReservePrefix(skey_2b, types.StoreTypePersistent) + require.NoError(t, err) + err = migratedOpts.ReservePrefix(skey_4, types.StoreTypePersistent) + require.NoError(t, err) + reload, err = NewRootStore(db, migratedOpts) + require.Nil(t, err) + require.Equal(t, migratedID, reload.LastCommitID()) + + // query this new store + rl1 := reload.GetKVStore(skey_1) + require.NotNil(t, rl1) + require.Equal(t, v1, rl1.Get(k1)) + + rl2 := reload.GetKVStore(skey_2b) + require.NotNil(t, rl2) + require.Equal(t, v2, rl2.Get(k2)) + + rl4 := reload.GetKVStore(skey_4) + require.NotNil(t, rl4) + require.Equal(t, v4, rl4.Get(k4)) + + // load and check a view of the store at first commit + view, err := store.GetVersion(cid.Version) + require.NoError(t, err) + + s1 = view.GetKVStore(skey_1) + require.NotNil(t, s1) + require.Equal(t, v1, s1.Get(k1)) + + s2 = view.GetKVStore(skey_2) + require.NotNil(t, s2) + require.Equal(t, v2, s2.Get(k2)) + + s3 = view.GetKVStore(skey_3) + require.NotNil(t, s3) + require.Equal(t, v3, s3.Get(k3)) + + require.Panics(t, func() { view.GetKVStore(skey_4) }) +} diff --git a/store/v2/flat/store.go b/store/v2/flat/store.go index b00ff6849392..926b4688c539 100644 --- a/store/v2/flat/store.go +++ b/store/v2/flat/store.go @@ -8,10 +8,10 @@ import ( "math" "sync" - dbm "github.com/cosmos/cosmos-sdk/db" - "github.com/cosmos/cosmos-sdk/db/prefix" abci "github.com/tendermint/tendermint/abci/types" + dbm "github.com/cosmos/cosmos-sdk/db" + "github.com/cosmos/cosmos-sdk/db/prefix" util "github.com/cosmos/cosmos-sdk/internal" "github.com/cosmos/cosmos-sdk/store/cachekv" "github.com/cosmos/cosmos-sdk/store/listenkv" @@ -43,11 +43,11 @@ var ( type StoreConfig struct { // Version pruning options for backing DBs. - Pruning types.PruningOptions + Pruning types.PruningOptions + InitialVersion uint64 // The backing DB to use for the state commitment Merkle tree data. // If nil, Merkle data is stored in the state storage DB under a separate prefix. - MerkleDB dbm.DBConnection - InitialVersion uint64 + MerkleDB dbm.DBConnection } // Store is a CommitKVStore which handles state storage and commitments as separate concerns, @@ -68,7 +68,7 @@ type Store struct { var DefaultStoreConfig = StoreConfig{Pruning: types.PruneDefault, MerkleDB: nil} -// NewStore creates a new Store, or loads one if db contains existing data. +// NewStore creates a new Store, or loads one if the DB contains existing data. func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { versions, err := db.Versions() if err != nil { @@ -394,40 +394,18 @@ func (s *Store) Query(req abci.RequestQuery) (res abci.ResponseQuery) { var err error res.Key = req.Data // data holds the key bytes - dbr, err := s.stateDB.ReaderAt(uint64(height)) + view, err := s.GetVersion(height) if err != nil { if errors.Is(err, dbm.ErrVersionDoesNotExist) { err = sdkerrors.ErrInvalidHeight } return sdkerrors.QueryResult(err, false) } - defer dbr.Discard() - contents := prefix.NewPrefixReader(dbr, dataPrefix) - res.Value, err = contents.Get(res.Key) - if err != nil { - return sdkerrors.QueryResult(err, false) - } + res.Value = view.Get(res.Key) if !req.Prove { break } - merkleView := dbr - if s.opts.MerkleDB != nil { - merkleView, err = s.opts.MerkleDB.ReaderAt(uint64(height)) - if err != nil { - return sdkerrors.QueryResult( - fmt.Errorf("version exists in state DB but not Merkle DB: %v", height), false) - } - defer merkleView.Discard() - } - root, err := dbr.Get(merkleRootKey) - if err != nil { - return sdkerrors.QueryResult(err, false) - } - if root == nil { - return sdkerrors.QueryResult(errors.New("Merkle root hash not found"), false) //nolint: stylecheck // proper name - } - merkleStore := loadSMT(dbm.ReaderAsReadWriter(merkleView), root) - res.ProofOps, err = merkleStore.GetProof(res.Key) + res.ProofOps, err = view.GetMerkleStore().GetProof(res.Key) if err != nil { return sdkerrors.QueryResult(fmt.Errorf("Merkle proof creation failed for key: %v", res.Key), false) //nolint: stylecheck // proper name } diff --git a/store/v2/flat/store_test.go b/store/v2/flat/store_test.go index c3f559b31c2f..f6fd5e59daa4 100644 --- a/store/v2/flat/store_test.go +++ b/store/v2/flat/store_test.go @@ -63,7 +63,7 @@ func TestGetSetHasDelete(t *testing.T) { require.Panics(t, func() { store.Set(nil, []byte("value")) }, "Set(nil key) should panic") require.Panics(t, func() { store.Set([]byte{}, []byte("value")) }, "Set(empty key) should panic") require.Panics(t, func() { store.Set([]byte("key"), nil) }, "Set(nil value) should panic") - store.indexTxn = rwCrudFails{store.indexTxn} + store.indexBucket = rwCrudFails{store.indexBucket} require.Panics(t, func() { store.Set([]byte("key"), []byte("value")) }, "Set() when index fails should panic") } diff --git a/store/v2/flat/view_store.go b/store/v2/flat/store_view.go similarity index 73% rename from store/v2/flat/view_store.go rename to store/v2/flat/store_view.go index 6206a3dc6f9c..76b578698853 100644 --- a/store/v2/flat/view_store.go +++ b/store/v2/flat/store_view.go @@ -18,7 +18,7 @@ import ( var ErrReadOnly = errors.New("cannot modify read-only store") // Represents a read-only view of a store's contents at a given version. -type viewStore struct { +type storeView struct { stateView dbm.DBReader dataBucket dbm.DBReader indexBucket dbm.DBReader @@ -26,8 +26,8 @@ type viewStore struct { merkleStore *smt.Store } -func (s *Store) GetVersion(ver int64) (ret *viewStore, err error) { - stateView, err := s.stateDB.ReaderAt(uint64(ver)) +func (s *Store) GetVersion(version int64) (ret *storeView, err error) { + stateView, err := s.stateDB.ReaderAt(uint64(version)) if err != nil { return } @@ -39,7 +39,7 @@ func (s *Store) GetVersion(ver int64) (ret *viewStore, err error) { merkleView := stateView if s.opts.MerkleDB != nil { - merkleView, err = s.opts.MerkleDB.ReaderAt(uint64(ver)) + merkleView, err = s.opts.MerkleDB.ReaderAt(uint64(version)) if err != nil { return } @@ -53,17 +53,21 @@ func (s *Store) GetVersion(ver int64) (ret *viewStore, err error) { if err != nil { return } - return &viewStore{ + return &storeView{ stateView: stateView, dataBucket: prefix.NewPrefixReader(stateView, dataPrefix), - merkleView: merkleView, indexBucket: prefix.NewPrefixReader(stateView, indexPrefix), + merkleView: merkleView, merkleStore: loadSMT(dbm.ReaderAsReadWriter(merkleView), root), }, nil } +func (s *storeView) GetMerkleStore() *smt.Store { + return s.merkleStore +} + // Get implements KVStore. -func (s *viewStore) Get(key []byte) []byte { +func (s *storeView) Get(key []byte) []byte { val, err := s.dataBucket.Get(key) if err != nil { panic(err) @@ -72,7 +76,7 @@ func (s *viewStore) Get(key []byte) []byte { } // Has implements KVStore. -func (s *viewStore) Has(key []byte) bool { +func (s *storeView) Has(key []byte) bool { has, err := s.dataBucket.Has(key) if err != nil { panic(err) @@ -81,17 +85,17 @@ func (s *viewStore) Has(key []byte) bool { } // Set implements KVStore. -func (s *viewStore) Set(key []byte, value []byte) { +func (s *storeView) Set(key []byte, value []byte) { panic(ErrReadOnly) } // Delete implements KVStore. -func (s *viewStore) Delete(key []byte) { +func (s *storeView) Delete(key []byte) { panic(ErrReadOnly) } // Iterator implements KVStore. -func (s *viewStore) Iterator(start, end []byte) types.Iterator { +func (s *storeView) Iterator(start, end []byte) types.Iterator { iter, err := s.dataBucket.Iterator(start, end) if err != nil { panic(err) @@ -100,7 +104,7 @@ func (s *viewStore) Iterator(start, end []byte) types.Iterator { } // ReverseIterator implements KVStore. -func (s *viewStore) ReverseIterator(start, end []byte) types.Iterator { +func (s *storeView) ReverseIterator(start, end []byte) types.Iterator { iter, err := s.dataBucket.ReverseIterator(start, end) if err != nil { panic(err) @@ -109,18 +113,18 @@ func (s *viewStore) ReverseIterator(start, end []byte) types.Iterator { } // GetStoreType implements Store. -func (s *viewStore) GetStoreType() types.StoreType { +func (s *storeView) GetStoreType() types.StoreType { return types.StoreTypeDecoupled } -func (st *viewStore) CacheWrap() types.CacheWrap { +func (st *storeView) CacheWrap() types.CacheWrap { return cachekv.NewStore(st) } -func (st *viewStore) CacheWrapWithTrace(w io.Writer, tc types.TraceContext) types.CacheWrap { +func (st *storeView) CacheWrapWithTrace(w io.Writer, tc types.TraceContext) types.CacheWrap { return cachekv.NewStore(tracekv.NewStore(st, w, tc)) } -func (st *viewStore) CacheWrapWithListeners(storeKey types.StoreKey, listeners []types.WriteListener) types.CacheWrap { +func (st *storeView) CacheWrapWithListeners(storeKey types.StoreKey, listeners []types.WriteListener) types.CacheWrap { return cachekv.NewStore(listenkv.NewStore(st, storeKey, listeners)) } diff --git a/store/v2/types.go b/store/v2/types.go index ba1f4af8fcb8..6762eed69cfc 100644 --- a/store/v2/types.go +++ b/store/v2/types.go @@ -10,6 +10,7 @@ import ( type StoreKey = v1.StoreKey type CommitID = v1.CommitID type StoreUpgrades = v1.StoreUpgrades +type StoreRename = v1.StoreRename type Iterator = v1.Iterator type PruningOptions = v1.PruningOptions @@ -24,6 +25,10 @@ type CacheKVStore = v1.CacheKVStore type Queryable = v1.Queryable type CacheWrap = v1.CacheWrap +type KVStoreKey = v1.KVStoreKey +type MemoryStoreKey = v1.MemoryStoreKey +type TransientStoreKey = v1.TransientStoreKey + var ( PruneDefault = v1.PruneDefault PruneEverything = v1.PruneEverything @@ -31,7 +36,10 @@ var ( ) type BasicRootStore interface { + // Returns a KVStore which has access only to the namespace of the StoreKey. + // Panics if the key is not found in the schema. GetKVStore(StoreKey) KVStore + // Returns a branched whose modifications are later merged back in. CacheRootStore() CacheRootStore } @@ -43,17 +51,19 @@ type rootStoreTraceListen interface { AddListeners(key StoreKey, listeners []WriteListener) } -type RootStore interface { +type CommitRootStore interface { BasicRootStore rootStoreTraceListen - GetVersion(uint64) (BasicRootStore, error) -} + // Gets a read-only view of the store at a specific version. + // Returns an error if the version is not found. + GetVersion(int64) (BasicRootStore, error) + // Closes the store and all backing transactions. + Close() error -type CommitRootStore interface { - RootStore + // RootStore Committer - snapshottypes.Snapshotter // or PortableStore + snapshottypes.Snapshotter // todo: PortableStore? SetInitialVersion(uint64) error } @@ -76,5 +86,9 @@ type StoreType = v1.StoreType const StoreTypeMemory = v1.StoreTypeMemory const StoreTypeTransient = v1.StoreTypeTransient const StoreTypeDecoupled = v1.StoreTypeDecoupled +const StoreTypeDB = v1.StoreTypeDB const StoreTypeSMT = v1.StoreTypeSMT const StoreTypePersistent = StoreTypeDecoupled + +var NewKVStoreKey = v1.NewKVStoreKey +var PrefixEndBytes = v1.PrefixEndBytes From 3b3f63ec48bc599fbba9dfaf4283d2ab3eeb8f43 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Fri, 12 Nov 2021 16:31:05 +0800 Subject: [PATCH 07/47] rename Merkle* to StateCommitment* --- store/v2/flat/root_store.go | 2 +- store/v2/flat/store.go | 92 ++++++++++++++++++------------------- store/v2/flat/store_test.go | 34 +++++++------- store/v2/flat/store_view.go | 32 ++++++------- 4 files changed, 80 insertions(+), 80 deletions(-) diff --git a/store/v2/flat/root_store.go b/store/v2/flat/root_store.go index 75f0f490aee8..dc1baf01cac0 100644 --- a/store/v2/flat/root_store.go +++ b/store/v2/flat/root_store.go @@ -464,7 +464,7 @@ func (rs *rootStore) Query(req abci.RequestQuery) (res abci.ResponseQuery) { } // res.ProofOps, err = view.prove(storeName, res.Key) fullkey := storeName + string(res.Key) - res.ProofOps, err = view.merkleStore.GetProof([]byte(fullkey)) + res.ProofOps, err = view.stateCommitmentStore.GetProof([]byte(fullkey)) if err != nil { return sdkerrors.QueryResult(fmt.Errorf("Merkle proof creation failed for key: %v", res.Key), false) } diff --git a/store/v2/flat/store.go b/store/v2/flat/store.go index 926b4688c539..64ba923080ff 100644 --- a/store/v2/flat/store.go +++ b/store/v2/flat/store.go @@ -47,26 +47,26 @@ type StoreConfig struct { InitialVersion uint64 // The backing DB to use for the state commitment Merkle tree data. // If nil, Merkle data is stored in the state storage DB under a separate prefix. - MerkleDB dbm.DBConnection + StateCommitmentDB dbm.DBConnection } // Store is a CommitKVStore which handles state storage and commitments as separate concerns, // optionally using separate backing key-value DBs for each. // Allows synchronized R/W access by locking. type Store struct { - stateDB dbm.DBConnection - stateTxn dbm.DBReadWriter - dataBucket dbm.DBReadWriter - indexBucket dbm.DBReadWriter - merkleTxn dbm.DBReadWriter + stateDB dbm.DBConnection + stateTxn dbm.DBReadWriter + dataBucket dbm.DBReadWriter + indexBucket dbm.DBReadWriter + stateCommitmentTxn dbm.DBReadWriter // State commitment (SC) KV store for current version - merkleStore *smt.Store + stateCommitmentStore *smt.Store opts StoreConfig mtx sync.RWMutex } -var DefaultStoreConfig = StoreConfig{Pruning: types.PruneDefault, MerkleDB: nil} +var DefaultStoreConfig = StoreConfig{Pruning: types.PruneDefault, StateCommitmentDB: nil} // NewStore creates a new Store, or loads one if the DB contains existing data. func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { @@ -93,26 +93,26 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { err = util.CombineErrors(err, stateTxn.Discard(), "stateTxn.Discard also failed") } }() - merkleTxn := stateTxn - if opts.MerkleDB != nil { + stateCommitmentTxn := stateTxn + if opts.StateCommitmentDB != nil { var mversions dbm.VersionSet - mversions, err = opts.MerkleDB.Versions() + mversions, err = opts.StateCommitmentDB.Versions() if err != nil { return } // Version sets of each DB must match if !versions.Equal(mversions) { - err = fmt.Errorf("storage and Merkle DB have different version history") + err = fmt.Errorf("Storage and StateCommitment DB have different version history") //nolint:stylecheck return } - err = opts.MerkleDB.Revert() + err = opts.StateCommitmentDB.Revert() if err != nil { return } - merkleTxn = opts.MerkleDB.ReadWriter() + stateCommitmentTxn = opts.StateCommitmentDB.ReadWriter() } - var merkleStore *smt.Store + var stateCommitmentStore *smt.Store if loadExisting { var root []byte root, err = stateTxn.Get(merkleRootKey) @@ -123,27 +123,27 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { err = fmt.Errorf("could not get root of SMT") return } - merkleStore = loadSMT(merkleTxn, root) + stateCommitmentStore = loadSMT(stateCommitmentTxn, root) } else { - merkleNodes := prefix.NewPrefixReadWriter(merkleTxn, merkleNodePrefix) - merkleValues := prefix.NewPrefixReadWriter(merkleTxn, merkleValuePrefix) - merkleStore = smt.NewStore(merkleNodes, merkleValues) + merkleNodes := prefix.NewPrefixReadWriter(stateCommitmentTxn, merkleNodePrefix) + merkleValues := prefix.NewPrefixReadWriter(stateCommitmentTxn, merkleValuePrefix) + stateCommitmentStore = smt.NewStore(merkleNodes, merkleValues) } return &Store{ - stateDB: db, - stateTxn: stateTxn, - dataBucket: prefix.NewPrefixReadWriter(stateTxn, dataPrefix), - indexBucket: prefix.NewPrefixReadWriter(stateTxn, indexPrefix), - merkleTxn: merkleTxn, - merkleStore: merkleStore, - opts: opts, + stateDB: db, + stateTxn: stateTxn, + dataBucket: prefix.NewPrefixReadWriter(stateTxn, dataPrefix), + indexBucket: prefix.NewPrefixReadWriter(stateTxn, indexPrefix), + stateCommitmentTxn: stateCommitmentTxn, + stateCommitmentStore: stateCommitmentStore, + opts: opts, }, nil } func (s *Store) Close() error { err := s.stateTxn.Discard() - if s.opts.MerkleDB != nil { - err = util.CombineErrors(err, s.merkleTxn.Discard(), "merkleTxn.Discard also failed") + if s.opts.StateCommitmentDB != nil { + err = util.CombineErrors(err, s.stateCommitmentTxn.Discard(), "stateCommitmentTxn.Discard also failed") } return err } @@ -181,7 +181,7 @@ func (s *Store) Set(key, value []byte) { if err != nil { panic(err) } - s.merkleStore.Set(key, value) + s.stateCommitmentStore.Set(key, value) khash := sha256.Sum256(key) err = s.indexBucket.Set(khash[:], key) if err != nil { @@ -195,7 +195,7 @@ func (s *Store) Delete(key []byte) { s.mtx.Lock() defer s.mtx.Unlock() - s.merkleStore.Delete(key) + s.stateCommitmentStore.Delete(key) _ = s.indexBucket.Delete(khash[:]) _ = s.dataBucket.Delete(key) } @@ -264,8 +264,8 @@ func (s *Store) Commit() types.CommitID { for version := firstPrunable; version <= lastPrunable; version++ { if s.opts.Pruning.KeepEvery == 0 || version%int64(s.opts.Pruning.KeepEvery) != 0 { s.stateDB.DeleteVersion(uint64(version)) - if s.opts.MerkleDB != nil { - s.opts.MerkleDB.DeleteVersion(uint64(version)) + if s.opts.StateCommitmentDB != nil { + s.opts.StateCommitmentDB.DeleteVersion(uint64(version)) } } } @@ -274,7 +274,7 @@ func (s *Store) Commit() types.CommitID { } func (s *Store) commit(target uint64) (id *types.CommitID, err error) { - root := s.merkleStore.Root() + root := s.stateCommitmentStore.Root() err = s.stateTxn.Set(merkleRootKey, root) if err != nil { return @@ -299,10 +299,10 @@ func (s *Store) commit(target uint64) (id *types.CommitID, err error) { err = util.CombineErrors(err, stateTxn.Discard(), "stateTxn.Discard also failed") } }() - merkleTxn := stateTxn + stateCommitmentTxn := stateTxn - // If DBs are not separate, Merkle state has been commmitted & snapshotted - if s.opts.MerkleDB != nil { + // If DBs are not separate, StateCommitment state has been commmitted & snapshotted + if s.opts.StateCommitmentDB != nil { defer func() { if err != nil { if delerr := s.stateDB.DeleteVersion(target); delerr != nil { @@ -311,28 +311,28 @@ func (s *Store) commit(target uint64) (id *types.CommitID, err error) { } }() - err = s.merkleTxn.Commit() + err = s.stateCommitmentTxn.Commit() if err != nil { return } defer func() { if err != nil { - err = util.CombineErrors(err, s.opts.MerkleDB.Revert(), "merkleDB.Revert also failed") + err = util.CombineErrors(err, s.opts.StateCommitmentDB.Revert(), "stateCommitmentDB.Revert also failed") } }() - err = s.opts.MerkleDB.SaveVersion(target) + err = s.opts.StateCommitmentDB.SaveVersion(target) if err != nil { return } - merkleTxn = s.opts.MerkleDB.ReadWriter() + stateCommitmentTxn = s.opts.StateCommitmentDB.ReadWriter() } s.stateTxn = stateTxn s.dataBucket = prefix.NewPrefixReadWriter(stateTxn, dataPrefix) s.indexBucket = prefix.NewPrefixReadWriter(stateTxn, indexPrefix) - s.merkleTxn = merkleTxn - s.merkleStore = loadSMT(merkleTxn, root) + s.stateCommitmentTxn = stateCommitmentTxn + s.stateCommitmentStore = loadSMT(stateCommitmentTxn, root) return &types.CommitID{Version: int64(target), Hash: root}, nil } @@ -405,7 +405,7 @@ func (s *Store) Query(req abci.RequestQuery) (res abci.ResponseQuery) { if !req.Prove { break } - res.ProofOps, err = view.GetMerkleStore().GetProof(res.Key) + res.ProofOps, err = view.GetStateCommitmentStore().GetProof(res.Key) if err != nil { return sdkerrors.QueryResult(fmt.Errorf("Merkle proof creation failed for key: %v", res.Key), false) //nolint: stylecheck // proper name } @@ -438,9 +438,9 @@ func (s *Store) Query(req abci.RequestQuery) (res abci.ResponseQuery) { return res } -func loadSMT(merkleTxn dbm.DBReadWriter, root []byte) *smt.Store { - merkleNodes := prefix.NewPrefixReadWriter(merkleTxn, merkleNodePrefix) - merkleValues := prefix.NewPrefixReadWriter(merkleTxn, merkleValuePrefix) +func loadSMT(stateCommitmentTxn dbm.DBReadWriter, root []byte) *smt.Store { + merkleNodes := prefix.NewPrefixReadWriter(stateCommitmentTxn, merkleNodePrefix) + merkleValues := prefix.NewPrefixReadWriter(stateCommitmentTxn, merkleValuePrefix) return smt.LoadStore(merkleNodes, merkleValues, root) } diff --git a/store/v2/flat/store_test.go b/store/v2/flat/store_test.go index f6fd5e59daa4..1fda4c52a55c 100644 --- a/store/v2/flat/store_test.go +++ b/store/v2/flat/store_test.go @@ -89,7 +89,7 @@ func TestConstructors(t *testing.T) { store, err = NewStore(dbVersionsFails{memdb.NewDB()}, DefaultStoreConfig) require.Error(t, err) - store, err = NewStore(db, StoreConfig{MerkleDB: dbVersionsFails{memdb.NewDB()}}) + store, err = NewStore(db, StoreConfig{StateCommitmentDB: dbVersionsFails{memdb.NewDB()}}) require.Error(t, err) // can't use a DB with open writers @@ -100,13 +100,13 @@ func TestConstructors(t *testing.T) { require.Error(t, err) w.Discard() w = merkledb.Writer() - store, err = NewStore(db, StoreConfig{MerkleDB: merkledb}) + store, err = NewStore(db, StoreConfig{StateCommitmentDB: merkledb}) require.Error(t, err) w.Discard() // can't use DBs with different version history merkledb.SaveNextVersion() - store, err = NewStore(db, StoreConfig{MerkleDB: merkledb}) + store, err = NewStore(db, StoreConfig{StateCommitmentDB: merkledb}) require.Error(t, err) merkledb.Close() @@ -217,7 +217,7 @@ func TestCommit(t *testing.T) { } } testBasic(StoreConfig{Pruning: types.PruneNothing}) - testBasic(StoreConfig{Pruning: types.PruneNothing, MerkleDB: memdb.NewDB()}) + testBasic(StoreConfig{Pruning: types.PruneNothing, StateCommitmentDB: memdb.NewDB()}) testFailedCommit := func(t *testing.T, store *Store, db dbm.DBConnection) { opts := store.opts @@ -231,8 +231,8 @@ func TestCommit(t *testing.T) { versions, _ := db.Versions() require.Equal(t, 0, versions.Count()) - if opts.MerkleDB != nil { - versions, _ = opts.MerkleDB.Versions() + if opts.StateCommitmentDB != nil { + versions, _ = opts.StateCommitmentDB.Versions() require.Equal(t, 0, versions.Count()) } @@ -257,15 +257,15 @@ func TestCommit(t *testing.T) { require.NoError(t, err) testFailedCommit(t, store, nil) }) - t.Run("recover after failed MerkleDB Commit", func(t *testing.T) { + t.Run("recover after failed StateCommitmentDB Commit", func(t *testing.T) { store, err := NewStore(memdb.NewDB(), - StoreConfig{MerkleDB: dbRWCommitFails{memdb.NewDB()}, Pruning: types.PruneNothing}) + StoreConfig{StateCommitmentDB: dbRWCommitFails{memdb.NewDB()}, Pruning: types.PruneNothing}) require.NoError(t, err) testFailedCommit(t, store, nil) }) - t.Run("recover after failed MerkleDB SaveVersion", func(t *testing.T) { + t.Run("recover after failed StateCommitmentDB SaveVersion", func(t *testing.T) { store, err := NewStore(memdb.NewDB(), - StoreConfig{MerkleDB: dbSaveVersionFails{memdb.NewDB()}, Pruning: types.PruneNothing}) + StoreConfig{StateCommitmentDB: dbSaveVersionFails{memdb.NewDB()}, Pruning: types.PruneNothing}) require.NoError(t, err) testFailedCommit(t, store, nil) }) @@ -285,9 +285,9 @@ func TestCommit(t *testing.T) { }) t.Run("stateDB.DeleteVersion error triggers failure", func(t *testing.T) { - store, err := NewStore(memdb.NewDB(), StoreConfig{MerkleDB: memdb.NewDB()}) + store, err := NewStore(memdb.NewDB(), StoreConfig{StateCommitmentDB: memdb.NewDB()}) require.NoError(t, err) - store.merkleTxn = rwCommitFails{store.merkleTxn} + store.stateCommitmentTxn = rwCommitFails{store.stateCommitmentTxn} store.stateDB = dbDeleteVersionFails{store.stateDB} require.Panics(t, func() { store.Commit() }) }) @@ -302,17 +302,17 @@ func TestCommit(t *testing.T) { // setting initial version store, err := NewStore(memdb.NewDB(), - StoreConfig{InitialVersion: 5, Pruning: types.PruneNothing, MerkleDB: memdb.NewDB()}) + StoreConfig{InitialVersion: 5, Pruning: types.PruneNothing, StateCommitmentDB: memdb.NewDB()}) require.NoError(t, err) require.Equal(t, int64(5), store.Commit().Version) - store, err = NewStore(memdb.NewDB(), StoreConfig{MerkleDB: memdb.NewDB()}) + store, err = NewStore(memdb.NewDB(), StoreConfig{StateCommitmentDB: memdb.NewDB()}) require.NoError(t, err) store.Commit() store.stateDB = dbVersionsFails{store.stateDB} require.Panics(t, func() { store.LastCommitID() }) - store, err = NewStore(memdb.NewDB(), StoreConfig{MerkleDB: memdb.NewDB()}) + store, err = NewStore(memdb.NewDB(), StoreConfig{StateCommitmentDB: memdb.NewDB()}) require.NoError(t, err) store.Commit() store.stateTxn = rwCrudFails{store.stateTxn} @@ -341,7 +341,7 @@ func TestPruning(t *testing.T) { for tci, tc := range testCases { dbs := []dbm.DBConnection{memdb.NewDB(), memdb.NewDB()} - store, err := NewStore(dbs[0], StoreConfig{Pruning: tc.PruningOptions, MerkleDB: dbs[1]}) + store, err := NewStore(dbs[0], StoreConfig{Pruning: tc.PruningOptions, StateCommitmentDB: dbs[1]}) require.NoError(t, err) for i := byte(1); i <= 10; i++ { @@ -545,7 +545,7 @@ func TestQuery(t *testing.T) { testProve() store.Close() - store, err = NewStore(memdb.NewDB(), StoreConfig{MerkleDB: memdb.NewDB()}) + store, err = NewStore(memdb.NewDB(), StoreConfig{StateCommitmentDB: memdb.NewDB()}) require.NoError(t, err) store.Set(k1, v1) store.Commit() diff --git a/store/v2/flat/store_view.go b/store/v2/flat/store_view.go index 76b578698853..bcc5e08be398 100644 --- a/store/v2/flat/store_view.go +++ b/store/v2/flat/store_view.go @@ -19,11 +19,11 @@ var ErrReadOnly = errors.New("cannot modify read-only store") // Represents a read-only view of a store's contents at a given version. type storeView struct { - stateView dbm.DBReader - dataBucket dbm.DBReader - indexBucket dbm.DBReader - merkleView dbm.DBReader - merkleStore *smt.Store + stateView dbm.DBReader + dataBucket dbm.DBReader + indexBucket dbm.DBReader + stateCommitmentView dbm.DBReader + stateCommitmentStore *smt.Store } func (s *Store) GetVersion(version int64) (ret *storeView, err error) { @@ -37,15 +37,15 @@ func (s *Store) GetVersion(version int64) (ret *storeView, err error) { } }() - merkleView := stateView - if s.opts.MerkleDB != nil { - merkleView, err = s.opts.MerkleDB.ReaderAt(uint64(version)) + stateCommitmentView := stateView + if s.opts.StateCommitmentDB != nil { + stateCommitmentView, err = s.opts.StateCommitmentDB.ReaderAt(uint64(version)) if err != nil { return } defer func() { if err != nil { - err = util.CombineErrors(err, merkleView.Discard(), "merkleView.Discard also failed") + err = util.CombineErrors(err, stateCommitmentView.Discard(), "stateCommitmentView.Discard also failed") } }() } @@ -54,16 +54,16 @@ func (s *Store) GetVersion(version int64) (ret *storeView, err error) { return } return &storeView{ - stateView: stateView, - dataBucket: prefix.NewPrefixReader(stateView, dataPrefix), - indexBucket: prefix.NewPrefixReader(stateView, indexPrefix), - merkleView: merkleView, - merkleStore: loadSMT(dbm.ReaderAsReadWriter(merkleView), root), + stateView: stateView, + dataBucket: prefix.NewPrefixReader(stateView, dataPrefix), + indexBucket: prefix.NewPrefixReader(stateView, indexPrefix), + stateCommitmentView: stateCommitmentView, + stateCommitmentStore: loadSMT(dbm.ReaderAsReadWriter(stateCommitmentView), root), }, nil } -func (s *storeView) GetMerkleStore() *smt.Store { - return s.merkleStore +func (s *storeView) GetStateCommitmentStore() *smt.Store { + return s.stateCommitmentStore } // Get implements KVStore. From 792e93fe5087a256d381edadbc5a9dccb1e36a7c Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Fri, 12 Nov 2021 17:17:27 +0800 Subject: [PATCH 08/47] move store/v2/flat.* => store/v2/root.* --- store/v2/{flat => root}/root_store.go | 2 +- store/v2/{flat => root}/root_store_test.go | 2 +- store/v2/{flat => root}/store.go | 2 +- store/v2/{flat => root}/store_test.go | 2 +- store/v2/{flat => root}/store_view.go | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) rename store/v2/{flat => root}/root_store.go (99%) rename store/v2/{flat => root}/root_store_test.go (99%) rename store/v2/{flat => root}/store.go (99%) rename store/v2/{flat => root}/store_test.go (99%) rename store/v2/{flat => root}/store_view.go (99%) diff --git a/store/v2/flat/root_store.go b/store/v2/root/root_store.go similarity index 99% rename from store/v2/flat/root_store.go rename to store/v2/root/root_store.go index dc1baf01cac0..2a88374b5442 100644 --- a/store/v2/flat/root_store.go +++ b/store/v2/root/root_store.go @@ -1,4 +1,4 @@ -package flat +package root import ( "errors" diff --git a/store/v2/flat/root_store_test.go b/store/v2/root/root_store_test.go similarity index 99% rename from store/v2/flat/root_store_test.go rename to store/v2/root/root_store_test.go index de26f9fdd143..bc10e7e36b86 100644 --- a/store/v2/flat/root_store_test.go +++ b/store/v2/root/root_store_test.go @@ -1,4 +1,4 @@ -package flat +package root import ( "testing" diff --git a/store/v2/flat/store.go b/store/v2/root/store.go similarity index 99% rename from store/v2/flat/store.go rename to store/v2/root/store.go index 64ba923080ff..a177f43833f8 100644 --- a/store/v2/flat/store.go +++ b/store/v2/root/store.go @@ -1,4 +1,4 @@ -package flat +package root import ( "crypto/sha256" diff --git a/store/v2/flat/store_test.go b/store/v2/root/store_test.go similarity index 99% rename from store/v2/flat/store_test.go rename to store/v2/root/store_test.go index 1fda4c52a55c..e70974c3f988 100644 --- a/store/v2/flat/store_test.go +++ b/store/v2/root/store_test.go @@ -1,4 +1,4 @@ -package flat +package root import ( "errors" diff --git a/store/v2/flat/store_view.go b/store/v2/root/store_view.go similarity index 99% rename from store/v2/flat/store_view.go rename to store/v2/root/store_view.go index bcc5e08be398..a5ed1cded9b6 100644 --- a/store/v2/flat/store_view.go +++ b/store/v2/root/store_view.go @@ -1,4 +1,4 @@ -package flat +package root import ( "errors" From 46bf6a3e9b0860429e87d829043282b1e23e91c4 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Fri, 12 Nov 2021 17:17:46 +0800 Subject: [PATCH 09/47] update docs --- docs/core/store.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/core/store.md b/docs/core/store.md index afded48f76a2..34fcb57f4e52 100644 --- a/docs/core/store.md +++ b/docs/core/store.md @@ -240,13 +240,13 @@ The SDK is in the process of transitioning to use the types listed here as the d An interface providing only the basic CRUD functionality (`Get`, `Set`, `Has`, and `Delete` methods), without iteration or caching. This is used to partially expose components of a larger store, such as a `flat.Store`. -### Flat Store +### Root Store -`flat.Store` is the new default persistent store, which internally decouples the concerns of state storage and commitment scheme. Values are stored directly in the backing key-value database (the "storage" bucket), while the value's hash is mapped in a separate store which is able to generate a cryptographic commitment (the "state commitment" bucket, implmented with `smt.Store`). +`root.RootStore` is the new default persistent store, which internally decouples the concerns of state storage and state commitment. Values are stored directly in the backing key-value database (the "storage" bucket), while the value's hash is mapped in a separate store which is able to generate a cryptographic commitment (the "state commitment" bucket, implmented with `smt.Store`). -This can optionally be constructed to use different backend databases for each bucket. +This can optionally be configured to use different backend databases for each bucket. - + ### SMT Store From 65105e70bf4ba4be4e6ef526ce8ce0afba716dc6 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Tue, 16 Nov 2021 19:19:51 +0800 Subject: [PATCH 10/47] rm StoreTypeDecoupled --- store/types/store.go | 6 +----- store/v2/types.go | 3 +-- 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/store/types/store.go b/store/types/store.go index 7af13914bc89..19d7f258617c 100644 --- a/store/types/store.go +++ b/store/types/store.go @@ -297,8 +297,7 @@ const ( StoreTypeTransient StoreTypeMemory StoreTypeSMT - StoreTypeDecoupled - StoreTypePersistent = StoreTypeDecoupled + StoreTypePersistent ) func (st StoreType) String() string { @@ -320,9 +319,6 @@ func (st StoreType) String() string { case StoreTypeSMT: return "StoreTypeSMT" - - case StoreTypeDecoupled: - return "StoreTypeDecoupled" } return "unknown store type" diff --git a/store/v2/types.go b/store/v2/types.go index 6762eed69cfc..f47376ba5cc4 100644 --- a/store/v2/types.go +++ b/store/v2/types.go @@ -85,10 +85,9 @@ type StoreType = v1.StoreType // Valid types const StoreTypeMemory = v1.StoreTypeMemory const StoreTypeTransient = v1.StoreTypeTransient -const StoreTypeDecoupled = v1.StoreTypeDecoupled const StoreTypeDB = v1.StoreTypeDB const StoreTypeSMT = v1.StoreTypeSMT -const StoreTypePersistent = StoreTypeDecoupled +const StoreTypePersistent = v1.StoreTypePersistent var NewKVStoreKey = v1.NewKVStoreKey var PrefixEndBytes = v1.PrefixEndBytes From 77effa497ae2c91bd0ff27e564e7122e21557f12 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 17 Nov 2021 20:29:46 +0800 Subject: [PATCH 11/47] smt.Store - raise smt.InvalidKeyError --- store/v2/smt/store.go | 26 +++++++++++++++++++++----- store/v2/smt/store_test.go | 12 +++++++----- 2 files changed, 28 insertions(+), 10 deletions(-) diff --git a/store/v2/smt/store.go b/store/v2/smt/store.go index ce4130174337..953a08b297ed 100644 --- a/store/v2/smt/store.go +++ b/store/v2/smt/store.go @@ -4,14 +4,16 @@ import ( "crypto/sha256" "errors" + dbm "github.com/cosmos/cosmos-sdk/db" "github.com/cosmos/cosmos-sdk/store/types" - tmcrypto "github.com/tendermint/tendermint/proto/tendermint/crypto" "github.com/lazyledger/smt" + tmcrypto "github.com/tendermint/tendermint/proto/tendermint/crypto" ) var ( _ types.BasicKVStore = (*Store)(nil) + _ smt.MapStore = (dbMapStore{}) ) var ( @@ -24,15 +26,18 @@ type Store struct { tree *smt.SparseMerkleTree } -func NewStore(nodes, values smt.MapStore) *Store { +// MapStore that wraps Get to raise InvalidKeyError +type dbMapStore struct{ dbm.DBReadWriter } + +func NewStore(nodes, values dbm.DBReadWriter) *Store { return &Store{ - tree: smt.NewSparseMerkleTree(nodes, values, sha256.New()), + tree: smt.NewSparseMerkleTree(dbMapStore{nodes}, dbMapStore{values}, sha256.New()), } } -func LoadStore(nodes, values smt.MapStore, root []byte) *Store { +func LoadStore(nodes, values dbm.DBReadWriter, root []byte) *Store { return &Store{ - tree: smt.ImportSparseMerkleTree(nodes, values, sha256.New(), root), + tree: smt.ImportSparseMerkleTree(dbMapStore{nodes}, dbMapStore{values}, sha256.New(), root), } } @@ -97,3 +102,14 @@ func (s *Store) Delete(key []byte) { panic(err) } } + +func (ms dbMapStore) Get(key []byte) ([]byte, error) { + val, err := ms.DBReadWriter.Get(key) + if err != nil { + return nil, err + } + if val == nil { + return nil, &smt.InvalidKeyError{key} + } + return val, nil +} diff --git a/store/v2/smt/store_test.go b/store/v2/smt/store_test.go index 1bb18fb13fd7..9a7c9192a072 100644 --- a/store/v2/smt/store_test.go +++ b/store/v2/smt/store_test.go @@ -5,12 +5,13 @@ import ( "github.com/stretchr/testify/assert" + "github.com/cosmos/cosmos-sdk/db/memdb" store "github.com/cosmos/cosmos-sdk/store/v2/smt" - "github.com/lazyledger/smt" ) func TestGetSetHasDelete(t *testing.T) { - s := store.NewStore(smt.NewSimpleMap(), smt.NewSimpleMap()) + nodes, values := memdb.NewDB(), memdb.NewDB() + s := store.NewStore(nodes.ReadWriter(), values.ReadWriter()) s.Set([]byte("foo"), []byte("bar")) assert.Equal(t, []byte("bar"), s.Get([]byte("foo"))) @@ -28,15 +29,16 @@ func TestGetSetHasDelete(t *testing.T) { } func TestLoadStore(t *testing.T) { - nodes, values := smt.NewSimpleMap(), smt.NewSimpleMap() - s := store.NewStore(nodes, values) + nodes, values := memdb.NewDB(), memdb.NewDB() + nmap, vmap := nodes.ReadWriter(), values.ReadWriter() + s := store.NewStore(nmap, vmap) s.Set([]byte{0}, []byte{0}) s.Set([]byte{1}, []byte{1}) s.Delete([]byte{1}) root := s.Root() - s = store.LoadStore(nodes, values, root) + s = store.LoadStore(nmap, vmap, root) assert.Equal(t, []byte{0}, s.Get([]byte{0})) assert.False(t, s.Has([]byte{1})) } From 2ada818764c173f9d7a3ecf1cbc8e29160f0cf04 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Fri, 12 Nov 2021 19:38:34 +0800 Subject: [PATCH 12/47] Combine root/store & root/root_store --- internal/db/iterator_adapter.go | 2 +- store/prefix/store.go | 16 +- store/v2/root/kv_store.go | 136 +++++++ store/v2/root/root_store.go | 637 ------------------------------- store/v2/root/root_store_test.go | 242 ------------ store/v2/root/store.go | 619 +++++++++++++++++++++++------- store/v2/root/store_test.go | 451 ++++++++++++++++------ store/v2/root/store_view.go | 130 ------- store/v2/root/test_util.go | 40 ++ store/v2/root/view_store.go | 155 ++++++++ store/v2/types.go | 2 + 11 files changed, 1158 insertions(+), 1272 deletions(-) create mode 100644 store/v2/root/kv_store.go delete mode 100644 store/v2/root/root_store.go delete mode 100644 store/v2/root/root_store_test.go delete mode 100644 store/v2/root/store_view.go create mode 100644 store/v2/root/test_util.go create mode 100644 store/v2/root/view_store.go diff --git a/internal/db/iterator_adapter.go b/internal/db/iterator_adapter.go index e7ac348b9223..3ce99e12f62b 100644 --- a/internal/db/iterator_adapter.go +++ b/internal/db/iterator_adapter.go @@ -13,7 +13,7 @@ type dbAsStoreIter struct { } // DBToStoreIterator returns an iterator wrapping the given iterator so that it satisfies the -// store/types.Iterator interface. +// (store/types).Iterator interface. func DBToStoreIterator(source dbm.Iterator) *dbAsStoreIter { ret := &dbAsStoreIter{Iterator: source} ret.Next() diff --git a/store/prefix/store.go b/store/prefix/store.go index 295278a0a853..941c920f48c7 100644 --- a/store/prefix/store.go +++ b/store/prefix/store.go @@ -36,8 +36,8 @@ func cloneAppend(bz []byte, tail []byte) (res []byte) { } func (s Store) key(key []byte) (res []byte) { - if key == nil { - panic("nil key on Store") + if len(key) == 0 { + panic("nil or empty key on Store") } res = cloneAppend(s.prefix, key) return @@ -89,6 +89,12 @@ func (s Store) Delete(key []byte) { // Implements KVStore // Check https://github.com/tendermint/tendermint/blob/master/libs/db/prefix_db.go#L106 func (s Store) Iterator(start, end []byte) types.Iterator { + if start != nil && len(start) == 0 { + panic("empty start key") + } + if end != nil && len(end) == 0 { + panic("empty end key") + } newstart := cloneAppend(s.prefix, start) var newend []byte @@ -106,6 +112,12 @@ func (s Store) Iterator(start, end []byte) types.Iterator { // ReverseIterator implements KVStore // Check https://github.com/tendermint/tendermint/blob/master/libs/db/prefix_db.go#L129 func (s Store) ReverseIterator(start, end []byte) types.Iterator { + if start != nil && len(start) == 0 { + panic("empty start key") + } + if end != nil && len(end) == 0 { + panic("empty end key") + } newstart := cloneAppend(s.prefix, start) var newend []byte diff --git a/store/v2/root/kv_store.go b/store/v2/root/kv_store.go new file mode 100644 index 000000000000..e1b817c4eb77 --- /dev/null +++ b/store/v2/root/kv_store.go @@ -0,0 +1,136 @@ +package root + +import ( + "crypto/sha256" + "io" + + dbm "github.com/cosmos/cosmos-sdk/db" + "github.com/cosmos/cosmos-sdk/db/prefix" + "github.com/cosmos/cosmos-sdk/store/cachekv" + "github.com/cosmos/cosmos-sdk/store/listenkv" + "github.com/cosmos/cosmos-sdk/store/tracekv" + "github.com/cosmos/cosmos-sdk/store/types" + "github.com/cosmos/cosmos-sdk/store/v2/smt" +) + +var ( + _ types.KVStore = (*Store)(nil) +) + +// Store is a CommitKVStore which handles state storage and commitments as separate concerns, +// optionally using separate backing key-value DBs for each. +// Allows synchronized R/W access by locking. + +// var DefaultStoreConfig = StoreConfig{Pruning: types.PruneDefault, StateCommitmentDB: nil} + +// NewStore creates a new Store, or loads one if the DB contains existing data. + +// Get implements KVStore. +func (s *Store) Get(key []byte) []byte { + s.mtx.RLock() + defer s.mtx.RUnlock() + + val, err := s.dataBucket.Get(key) + if err != nil { + panic(err) + } + return val +} + +// Has implements KVStore. +func (s *Store) Has(key []byte) bool { + s.mtx.RLock() + defer s.mtx.RUnlock() + + has, err := s.dataBucket.Has(key) + if err != nil { + panic(err) + } + return has +} + +// Set implements KVStore. +func (s *Store) Set(key, value []byte) { + s.mtx.Lock() + defer s.mtx.Unlock() + + err := s.dataBucket.Set(key, value) + if err != nil { + panic(err) + } + s.stateCommitmentStore.Set(key, value) + khash := sha256.Sum256(key) + err = s.indexBucket.Set(khash[:], key) + if err != nil { + panic(err) + } +} + +// Delete implements KVStore. +func (s *Store) Delete(key []byte) { + khash := sha256.Sum256(key) + s.mtx.Lock() + defer s.mtx.Unlock() + + s.stateCommitmentStore.Delete(key) + _ = s.indexBucket.Delete(khash[:]) + _ = s.dataBucket.Delete(key) +} + +type contentsIterator struct { + dbm.Iterator + valid bool +} + +func newIterator(source dbm.Iterator) *contentsIterator { + ret := &contentsIterator{Iterator: source} + ret.Next() + return ret +} + +func (it *contentsIterator) Next() { it.valid = it.Iterator.Next() } +func (it *contentsIterator) Valid() bool { return it.valid } + +// Iterator implements KVStore. +func (s *Store) Iterator(start, end []byte) types.Iterator { + iter, err := s.dataBucket.Iterator(start, end) + if err != nil { + panic(err) + } + return newIterator(iter) +} + +// ReverseIterator implements KVStore. +func (s *Store) ReverseIterator(start, end []byte) types.Iterator { + iter, err := s.dataBucket.ReverseIterator(start, end) + if err != nil { + panic(err) + } + return newIterator(iter) +} + +// GetStoreType implements Store. +func (s *Store) GetStoreType() types.StoreType { + return types.StoreTypePersistent +} + +func (s *Store) GetPruning() types.PruningOptions { return s.Pruning } +func (s *Store) SetPruning(po types.PruningOptions) { s.Pruning = po } + +func loadSMT(stateCommitmentTxn dbm.DBReadWriter, root []byte) *smt.Store { + merkleNodes := prefix.NewPrefixReadWriter(stateCommitmentTxn, merkleNodePrefix) + merkleValues := prefix.NewPrefixReadWriter(stateCommitmentTxn, merkleValuePrefix) + return smt.LoadStore(merkleNodes, merkleValues, root) +} + +func (s *Store) CacheWrap() types.CacheWrap { + return cachekv.NewStore(s) +} + +func (s *Store) CacheWrapWithTrace(w io.Writer, tc types.TraceContext) types.CacheWrap { + return cachekv.NewStore(tracekv.NewStore(s, w, tc)) +} + +func (s *Store) CacheWrapWithListeners(storeKey types.StoreKey, listeners []types.WriteListener) types.CacheWrap { + return cachekv.NewStore(listenkv.NewStore(s, storeKey, listeners)) +} diff --git a/store/v2/root/root_store.go b/store/v2/root/root_store.go deleted file mode 100644 index 2a88374b5442..000000000000 --- a/store/v2/root/root_store.go +++ /dev/null @@ -1,637 +0,0 @@ -package root - -import ( - "errors" - "fmt" - "io" - "strings" - - abci "github.com/tendermint/tendermint/abci/types" - - dbm "github.com/cosmos/cosmos-sdk/db" - "github.com/cosmos/cosmos-sdk/db/memdb" - prefixdb "github.com/cosmos/cosmos-sdk/db/prefix" - util "github.com/cosmos/cosmos-sdk/internal" - "github.com/cosmos/cosmos-sdk/store/cachekv" - "github.com/cosmos/cosmos-sdk/store/prefix" - types "github.com/cosmos/cosmos-sdk/store/v2" - "github.com/cosmos/cosmos-sdk/store/v2/mem" - transkv "github.com/cosmos/cosmos-sdk/store/v2/transient" - sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" - "github.com/cosmos/cosmos-sdk/types/kv" -) - -var ( - _ types.CommitRootStore = (*rootStore)(nil) - _ types.CacheRootStore = (*rootCache)(nil) - _ types.BasicRootStore = (*rootView)(nil) -) - -var ( - schemaPrefix = []byte{5} // Prefix for store keys (prefixes) -) - -// RootStoreConfig is used to define a schema and pass options to the RootStore constructor. -type RootStoreConfig struct { - StoreConfig - PersistentCache types.RootStorePersistentCache - Upgrades []types.StoreUpgrades - prefixRegistry - *listenerMixin - *traceMixin -} - -// Represents the valid store types for a RootStore schema, a subset of the StoreType values -type subStoreType byte - -const ( - subStorePersistent subStoreType = iota - subStoreMemory - subStoreTransient -) - -// A loaded mapping of store names to types -type storeSchema map[string]subStoreType - -// Builder type used to create a valid schema with no prefix conflicts -type prefixRegistry struct { - storeSchema - reserved []string -} - -// Mixin types that will be composed into each distinct root store variant type -type listenerMixin struct { - listeners map[types.StoreKey][]types.WriteListener -} - -type traceMixin struct { - TraceWriter io.Writer - TraceContext types.TraceContext -} - -type storeMixin struct { - schema storeSchema - *listenerMixin - *traceMixin -} - -// Main persistent store type -type rootStore struct { - *Store - mem *mem.Store - tran *transkv.Store - storeMixin -} - -// Branched state -type rootCache struct { - types.CacheKVStore - mem, tran types.CacheKVStore - storeMixin -} - -// Read-only store for querying -type rootView struct { - *storeView - schema storeSchema - // storeMixin //? -} - -// Auxiliary type used only to avoid repetitive method implementations -type rootGeneric struct { - schema storeSchema - persist, mem, tran types.KVStore -} - -// DefaultRootStoreConfig returns a RootStore config with an empty schema, a single backing DB, -// pruning with PruneDefault, no listeners and no tracer. -func DefaultRootStoreConfig() RootStoreConfig { - return RootStoreConfig{ - StoreConfig: StoreConfig{Pruning: types.PruneDefault}, - prefixRegistry: prefixRegistry{ - storeSchema: storeSchema{}, - }, - listenerMixin: &listenerMixin{ - listeners: map[types.StoreKey][]types.WriteListener{}, - }, - traceMixin: &traceMixin{ - TraceWriter: nil, - TraceContext: nil, - }, - } -} - -func validSubStoreType(sst subStoreType) bool { - return byte(sst) <= byte(subStoreTransient) -} - -// Returns true iff both schema maps match exactly (including mem/tran stores) -func (this storeSchema) equal(that storeSchema) bool { - if len(this) != len(that) { - return false - } - for key, val := range that { - myval, has := this[key] - if !has { - return false - } - if val != myval { - return false - } - } - return true -} - -// Parses a schema from the DB -func readSchema(bucket dbm.DBReader) (*prefixRegistry, error) { - ret := prefixRegistry{storeSchema: storeSchema{}} - it, err := bucket.Iterator(nil, nil) - if err != nil { - return nil, err - } - for it.Next() { - value := it.Value() - if len(value) != 1 || !validSubStoreType(subStoreType(value[0])) { - return nil, fmt.Errorf("invalid mapping for store key: %v => %v", it.Key(), value) - } - ret.storeSchema[string(it.Key())] = subStoreType(value[0]) - ret.reserved = append(ret.reserved, string(it.Key())) // assume iter yields keys sorted - } - it.Close() - return &ret, nil -} - -// NewRootStore constructs a RootStore directly from a DB connection and options. -func NewRootStore(db dbm.DBConnection, opts RootStoreConfig) (*rootStore, error) { - base, err := NewStore(db, opts.StoreConfig) - if err != nil { - return nil, err - } - return makeRootStore(base, opts) -} - -// TODO: -// should config contain the pre- or post-migration schema? - currently pre -func makeRootStore(base *Store, opts RootStoreConfig) (ret *rootStore, err error) { - schemaView := prefixdb.NewPrefixReader(base.stateDB.Reader(), schemaPrefix) - defer func() { - if err != nil { - err = util.CombineErrors(err, schemaView.Discard(), "schemaView.Discard also failed") - err = util.CombineErrors(err, base.Close(), "base.Close also failed") - } - }() - pr, err := readSchema(schemaView) - if err != nil { - return - } - // If the loaded schema is empty, just copy the config schema; - // Otherwise, verify it is identical to the config schema - if len(pr.storeSchema) == 0 { - for k, v := range opts.storeSchema { - pr.storeSchema[k] = v - } - pr.reserved = make([]string, len(opts.reserved)) - copy(pr.reserved, opts.reserved) - } else { - if !pr.equal(opts.storeSchema) { - err = errors.New("loaded schema does not match configured schema") - return - } - } - // Apply migrations, then clear old schema and write the new one - for _, upgrades := range opts.Upgrades { - err = pr.migrate(base, upgrades) - if err != nil { - return - } - } - schemaWriter := prefixdb.NewPrefixWriter(base.stateTxn, schemaPrefix) - it, err := schemaView.Iterator(nil, nil) - if err != nil { - return - } - for it.Next() { - err = schemaWriter.Delete(it.Key()) - if err != nil { - return - } - } - err = it.Close() - if err != nil { - return - } - err = schemaView.Discard() - if err != nil { - return - } - for skey, typ := range pr.storeSchema { - err = schemaWriter.Set([]byte(skey), []byte{byte(typ)}) - if err != nil { - return - } - } - // The migrated contents and schema are not committed until the next store.Commit - ret = &rootStore{ - Store: base, - mem: mem.NewStore(memdb.NewDB()), - tran: transkv.NewStore(memdb.NewDB()), - storeMixin: storeMixin{ - schema: pr.storeSchema, - listenerMixin: opts.listenerMixin, - traceMixin: opts.traceMixin, - }, - } - return -} - -// Applies store upgrades to the DB contents. -func (pr *prefixRegistry) migrate(store *Store, upgrades types.StoreUpgrades) error { - // branch state to allow mutation while iterating - branch := cachekv.NewStore(store) - - for _, key := range upgrades.Deleted { - sst, ix, err := pr.storeInfo(key) - if err != nil { - return err - } - if sst != subStorePersistent { - return fmt.Errorf("prefix is for non-persistent substore: %v (%v)", key, sst) - } - pr.reserved = append(pr.reserved[:ix], pr.reserved[ix+1:]...) - delete(pr.storeSchema, key) - - sub := prefix.NewStore(store, []byte(key)) - subbranch := prefix.NewStore(branch, []byte(key)) - it := sub.Iterator(nil, nil) - for ; it.Valid(); it.Next() { - subbranch.Delete(it.Key()) - } - it.Close() - } - for _, rename := range upgrades.Renamed { - sst, ix, err := pr.storeInfo(rename.OldKey) - if err != nil { - return err - } - if sst != subStorePersistent { - return fmt.Errorf("prefix is for non-persistent substore: %v (%v)", rename.OldKey, sst) - } - pr.reserved = append(pr.reserved[:ix], pr.reserved[ix+1:]...) - delete(pr.storeSchema, rename.OldKey) - err = pr.ReservePrefix(types.NewKVStoreKey(rename.NewKey), types.StoreTypePersistent) - if err != nil { - return err - } - - sub := prefix.NewStore(store, []byte(rename.OldKey)) - subbranch := prefix.NewStore(branch, []byte(rename.NewKey)) - it := sub.Iterator(nil, nil) - for ; it.Valid(); it.Next() { - subbranch.Set(it.Key(), it.Value()) - } - it.Close() - } - branch.Write() - - for _, key := range upgrades.Added { - err := pr.ReservePrefix(types.NewKVStoreKey(key), types.StoreTypePersistent) - if err != nil { - return err - } - } - return nil -} - -func (rs *rootStore) GetKVStore(key types.StoreKey) types.KVStore { - return rs.generic().getStore(key.Name()) -} - -func (rs *rootStore) Commit() types.CommitID { - id := rs.Store.Commit() - rs.tran.Commit() - return id -} - -func (rs *rootStore) Close() error { return rs.Store.Close() } - -func (rs *Store) SetInitialVersion(version uint64) error { - rs.opts.InitialVersion = uint64(version) - return nil -} - -func (rs *rootStore) GetVersion(version int64) (types.BasicRootStore, error) { - return rs.getView(version) -} - -func (rs *rootStore) getView(version int64) (*rootView, error) { - view, err := rs.Store.GetVersion(version) - if err != nil { - return nil, err - } - return rs.makeRootView(view) -} - -func (rs *rootStore) makeRootView(view *storeView) (ret *rootView, err error) { - schemaView := prefixdb.NewPrefixReader(view.stateView, schemaPrefix) - defer func() { - if err != nil { - err = util.CombineErrors(err, schemaView.Discard(), "schemaView.Discard also failed") - } - }() - pr, err := readSchema(schemaView) - if err != nil { - return - } - // The migrated contents and schema are not committed until the next store.Commit - return &rootView{ - storeView: view, - schema: pr.storeSchema, - }, nil -} - -// if the schema indicates a mem/tran store, it's ignored -func (rv *rootView) generic() rootGeneric { return rootGeneric{rv.schema, rv, nil, nil} } - -func (rv *rootView) GetKVStore(key types.StoreKey) types.KVStore { - return rv.generic().getStore(key.Name()) -} - -// Copies only the schema -func newStoreMixin(schema storeSchema) storeMixin { - return storeMixin{ - schema: schema, - listenerMixin: &listenerMixin{}, - traceMixin: &traceMixin{}, - } -} - -func (rv *rootView) CacheRootStore() types.CacheRootStore { - return &rootCache{ - CacheKVStore: cachekv.NewStore(rv), - mem: cachekv.NewStore(mem.NewStore(memdb.NewDB())), - tran: cachekv.NewStore(transkv.NewStore(memdb.NewDB())), - storeMixin: newStoreMixin(rv.schema), - } -} - -func (rs *rootStore) CacheRootStore() types.CacheRootStore { - return &rootCache{ - CacheKVStore: cachekv.NewStore(rs), - mem: cachekv.NewStore(rs.mem), - tran: cachekv.NewStore(rs.tran), - storeMixin: newStoreMixin(rs.schema), - } -} - -// parsePath expects a format like /[/] -// Must start with /, subpath may be empty -// Returns error if it doesn't start with / -func parsePath(path string) (storeName string, subpath string, err error) { - if !strings.HasPrefix(path, "/") { - return storeName, subpath, sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "invalid path: %s", path) - } - - paths := strings.SplitN(path[1:], "/", 2) - storeName = paths[0] - - if len(paths) == 2 { - subpath = "/" + paths[1] - } - - return storeName, subpath, nil -} - -// Query implements ABCI interface, allows queries. -// -// by default we will return from (latest height -1), -// as we will have merkle proofs immediately (header height = data height + 1) -// If latest-1 is not present, use latest (which must be present) -// if you care to have the latest data to see a tx results, you must -// explicitly set the height you want to see -func (rs *rootStore) Query(req abci.RequestQuery) (res abci.ResponseQuery) { - if len(req.Data) == 0 { - return sdkerrors.QueryResult(sdkerrors.Wrap(sdkerrors.ErrTxDecode, "query cannot be zero length"), false) - } - - // if height is 0, use the latest height - height := req.Height - if height == 0 { - versions, err := rs.stateDB.Versions() - if err != nil { - return sdkerrors.QueryResult(errors.New("failed to get version info"), false) - } - latest := versions.Last() - if versions.Exists(latest - 1) { - height = int64(latest - 1) - } else { - height = int64(latest) - } - } - if height < 0 { - return sdkerrors.QueryResult(fmt.Errorf("height overflow: %v", height), false) - } - res.Height = height - - storeName, subpath, err := parsePath(req.Path) - if err != nil { - return sdkerrors.QueryResult(err, false) - } - - // trim the path and make the query - // req.Path = subpath - // res := rs.Store.Query(req) - - switch subpath { - case "/key": - var err error - res.Key = req.Data // data holds the key bytes - - view, err := rs.getView(height) - if err != nil { - if errors.Is(err, dbm.ErrVersionDoesNotExist) { - err = sdkerrors.ErrInvalidHeight - } - return sdkerrors.QueryResult(err, false) - } - - substore := view.generic().getStore(storeName) - if substore == nil { - return sdkerrors.QueryResult(sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "no such store: %s", storeName), false) - } - res.Value = substore.Get(res.Key) - if !req.Prove { - break - } - // res.ProofOps, err = view.prove(storeName, res.Key) - fullkey := storeName + string(res.Key) - res.ProofOps, err = view.stateCommitmentStore.GetProof([]byte(fullkey)) - if err != nil { - return sdkerrors.QueryResult(fmt.Errorf("Merkle proof creation failed for key: %v", res.Key), false) - } - - case "/subspace": - pairs := kv.Pairs{ - Pairs: make([]kv.Pair, 0), - } - - subspace := req.Data - res.Key = subspace - - iterator := rs.Iterator(subspace, types.PrefixEndBytes(subspace)) - for ; iterator.Valid(); iterator.Next() { - pairs.Pairs = append(pairs.Pairs, kv.Pair{Key: iterator.Key(), Value: iterator.Value()}) - } - iterator.Close() - - bz, err := pairs.Marshal() - if err != nil { - panic(fmt.Errorf("failed to marshal KV pairs: %w", err)) - } - - res.Value = bz - - default: - return sdkerrors.QueryResult(sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "unexpected query path: %v", req.Path), false) - } - - return res -} - -func (rs *rootStore) generic() rootGeneric { return rootGeneric{rs.schema, rs, rs.mem, rs.tran} } - -func (store rootGeneric) getStore(key string) types.KVStore { - var sub types.KVStore - if typ, ok := store.schema[key]; ok { - switch typ { - case subStorePersistent: - sub = store.persist - case subStoreMemory: - sub = store.mem - case subStoreTransient: - sub = store.tran - } - } - if sub == nil { - panic(fmt.Sprintf("store does not exist for key: %s", key)) - } - return prefix.NewStore(sub, []byte(key)) -} - -func (rs *rootCache) GetKVStore(key types.StoreKey) types.KVStore { - return rs.generic().getStore(key.Name()) -} - -func (rs *rootCache) Write() { - rs.CacheKVStore.Write() - rs.mem.Write() - rs.tran.Write() -} - -// Recursively wraps the CacheRootStore in another cache store. -func (rs *rootCache) CacheRootStore() types.CacheRootStore { - return &rootCache{ - CacheKVStore: cachekv.NewStore(rs), - mem: cachekv.NewStore(rs.mem), - tran: cachekv.NewStore(rs.tran), - storeMixin: newStoreMixin(rs.schema), - } -} - -func (rs *rootCache) generic() rootGeneric { return rootGeneric{rs.schema, rs, rs.mem, rs.tran} } - -// Returns closest index and whether it's a match -func binarySearch(hay []string, ndl string) (int, bool) { - var mid int - from, to := 0, len(hay)-1 - for from <= to { - mid = (from + to) / 2 - switch strings.Compare(hay[mid], ndl) { - case -1: - from = mid + 1 - case 1: - to = mid - 1 - default: - return mid, true - } - } - return from, false -} - -func (pr *prefixRegistry) storeInfo(key string) (sst subStoreType, ix int, err error) { - ix, has := binarySearch(pr.reserved, key) - if !has { - err = fmt.Errorf("prefix does not exist: %v", key) - return - } - sst, has = pr.storeSchema[key] - if !has { - err = fmt.Errorf("prefix is registered but not in schema: %v", key) - } - - return -} - -func (pr *prefixRegistry) ReservePrefix(key types.StoreKey, typ types.StoreType) error { - // Find the neighboring reserved prefix, and check for duplicates and conflicts - i, has := binarySearch(pr.reserved, key.Name()) - if has { - return fmt.Errorf("prefix already exists: %v", key) - } - if i > 0 && strings.HasPrefix(key.Name(), pr.reserved[i-1]) { - return fmt.Errorf("prefix conflict: '%v' exists, cannot add '%v'", pr.reserved[i-1], key.Name()) - } - if i < len(pr.reserved) && strings.HasPrefix(pr.reserved[i], key.Name()) { - return fmt.Errorf("prefix conflict: '%v' exists, cannot add '%v'", pr.reserved[i], key.Name()) - } - reserved := pr.reserved[:i] - reserved = append(reserved, key.Name()) - pr.reserved = append(reserved, pr.reserved[i:]...) - - var sstype subStoreType - switch typ { - case types.StoreTypeDecoupled: - sstype = subStorePersistent - case types.StoreTypeMemory: - sstype = subStoreMemory - case types.StoreTypeTransient: - sstype = subStoreTransient - // case types.StoreTypeSMT: // could be used for external storage - default: - return fmt.Errorf("StoreType not supported: %v", typ) - } - pr.storeSchema[key.Name()] = sstype - return nil -} - -func (lreg *listenerMixin) AddListeners(key types.StoreKey, listeners []types.WriteListener) { - if ls, ok := lreg.listeners[key]; ok { - lreg.listeners[key] = append(ls, listeners...) - } else { - lreg.listeners[key] = listeners - } -} - -// ListeningEnabled returns if listening is enabled for a specific KVStore -func (lreg *listenerMixin) ListeningEnabled(key types.StoreKey) bool { - if ls, ok := lreg.listeners[key]; ok { - return len(ls) != 0 - } - return false -} - -func (treg *traceMixin) TracingEnabled() bool { - return treg.TraceWriter != nil -} -func (treg *traceMixin) SetTracer(w io.Writer) { - treg.TraceWriter = w -} -func (treg *traceMixin) SetTraceContext(tc types.TraceContext) { - treg.TraceContext = tc -} - -func (rs *rootStore) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser, ready chan<- struct{}) error { - return nil -} -func (rs *rootStore) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) { - return nil, nil -} diff --git a/store/v2/root/root_store_test.go b/store/v2/root/root_store_test.go deleted file mode 100644 index bc10e7e36b86..000000000000 --- a/store/v2/root/root_store_test.go +++ /dev/null @@ -1,242 +0,0 @@ -package root - -import ( - "testing" - - "github.com/stretchr/testify/require" - - "github.com/cosmos/cosmos-sdk/db/memdb" - types "github.com/cosmos/cosmos-sdk/store/v2" -) - -var ( - skey_1 = types.NewKVStoreKey("store1") - skey_2 = types.NewKVStoreKey("store2") - skey_3 = types.NewKVStoreKey("store3") - skey_1b = types.NewKVStoreKey("store1b") - skey_2b = types.NewKVStoreKey("store2b") - skey_3b = types.NewKVStoreKey("store3b") -) - -func storeConfig123(t *testing.T) RootStoreConfig { - opts := DefaultRootStoreConfig() - opts.Pruning = types.PruneNothing - err := opts.ReservePrefix(skey_1, types.StoreTypePersistent) - require.NoError(t, err) - err = opts.ReservePrefix(skey_2, types.StoreTypePersistent) - require.NoError(t, err) - err = opts.ReservePrefix(skey_3, types.StoreTypePersistent) - require.NoError(t, err) - return opts -} - -func TestRootStoreConfig(t *testing.T) { - opts := DefaultRootStoreConfig() - // Ensure that no prefix conflicts are allowed - err := opts.ReservePrefix(skey_1, types.StoreTypePersistent) - require.NoError(t, err) - err = opts.ReservePrefix(skey_2, types.StoreTypePersistent) - require.NoError(t, err) - err = opts.ReservePrefix(skey_3b, types.StoreTypePersistent) - require.NoError(t, err) - err = opts.ReservePrefix(skey_1b, types.StoreTypePersistent) - require.Error(t, err) - err = opts.ReservePrefix(skey_2b, types.StoreTypePersistent) - require.Error(t, err) - err = opts.ReservePrefix(skey_3, types.StoreTypePersistent) - require.Error(t, err) -} - -func TestRootStoreBasic(t *testing.T) { - opts := DefaultRootStoreConfig() - err := opts.ReservePrefix(skey_1, types.StoreTypePersistent) - require.NoError(t, err) - db := memdb.NewDB() - store, err := NewRootStore(db, opts) - require.NoError(t, err) - - store_1 := store.GetKVStore(skey_1) - require.NotNil(t, store_1) - store_1.Set([]byte{0}, []byte{0}) - val := store_1.Get([]byte{0}) - require.Equal(t, []byte{0}, val) - store_1.Delete([]byte{0}) - val = store_1.Get([]byte{0}) - require.Equal(t, []byte(nil), val) -} - -func TestGetVersion(t *testing.T) { - db := memdb.NewDB() - opts := storeConfig123(t) - store, err := NewRootStore(db, opts) - require.NoError(t, err) - - cid := store.Commit() - // opts := DefaultRootStoreConfig() - - view, err := store.GetVersion(cid.Version) - require.NoError(t, err) - subview := view.GetKVStore(skey_1) - require.NotNil(t, subview) - - // version view should be read-only - require.Panics(t, func() { subview.Set([]byte{1}, []byte{1}) }) - require.Panics(t, func() { subview.Delete([]byte{0}) }) - // nonexistent version shouldn't be accessible - view, err = store.GetVersion(cid.Version + 1) - require.Equal(t, ErrVersionDoesNotExist, err) - - substore := store.GetKVStore(skey_1) - require.NotNil(t, substore) - substore.Set([]byte{0}, []byte{0}) - // setting a new value shouldn't affect old version - require.False(t, subview.Has([]byte{0})) - - cid = store.Commit() - view, err = store.GetVersion(cid.Version) - require.NoError(t, err) - subview = view.GetKVStore(skey_1) - require.NotNil(t, subview) - - store.Delete([]byte{0}) - require.Equal(t, []byte{0}, subview.Get([]byte{0})) -} - -func TestRootStoreMigration(t *testing.T) { - skey_2b := types.NewKVStoreKey("store2b") - skey_4 := types.NewKVStoreKey("store4") - - db := memdb.NewDB() - opts := storeConfig123(t) - store, err := NewRootStore(db, opts) - require.NoError(t, err) - - // write some data in all stores - k1, v1 := []byte("first"), []byte("store") - s1 := store.GetKVStore(skey_1) - require.NotNil(t, s1) - s1.Set(k1, v1) - - k2, v2 := []byte("second"), []byte("restore") - s2 := store.GetKVStore(skey_2) - require.NotNil(t, s2) - s2.Set(k2, v2) - - k3, v3 := []byte("third"), []byte("dropped") - s3 := store.GetKVStore(skey_3) - require.NotNil(t, s3) - s3.Set(k3, v3) - - require.Panics(t, func() { store.GetKVStore(skey_4) }) - - cid := store.Commit() - require.NoError(t, store.Close()) - - // Load without changes and make sure it is sensible - store, err = NewRootStore(db, opts) - require.NoError(t, err) - - // let's query data to see it was saved properly - s2 = store.GetKVStore(skey_2) - require.NotNil(t, s2) - require.Equal(t, v2, s2.Get(k2)) - require.NoError(t, store.Close()) - - // now, let's load with upgrades... - opts.Upgrades = []types.StoreUpgrades{ - types.StoreUpgrades{ - Added: []string{skey_4.Name()}, - Renamed: []types.StoreRename{{ - OldKey: skey_2.Name(), - NewKey: skey_2b.Name(), - }}, - Deleted: []string{skey_3.Name()}, - }, - } - restore, err := NewRootStore(db, opts) - require.Nil(t, err) - - // s1 was not changed - s1 = restore.GetKVStore(skey_1) - require.NotNil(t, s1) - require.Equal(t, v1, s1.Get(k1)) - - // store3 is gone - require.Panics(t, func() { s3 = restore.GetKVStore(skey_3) }) - - // store4 is mounted, with empty data - s4 := restore.GetKVStore(skey_4) - require.NotNil(t, s4) - - values := 0 - it := s4.Iterator(nil, nil) - for ; it.Valid(); it.Next() { - values += 1 - } - require.Zero(t, values) - require.NoError(t, it.Close()) - - // write something inside store4 - k4, v4 := []byte("fourth"), []byte("created") - s4.Set(k4, v4) - - // store2 is no longer mounted - require.Panics(t, func() { restore.GetKVStore(skey_2) }) - - // restore2 has the old data - rs2 := restore.GetKVStore(skey_2b) - require.NotNil(t, rs2) - require.Equal(t, v2, rs2.Get(k2)) - - // store this migrated data, and load it again without migrations - migratedID := restore.Commit() - require.Equal(t, migratedID.Version, int64(2)) - require.NoError(t, restore.Close()) - - // fail to load the migrated store with the old schema - reload, err := NewRootStore(db, storeConfig123(t)) - require.Error(t, err) - - // pass in a schema reflecting the migrations - migratedOpts := DefaultRootStoreConfig() - err = migratedOpts.ReservePrefix(skey_1, types.StoreTypePersistent) - require.NoError(t, err) - err = migratedOpts.ReservePrefix(skey_2b, types.StoreTypePersistent) - require.NoError(t, err) - err = migratedOpts.ReservePrefix(skey_4, types.StoreTypePersistent) - require.NoError(t, err) - reload, err = NewRootStore(db, migratedOpts) - require.Nil(t, err) - require.Equal(t, migratedID, reload.LastCommitID()) - - // query this new store - rl1 := reload.GetKVStore(skey_1) - require.NotNil(t, rl1) - require.Equal(t, v1, rl1.Get(k1)) - - rl2 := reload.GetKVStore(skey_2b) - require.NotNil(t, rl2) - require.Equal(t, v2, rl2.Get(k2)) - - rl4 := reload.GetKVStore(skey_4) - require.NotNil(t, rl4) - require.Equal(t, v4, rl4.Get(k4)) - - // load and check a view of the store at first commit - view, err := store.GetVersion(cid.Version) - require.NoError(t, err) - - s1 = view.GetKVStore(skey_1) - require.NotNil(t, s1) - require.Equal(t, v1, s1.Get(k1)) - - s2 = view.GetKVStore(skey_2) - require.NotNil(t, s2) - require.Equal(t, v2, s2.Get(k2)) - - s3 = view.GetKVStore(skey_3) - require.NotNil(t, s3) - require.Equal(t, v3, s3.Get(k3)) - - require.Panics(t, func() { view.GetKVStore(skey_4) }) -} diff --git a/store/v2/root/store.go b/store/v2/root/store.go index a177f43833f8..0d8abaa4785d 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -1,31 +1,37 @@ +// RootStore supports a subset of the StoreType values: Persistent, Memory, and Transient + package root import ( - "crypto/sha256" "errors" "fmt" "io" "math" + "strings" "sync" abci "github.com/tendermint/tendermint/abci/types" dbm "github.com/cosmos/cosmos-sdk/db" - "github.com/cosmos/cosmos-sdk/db/prefix" + "github.com/cosmos/cosmos-sdk/db/memdb" + prefixdb "github.com/cosmos/cosmos-sdk/db/prefix" util "github.com/cosmos/cosmos-sdk/internal" "github.com/cosmos/cosmos-sdk/store/cachekv" - "github.com/cosmos/cosmos-sdk/store/listenkv" - "github.com/cosmos/cosmos-sdk/store/tracekv" - "github.com/cosmos/cosmos-sdk/store/types" + "github.com/cosmos/cosmos-sdk/store/prefix" + types "github.com/cosmos/cosmos-sdk/store/v2" + "github.com/cosmos/cosmos-sdk/store/v2/mem" "github.com/cosmos/cosmos-sdk/store/v2/smt" + transkv "github.com/cosmos/cosmos-sdk/store/v2/transient" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" "github.com/cosmos/cosmos-sdk/types/kv" ) var ( - _ types.KVStore = (*Store)(nil) - _ types.CommitKVStore = (*Store)(nil) - _ types.Queryable = (*Store)(nil) + _ types.KVStore = (*Store)(nil) + _ types.Queryable = (*Store)(nil) + _ types.CommitRootStore = (*Store)(nil) + _ types.CacheRootStore = (*cacheStore)(nil) + _ types.BasicRootStore = (*viewStore)(nil) ) var ( @@ -34,6 +40,7 @@ var ( indexPrefix = []byte{2} // Prefix for Store reverse index merkleNodePrefix = []byte{3} // Prefix for Merkle tree nodes merkleValuePrefix = []byte{4} // Prefix for Merkle value mappings + schemaPrefix = []byte{5} // Prefix for store keys (namespaces) ) var ( @@ -41,6 +48,7 @@ var ( ErrMaximumHeight = errors.New("maximum block height reached") ) +// StoreConfig is used to define a schema and pass options to the RootStore constructor. type StoreConfig struct { // Version pruning options for backing DBs. Pruning types.PruningOptions @@ -48,11 +56,35 @@ type StoreConfig struct { // The backing DB to use for the state commitment Merkle tree data. // If nil, Merkle data is stored in the state storage DB under a separate prefix. StateCommitmentDB dbm.DBConnection + + prefixRegistry + PersistentCache types.RootStorePersistentCache + Upgrades []types.StoreUpgrades + + *listenerMixin + *traceMixin +} + +// A loaded mapping of substore keys to store types +type StoreSchema map[string]types.StoreType + +// Builder type used to create a valid schema with no prefix conflicts +type prefixRegistry struct { + StoreSchema + reserved []string } -// Store is a CommitKVStore which handles state storage and commitments as separate concerns, -// optionally using separate backing key-value DBs for each. -// Allows synchronized R/W access by locking. +// Mixin types that will be composed into each distinct root store variant type +type listenerMixin struct { + listeners map[types.StoreKey][]types.WriteListener +} + +type traceMixin struct { + TraceWriter io.Writer + TraceContext types.TraceContext +} + +// Main persistent store type type Store struct { stateDB dbm.DBConnection stateTxn dbm.DBReadWriter @@ -62,13 +94,116 @@ type Store struct { // State commitment (SC) KV store for current version stateCommitmentStore *smt.Store - opts StoreConfig - mtx sync.RWMutex + Pruning types.PruningOptions + InitialVersion uint64 + StateCommitmentDB dbm.DBConnection + + schema StoreSchema + mem *mem.Store + tran *transkv.Store + *listenerMixin + *traceMixin + + mtx sync.RWMutex + + PersistentCache types.RootStorePersistentCache +} + +// Branched state +type cacheStore struct { + types.CacheKVStore + mem, tran types.CacheKVStore + schema StoreSchema + *listenerMixin + *traceMixin +} + +// Read-only store for querying +type viewStore struct { + stateView dbm.DBReader + dataBucket dbm.DBReader + indexBucket dbm.DBReader + stateCommitmentView dbm.DBReader + stateCommitmentStore *smt.Store + + schema StoreSchema } -var DefaultStoreConfig = StoreConfig{Pruning: types.PruneDefault, StateCommitmentDB: nil} +// Auxiliary type used only to avoid repetitive method implementations +type rootGeneric struct { + schema StoreSchema + persist, mem, tran types.KVStore +} -// NewStore creates a new Store, or loads one if the DB contains existing data. +// DefaultStoreConfig returns a RootStore config with an empty schema, a single backing DB, +// pruning with PruneDefault, no listeners and no tracer. +func DefaultStoreConfig() StoreConfig { + return StoreConfig{ + Pruning: types.PruneDefault, + prefixRegistry: prefixRegistry{ + StoreSchema: StoreSchema{}, + }, + listenerMixin: &listenerMixin{ + listeners: map[types.StoreKey][]types.WriteListener{}, + }, + traceMixin: &traceMixin{ + TraceWriter: nil, + TraceContext: nil, + }, + } +} + +// Returns true for valid store types for a RootStore schema +func validSubStoreType(sst types.StoreType) bool { + switch sst { + case types.StoreTypePersistent: + return true + case types.StoreTypeMemory: + return true + case types.StoreTypeTransient: + return true + default: + return false + } +} + +// Returns true iff both schema maps match exactly (including mem/tran stores) +func (this StoreSchema) equal(that StoreSchema) bool { + if len(this) != len(that) { + return false + } + for key, val := range that { + myval, has := this[key] + if !has { + return false + } + if val != myval { + return false + } + } + return true +} + +// Parses a schema from the DB +func readSavedSchema(bucket dbm.DBReader) (*prefixRegistry, error) { + ret := prefixRegistry{StoreSchema: StoreSchema{}} + it, err := bucket.Iterator(nil, nil) + if err != nil { + return nil, err + } + for it.Next() { + value := it.Value() + if len(value) != 1 || !validSubStoreType(types.StoreType(value[0])) { + return nil, fmt.Errorf("invalid mapping for store key: %v => %v", it.Key(), value) + } + ret.StoreSchema[string(it.Key())] = types.StoreType(value[0]) + ret.reserved = append(ret.reserved, string(it.Key())) // assume iter yields keys sorted + } + it.Close() + return &ret, nil +} + +// NewStore constructs a RootStore directly from a DB connection and options. func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { versions, err := db.Versions() if err != nil { @@ -125,116 +260,159 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { } stateCommitmentStore = loadSMT(stateCommitmentTxn, root) } else { - merkleNodes := prefix.NewPrefixReadWriter(stateCommitmentTxn, merkleNodePrefix) - merkleValues := prefix.NewPrefixReadWriter(stateCommitmentTxn, merkleValuePrefix) + merkleNodes := prefixdb.NewPrefixReadWriter(stateCommitmentTxn, merkleNodePrefix) + merkleValues := prefixdb.NewPrefixReadWriter(stateCommitmentTxn, merkleValuePrefix) stateCommitmentStore = smt.NewStore(merkleNodes, merkleValues) } - return &Store{ + ret = &Store{ stateDB: db, stateTxn: stateTxn, - dataBucket: prefix.NewPrefixReadWriter(stateTxn, dataPrefix), - indexBucket: prefix.NewPrefixReadWriter(stateTxn, indexPrefix), + dataBucket: prefixdb.NewPrefixReadWriter(stateTxn, dataPrefix), + indexBucket: prefixdb.NewPrefixReadWriter(stateTxn, indexPrefix), stateCommitmentTxn: stateCommitmentTxn, stateCommitmentStore: stateCommitmentStore, - opts: opts, - }, nil -} -func (s *Store) Close() error { - err := s.stateTxn.Discard() - if s.opts.StateCommitmentDB != nil { - err = util.CombineErrors(err, s.stateCommitmentTxn.Discard(), "stateCommitmentTxn.Discard also failed") + Pruning: opts.Pruning, + InitialVersion: opts.InitialVersion, + StateCommitmentDB: opts.StateCommitmentDB, + PersistentCache: opts.PersistentCache, + listenerMixin: opts.listenerMixin, + traceMixin: opts.traceMixin, } - return err -} -// Get implements KVStore. -func (s *Store) Get(key []byte) []byte { - s.mtx.RLock() - defer s.mtx.RUnlock() - - val, err := s.dataBucket.Get(key) + // Now load the substore schema + schemaView := prefixdb.NewPrefixReader(ret.stateDB.Reader(), schemaPrefix) + defer func() { + if err != nil { + err = util.CombineErrors(err, schemaView.Discard(), "schemaView.Discard also failed") + err = util.CombineErrors(err, ret.Close(), "base.Close also failed") + } + }() + reg, err := readSavedSchema(schemaView) if err != nil { - panic(err) + return } - return val -} - -// Has implements KVStore. -func (s *Store) Has(key []byte) bool { - s.mtx.RLock() - defer s.mtx.RUnlock() - - has, err := s.dataBucket.Has(key) + // If the loaded schema is empty, just copy the config schema; + // Otherwise, verify it is identical to the config schema + if len(reg.StoreSchema) == 0 { + for k, v := range opts.StoreSchema { + reg.StoreSchema[k] = v + } + reg.reserved = make([]string, len(opts.reserved)) + copy(reg.reserved, opts.reserved) + } else { + if !reg.equal(opts.StoreSchema) { + err = errors.New("loaded schema does not match configured schema") + return + } + } + // Apply migrations, then clear old schema and write the new one + for _, upgrades := range opts.Upgrades { + err = reg.migrate(ret, upgrades) + if err != nil { + return + } + } + schemaWriter := prefixdb.NewPrefixWriter(ret.stateTxn, schemaPrefix) + it, err := schemaView.Iterator(nil, nil) if err != nil { - panic(err) + return } - return has -} - -// Set implements KVStore. -func (s *Store) Set(key, value []byte) { - s.mtx.Lock() - defer s.mtx.Unlock() - - err := s.dataBucket.Set(key, value) + for it.Next() { + err = schemaWriter.Delete(it.Key()) + if err != nil { + return + } + } + err = it.Close() if err != nil { - panic(err) + return } - s.stateCommitmentStore.Set(key, value) - khash := sha256.Sum256(key) - err = s.indexBucket.Set(khash[:], key) + err = schemaView.Discard() if err != nil { - panic(err) + return } + for skey, typ := range reg.StoreSchema { + err = schemaWriter.Set([]byte(skey), []byte{byte(typ)}) + if err != nil { + return + } + } + // The migrated contents and schema are not committed until the next store.Commit + ret.mem = mem.NewStore(memdb.NewDB()) + ret.tran = transkv.NewStore(memdb.NewDB()) + ret.schema = reg.StoreSchema + return } -// Delete implements KVStore. -func (s *Store) Delete(key []byte) { - khash := sha256.Sum256(key) - s.mtx.Lock() - defer s.mtx.Unlock() - - s.stateCommitmentStore.Delete(key) - _ = s.indexBucket.Delete(khash[:]) - _ = s.dataBucket.Delete(key) -} - -type contentsIterator struct { - dbm.Iterator - valid bool +func (s *Store) Close() error { + err := s.stateTxn.Discard() + if s.StateCommitmentDB != nil { + err = util.CombineErrors(err, s.stateCommitmentTxn.Discard(), "stateCommitmentTxn.Discard also failed") + } + return err } -func newIterator(source dbm.Iterator) *contentsIterator { - ret := &contentsIterator{Iterator: source} - ret.Next() - return ret -} +// Applies store upgrades to the DB contents. +func (pr *prefixRegistry) migrate(store *Store, upgrades types.StoreUpgrades) error { + // branch state to allow mutation while iterating + branch := cachekv.NewStore(store) -func (it *contentsIterator) Next() { it.valid = it.Iterator.Next() } -func (it *contentsIterator) Valid() bool { return it.valid } + for _, key := range upgrades.Deleted { + sst, ix, err := pr.storeInfo(key) + if err != nil { + return err + } + if sst != types.StoreTypePersistent { + return fmt.Errorf("prefix is for non-persistent substore: %v (%v)", key, sst) + } + pr.reserved = append(pr.reserved[:ix], pr.reserved[ix+1:]...) + delete(pr.StoreSchema, key) + + sub := prefix.NewStore(store, []byte(key)) + subbranch := prefix.NewStore(branch, []byte(key)) + it := sub.Iterator(nil, nil) + for ; it.Valid(); it.Next() { + subbranch.Delete(it.Key()) + } + it.Close() + } + for _, rename := range upgrades.Renamed { + sst, ix, err := pr.storeInfo(rename.OldKey) + if err != nil { + return err + } + if sst != types.StoreTypePersistent { + return fmt.Errorf("prefix is for non-persistent substore: %v (%v)", rename.OldKey, sst) + } + pr.reserved = append(pr.reserved[:ix], pr.reserved[ix+1:]...) + delete(pr.StoreSchema, rename.OldKey) + err = pr.ReservePrefix(rename.NewKey, types.StoreTypePersistent) + if err != nil { + return err + } -// Iterator implements KVStore. -func (s *Store) Iterator(start, end []byte) types.Iterator { - iter, err := s.dataBucket.Iterator(start, end) - if err != nil { - panic(err) + sub := prefix.NewStore(store, []byte(rename.OldKey)) + subbranch := prefix.NewStore(branch, []byte(rename.NewKey)) + it := sub.Iterator(nil, nil) + for ; it.Valid(); it.Next() { + subbranch.Set(it.Key(), it.Value()) + } + it.Close() } - return newIterator(iter) -} + branch.Write() -// ReverseIterator implements KVStore. -func (s *Store) ReverseIterator(start, end []byte) types.Iterator { - iter, err := s.dataBucket.ReverseIterator(start, end) - if err != nil { - panic(err) + for _, key := range upgrades.Added { + err := pr.ReservePrefix(key, types.StoreTypePersistent) + if err != nil { + return err + } } - return newIterator(iter) + return nil } -// GetStoreType implements Store. -func (s *Store) GetStoreType() types.StoreType { - return types.StoreTypeDecoupled +func (rs *Store) GetKVStore(key types.StoreKey) types.KVStore { + return rs.generic().getStore(key.Name()) } // Commit implements Committer. @@ -248,8 +426,8 @@ func (s *Store) Commit() types.CommitID { panic(ErrMaximumHeight) } // Fast forward to initialversion if needed - if s.opts.InitialVersion != 0 && target < s.opts.InitialVersion { - target = s.opts.InitialVersion + if s.InitialVersion != 0 && target < s.InitialVersion { + target = s.InitialVersion } cid, err := s.commit(target) if err != nil { @@ -257,19 +435,22 @@ func (s *Store) Commit() types.CommitID { } previous := cid.Version - 1 - if s.opts.Pruning.KeepEvery != 1 && s.opts.Pruning.Interval != 0 && cid.Version%int64(s.opts.Pruning.Interval) == 0 { + if s.Pruning.KeepEvery != 1 && s.Pruning.Interval != 0 && cid.Version%int64(s.Pruning.Interval) == 0 { // The range of newly prunable versions - lastPrunable := previous - int64(s.opts.Pruning.KeepRecent) - firstPrunable := lastPrunable - int64(s.opts.Pruning.Interval) + lastPrunable := previous - int64(s.Pruning.KeepRecent) + firstPrunable := lastPrunable - int64(s.Pruning.Interval) for version := firstPrunable; version <= lastPrunable; version++ { - if s.opts.Pruning.KeepEvery == 0 || version%int64(s.opts.Pruning.KeepEvery) != 0 { + if s.Pruning.KeepEvery == 0 || version%int64(s.Pruning.KeepEvery) != 0 { s.stateDB.DeleteVersion(uint64(version)) - if s.opts.StateCommitmentDB != nil { - s.opts.StateCommitmentDB.DeleteVersion(uint64(version)) + if s.StateCommitmentDB != nil { + s.StateCommitmentDB.DeleteVersion(uint64(version)) } } } } + + s.tran.Commit() + return *cid } @@ -302,7 +483,7 @@ func (s *Store) commit(target uint64) (id *types.CommitID, err error) { stateCommitmentTxn := stateTxn // If DBs are not separate, StateCommitment state has been commmitted & snapshotted - if s.opts.StateCommitmentDB != nil { + if s.StateCommitmentDB != nil { defer func() { if err != nil { if delerr := s.stateDB.DeleteVersion(target); delerr != nil { @@ -317,20 +498,20 @@ func (s *Store) commit(target uint64) (id *types.CommitID, err error) { } defer func() { if err != nil { - err = util.CombineErrors(err, s.opts.StateCommitmentDB.Revert(), "stateCommitmentDB.Revert also failed") + err = util.CombineErrors(err, s.StateCommitmentDB.Revert(), "stateCommitmentDB.Revert also failed") } }() - err = s.opts.StateCommitmentDB.SaveVersion(target) + err = s.StateCommitmentDB.SaveVersion(target) if err != nil { return } - stateCommitmentTxn = s.opts.StateCommitmentDB.ReadWriter() + stateCommitmentTxn = s.StateCommitmentDB.ReadWriter() } s.stateTxn = stateTxn - s.dataBucket = prefix.NewPrefixReadWriter(stateTxn, dataPrefix) - s.indexBucket = prefix.NewPrefixReadWriter(stateTxn, indexPrefix) + s.dataBucket = prefixdb.NewPrefixReadWriter(stateTxn, dataPrefix) + s.indexBucket = prefixdb.NewPrefixReadWriter(stateTxn, indexPrefix) s.stateCommitmentTxn = stateCommitmentTxn s.stateCommitmentStore = loadSMT(stateCommitmentTxn, root) @@ -355,8 +536,43 @@ func (s *Store) LastCommitID() types.CommitID { return types.CommitID{Version: int64(last), Hash: hash} } -func (s *Store) GetPruning() types.PruningOptions { return s.opts.Pruning } -func (s *Store) SetPruning(po types.PruningOptions) { s.opts.Pruning = po } +func (rs *Store) SetInitialVersion(version uint64) error { + rs.InitialVersion = uint64(version) + return nil +} + +func (rs *Store) GetVersion(version int64) (types.BasicRootStore, error) { + return rs.getView(version) +} + +func (rs *Store) CacheRootStore() types.CacheRootStore { + return &cacheStore{ + CacheKVStore: cachekv.NewStore(rs), + mem: cachekv.NewStore(rs.mem), + tran: cachekv.NewStore(rs.tran), + schema: rs.schema, + listenerMixin: &listenerMixin{}, + traceMixin: &traceMixin{}, + } +} + +// parsePath expects a format like /[/] +// Must start with /, subpath may be empty +// Returns error if it doesn't start with / +func parsePath(path string) (storeName string, subpath string, err error) { + if !strings.HasPrefix(path, "/") { + return storeName, subpath, sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "invalid path: %s", path) + } + + paths := strings.SplitN(path[1:], "/", 2) + storeName = paths[0] + + if len(paths) == 2 { + subpath = "/" + paths[1] + } + + return storeName, subpath, nil +} // Query implements ABCI interface, allows queries. // @@ -365,7 +581,7 @@ func (s *Store) SetPruning(po types.PruningOptions) { s.opts.Pruning = po } // If latest-1 is not present, use latest (which must be present) // if you care to have the latest data to see a tx results, you must // explicitly set the height you want to see -func (s *Store) Query(req abci.RequestQuery) (res abci.ResponseQuery) { +func (rs *Store) Query(req abci.RequestQuery) (res abci.ResponseQuery) { if len(req.Data) == 0 { return sdkerrors.QueryResult(sdkerrors.Wrap(sdkerrors.ErrTxDecode, "query cannot be zero length"), false) } @@ -373,7 +589,7 @@ func (s *Store) Query(req abci.RequestQuery) (res abci.ResponseQuery) { // if height is 0, use the latest height height := req.Height if height == 0 { - versions, err := s.stateDB.Versions() + versions, err := rs.stateDB.Versions() if err != nil { return sdkerrors.QueryResult(errors.New("failed to get version info"), false) } @@ -389,36 +605,47 @@ func (s *Store) Query(req abci.RequestQuery) (res abci.ResponseQuery) { } res.Height = height - switch req.Path { + storeName, subpath, err := parsePath(req.Path) + if err != nil { + return sdkerrors.QueryResult(err, false) + } + view, err := rs.getView(height) + if err != nil { + if errors.Is(err, dbm.ErrVersionDoesNotExist) { + err = sdkerrors.ErrInvalidHeight + } + return sdkerrors.QueryResult(err, false) + } + + substore := view.generic().getStore(storeName) + if substore == nil { + return sdkerrors.QueryResult(sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "no such store: %s", storeName), false) + } + + switch subpath { case "/key": var err error res.Key = req.Data // data holds the key bytes - - view, err := s.GetVersion(height) - if err != nil { - if errors.Is(err, dbm.ErrVersionDoesNotExist) { - err = sdkerrors.ErrInvalidHeight - } - return sdkerrors.QueryResult(err, false) - } - res.Value = view.Get(res.Key) + res.Value = substore.Get(res.Key) if !req.Prove { break } - res.ProofOps, err = view.GetStateCommitmentStore().GetProof(res.Key) + // res.ProofOps, err = view.prove(storeName, res.Key) + res.ProofOps, err = view.stateCommitmentStore.GetProof([]byte(storeName + string(res.Key))) if err != nil { return sdkerrors.QueryResult(fmt.Errorf("Merkle proof creation failed for key: %v", res.Key), false) //nolint: stylecheck // proper name } case "/subspace": + res.Key = req.Data // data holds the subspace prefix + pairs := kv.Pairs{ Pairs: make([]kv.Pair, 0), } - subspace := req.Data - res.Key = subspace + res.Key = req.Data // data holds the subspace prefix - iterator := s.Iterator(subspace, types.PrefixEndBytes(subspace)) + iterator := substore.Iterator(res.Key, types.PrefixEndBytes(res.Key)) for ; iterator.Valid(); iterator.Next() { pairs.Pairs = append(pairs.Pairs, kv.Pair{Key: iterator.Key(), Value: iterator.Value()}) } @@ -438,20 +665,134 @@ func (s *Store) Query(req abci.RequestQuery) (res abci.ResponseQuery) { return res } -func loadSMT(stateCommitmentTxn dbm.DBReadWriter, root []byte) *smt.Store { - merkleNodes := prefix.NewPrefixReadWriter(stateCommitmentTxn, merkleNodePrefix) - merkleValues := prefix.NewPrefixReadWriter(stateCommitmentTxn, merkleValuePrefix) - return smt.LoadStore(merkleNodes, merkleValues, root) +func (rs *Store) generic() rootGeneric { return rootGeneric{rs.schema, rs, rs.mem, rs.tran} } + +func (store rootGeneric) getStore(key string) types.KVStore { + var sub types.KVStore + if typ, ok := store.schema[key]; ok { + switch typ { + case types.StoreTypePersistent: + sub = store.persist + case types.StoreTypeMemory: + sub = store.mem + case types.StoreTypeTransient: + sub = store.tran + } + } + if sub == nil { + panic(fmt.Errorf("store does not exist for key: %s", key)) + } + return prefix.NewStore(sub, []byte(key)) +} + +func (rs *cacheStore) GetKVStore(key types.StoreKey) types.KVStore { + return rs.generic().getStore(key.Name()) } -func (s *Store) CacheWrap() types.CacheWrap { - return cachekv.NewStore(s) +func (rs *cacheStore) Write() { + rs.CacheKVStore.Write() + rs.mem.Write() + rs.tran.Write() } -func (s *Store) CacheWrapWithTrace(w io.Writer, tc types.TraceContext) types.CacheWrap { - return cachekv.NewStore(tracekv.NewStore(s, w, tc)) +// Recursively wraps the CacheRootStore in another cache store. +func (rs *cacheStore) CacheRootStore() types.CacheRootStore { + return &cacheStore{ + CacheKVStore: cachekv.NewStore(rs), + mem: cachekv.NewStore(rs.mem), + tran: cachekv.NewStore(rs.tran), + schema: rs.schema, + listenerMixin: &listenerMixin{}, + traceMixin: &traceMixin{}, + } } -func (s *Store) CacheWrapWithListeners(storeKey types.StoreKey, listeners []types.WriteListener) types.CacheWrap { - return cachekv.NewStore(listenkv.NewStore(s, storeKey, listeners)) +func (rs *cacheStore) generic() rootGeneric { return rootGeneric{rs.schema, rs, rs.mem, rs.tran} } + +// Returns closest index and whether it's a match +func binarySearch(hay []string, ndl string) (int, bool) { + var mid int + from, to := 0, len(hay)-1 + for from <= to { + mid = (from + to) / 2 + switch strings.Compare(hay[mid], ndl) { + case -1: + from = mid + 1 + case 1: + to = mid - 1 + default: + return mid, true + } + } + return from, false +} + +func (pr *prefixRegistry) storeInfo(key string) (sst types.StoreType, ix int, err error) { + ix, has := binarySearch(pr.reserved, key) + if !has { + err = fmt.Errorf("prefix does not exist: %v", key) + return + } + sst, has = pr.StoreSchema[key] + if !has { + err = fmt.Errorf("prefix is registered but not in schema: %v", key) + } + + return +} + +func (pr *prefixRegistry) ReservePrefix(key string, typ types.StoreType) error { + if !validSubStoreType(typ) { + return fmt.Errorf("StoreType not supported: %v", typ) + } + + // Find the neighboring reserved prefix, and check for duplicates and conflicts + i, has := binarySearch(pr.reserved, key) + if has { + return fmt.Errorf("prefix already exists: %v", key) + } + if i > 0 && strings.HasPrefix(key, pr.reserved[i-1]) { + return fmt.Errorf("prefix conflict: '%v' exists, cannot add '%v'", pr.reserved[i-1], key) + } + if i < len(pr.reserved) && strings.HasPrefix(pr.reserved[i], key) { + return fmt.Errorf("prefix conflict: '%v' exists, cannot add '%v'", pr.reserved[i], key) + } + reserved := pr.reserved[:i] + reserved = append(reserved, key) + pr.reserved = append(reserved, pr.reserved[i:]...) + pr.StoreSchema[key] = typ + return nil +} + +func (lreg *listenerMixin) AddListeners(key types.StoreKey, listeners []types.WriteListener) { + if ls, ok := lreg.listeners[key]; ok { + lreg.listeners[key] = append(ls, listeners...) + } else { + lreg.listeners[key] = listeners + } +} + +// ListeningEnabled returns if listening is enabled for a specific KVStore +func (lreg *listenerMixin) ListeningEnabled(key types.StoreKey) bool { + if ls, ok := lreg.listeners[key]; ok { + return len(ls) != 0 + } + return false +} + +func (treg *traceMixin) TracingEnabled() bool { + return treg.TraceWriter != nil +} +func (treg *traceMixin) SetTracer(w io.Writer) { + treg.TraceWriter = w +} +func (treg *traceMixin) SetTraceContext(tc types.TraceContext) { + treg.TraceContext = tc +} + +func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser, ready chan<- struct{}) error { + return nil +} +func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) { + return nil, nil } diff --git a/store/v2/root/store_test.go b/store/v2/root/store_test.go index e70974c3f988..d2c9a38cf278 100644 --- a/store/v2/root/store_test.go +++ b/store/v2/root/store_test.go @@ -1,7 +1,6 @@ package root import ( - "errors" "math" "testing" @@ -11,7 +10,7 @@ import ( dbm "github.com/cosmos/cosmos-sdk/db" "github.com/cosmos/cosmos-sdk/db/memdb" - "github.com/cosmos/cosmos-sdk/store/types" + types "github.com/cosmos/cosmos-sdk/store/v2" "github.com/cosmos/cosmos-sdk/types/kv" ) @@ -21,24 +20,43 @@ var ( "hello": "goodbye", "aloha": "shalom", } + skey_1 = types.NewKVStoreKey("store1") + skey_2 = types.NewKVStoreKey("store2") + skey_3 = types.NewKVStoreKey("store3") + skey_4 = types.NewKVStoreKey("store4") + skey_1b = types.NewKVStoreKey("store1b") + skey_2b = types.NewKVStoreKey("store2b") + skey_3b = types.NewKVStoreKey("store3b") ) -func newStoreWithData(t *testing.T, db dbm.DBConnection, storeData map[string]string) *Store { - store, err := NewStore(db, DefaultStoreConfig) +func simpleStoreConfig(t *testing.T) StoreConfig { + opts := DefaultStoreConfig() + require.NoError(t, opts.ReservePrefix(skey_1.Name(), types.StoreTypePersistent)) + return opts +} + +func storeConfig123(t *testing.T) StoreConfig { + opts := DefaultStoreConfig() + opts.Pruning = types.PruneNothing + require.NoError(t, opts.ReservePrefix(skey_1.Name(), types.StoreTypePersistent)) + require.NoError(t, opts.ReservePrefix(skey_2.Name(), types.StoreTypePersistent)) + require.NoError(t, opts.ReservePrefix(skey_3.Name(), types.StoreTypePersistent)) + return opts +} + +func newSubStoreWithData(t *testing.T, db dbm.DBConnection, storeData map[string]string) (*Store, types.KVStore) { + root, err := NewStore(db, simpleStoreConfig(t)) require.NoError(t, err) + store := root.GetKVStore(skey_1) for k, v := range storeData { store.Set([]byte(k), []byte(v)) } - return store -} - -func newAlohaStore(t *testing.T, db dbm.DBConnection) *Store { - return newStoreWithData(t, db, alohaData) + return root, store } func TestGetSetHasDelete(t *testing.T) { - store := newAlohaStore(t, memdb.NewDB()) + root, store := newSubStoreWithData(t, memdb.NewDB(), alohaData) key := "hello" exists := store.Has([]byte(key)) @@ -63,7 +81,7 @@ func TestGetSetHasDelete(t *testing.T) { require.Panics(t, func() { store.Set(nil, []byte("value")) }, "Set(nil key) should panic") require.Panics(t, func() { store.Set([]byte{}, []byte("value")) }, "Set(empty key) should panic") require.Panics(t, func() { store.Set([]byte("key"), nil) }, "Set(nil value) should panic") - store.indexBucket = rwCrudFails{store.indexBucket} + root.indexBucket = rwCrudFails{root.indexBucket} require.Panics(t, func() { store.Set([]byte("key"), []byte("value")) }, "Set() when index fails should panic") } @@ -71,14 +89,10 @@ func TestGetSetHasDelete(t *testing.T) { func TestConstructors(t *testing.T) { db := memdb.NewDB() - store := newAlohaStore(t, db) - store.Commit() - require.NoError(t, store.Close()) - - store, err := NewStore(db, DefaultStoreConfig) + store, err := NewStore(db, simpleStoreConfig(t)) require.NoError(t, err) - value := store.Get([]byte("hello")) - require.Equal(t, []byte("goodbye"), value) + _ = store.GetKVStore(skey_1) + store.Commit() require.NoError(t, store.Close()) // Loading with an initial version beyond the lowest should error @@ -87,7 +101,7 @@ func TestConstructors(t *testing.T) { require.Error(t, err) db.Close() - store, err = NewStore(dbVersionsFails{memdb.NewDB()}, DefaultStoreConfig) + store, err = NewStore(dbVersionsFails{memdb.NewDB()}, DefaultStoreConfig()) require.Error(t, err) store, err = NewStore(db, StoreConfig{StateCommitmentDB: dbVersionsFails{memdb.NewDB()}}) require.Error(t, err) @@ -96,7 +110,7 @@ func TestConstructors(t *testing.T) { db = memdb.NewDB() merkledb := memdb.NewDB() w := db.Writer() - store, err = NewStore(db, DefaultStoreConfig) + store, err = NewStore(db, DefaultStoreConfig()) require.Error(t, err) w.Discard() w = merkledb.Writer() @@ -111,24 +125,24 @@ func TestConstructors(t *testing.T) { merkledb.Close() // can't load existing store when we can't access the latest Merkle root hash - store, err = NewStore(db, DefaultStoreConfig) + store, err = NewStore(db, DefaultStoreConfig()) require.NoError(t, err) store.Commit() require.NoError(t, store.Close()) - // because root is misssing + // ...whether because root is misssing w = db.Writer() w.Delete(merkleRootKey) w.Commit() db.SaveNextVersion() - store, err = NewStore(db, DefaultStoreConfig) + store, err = NewStore(db, DefaultStoreConfig()) require.Error(t, err) - // or, because of an error - store, err = NewStore(dbRWCrudFails{db}, DefaultStoreConfig) + // ...or, because of an error + store, err = NewStore(dbRWCrudFails{db}, DefaultStoreConfig()) require.Error(t, err) } func TestIterators(t *testing.T) { - store := newStoreWithData(t, memdb.NewDB(), map[string]string{ + _, store := newSubStoreWithData(t, memdb.NewDB(), map[string]string{ string([]byte{0x00}): "0", string([]byte{0x00, 0x00}): "0 0", string([]byte{0x00, 0x01}): "0 1", @@ -216,11 +230,16 @@ func TestCommit(t *testing.T) { require.NotEqual(t, previd.Version, id.Version) } } - testBasic(StoreConfig{Pruning: types.PruneNothing}) - testBasic(StoreConfig{Pruning: types.PruneNothing, StateCommitmentDB: memdb.NewDB()}) - - testFailedCommit := func(t *testing.T, store *Store, db dbm.DBConnection) { - opts := store.opts + basicOpts := simpleStoreConfig(t) + basicOpts.Pruning = types.PruneNothing + testBasic(basicOpts) + basicOpts.StateCommitmentDB = memdb.NewDB() + testBasic(basicOpts) + + testFailedCommit := func(t *testing.T, + store *Store, + db dbm.DBConnection, + opts StoreConfig) { if db == nil { db = store.stateDB } @@ -231,8 +250,8 @@ func TestCommit(t *testing.T) { versions, _ := db.Versions() require.Equal(t, 0, versions.Count()) - if opts.StateCommitmentDB != nil { - versions, _ = opts.StateCommitmentDB.Versions() + if store.StateCommitmentDB != nil { + versions, _ = store.StateCommitmentDB.Versions() require.Equal(t, 0, versions.Count()) } @@ -242,58 +261,62 @@ func TestCommit(t *testing.T) { require.NoError(t, store.Close()) } + opts := simpleStoreConfig(t) + opts.Pruning = types.PruneNothing + // Ensure storage commit is rolled back in each failure case t.Run("recover after failed Commit", func(t *testing.T) { - store, err := NewStore( - dbRWCommitFails{memdb.NewDB()}, - StoreConfig{Pruning: types.PruneNothing}) + store, err := NewStore(dbRWCommitFails{memdb.NewDB()}, opts) require.NoError(t, err) - testFailedCommit(t, store, nil) + testFailedCommit(t, store, nil, opts) }) t.Run("recover after failed SaveVersion", func(t *testing.T) { - store, err := NewStore( - dbSaveVersionFails{memdb.NewDB()}, - StoreConfig{Pruning: types.PruneNothing}) + store, err := NewStore(dbSaveVersionFails{memdb.NewDB()}, opts) require.NoError(t, err) - testFailedCommit(t, store, nil) + testFailedCommit(t, store, nil, opts) }) + t.Run("recover after failed StateCommitmentDB Commit", func(t *testing.T) { - store, err := NewStore(memdb.NewDB(), - StoreConfig{StateCommitmentDB: dbRWCommitFails{memdb.NewDB()}, Pruning: types.PruneNothing}) + opts.StateCommitmentDB = dbRWCommitFails{memdb.NewDB()} + store, err := NewStore(memdb.NewDB(), opts) require.NoError(t, err) - testFailedCommit(t, store, nil) + testFailedCommit(t, store, nil, opts) }) t.Run("recover after failed StateCommitmentDB SaveVersion", func(t *testing.T) { - store, err := NewStore(memdb.NewDB(), - StoreConfig{StateCommitmentDB: dbSaveVersionFails{memdb.NewDB()}, Pruning: types.PruneNothing}) + opts.StateCommitmentDB = dbSaveVersionFails{memdb.NewDB()} + store, err := NewStore(memdb.NewDB(), opts) require.NoError(t, err) - testFailedCommit(t, store, nil) + testFailedCommit(t, store, nil, opts) }) + opts = simpleStoreConfig(t) t.Run("recover after stateDB.Versions error triggers failure", func(t *testing.T) { db := memdb.NewDB() - store, err := NewStore(db, DefaultStoreConfig) + store, err := NewStore(db, opts) require.NoError(t, err) store.stateDB = dbVersionsFails{store.stateDB} - testFailedCommit(t, store, db) + testFailedCommit(t, store, db, opts) }) t.Run("recover after stateTxn.Set error triggers failure", func(t *testing.T) { - store, err := NewStore(memdb.NewDB(), DefaultStoreConfig) + store, err := NewStore(memdb.NewDB(), opts) require.NoError(t, err) store.stateTxn = rwCrudFails{store.stateTxn} - testFailedCommit(t, store, nil) + testFailedCommit(t, store, nil, opts) }) t.Run("stateDB.DeleteVersion error triggers failure", func(t *testing.T) { - store, err := NewStore(memdb.NewDB(), StoreConfig{StateCommitmentDB: memdb.NewDB()}) + opts.StateCommitmentDB = memdb.NewDB() + store, err := NewStore(memdb.NewDB(), opts) require.NoError(t, err) store.stateCommitmentTxn = rwCommitFails{store.stateCommitmentTxn} store.stateDB = dbDeleteVersionFails{store.stateDB} require.Panics(t, func() { store.Commit() }) }) t.Run("height overflow triggers failure", func(t *testing.T) { - store, err := NewStore(memdb.NewDB(), - StoreConfig{InitialVersion: math.MaxInt64, Pruning: types.PruneNothing}) + opts.StateCommitmentDB = nil + opts.InitialVersion = math.MaxInt64 + opts.Pruning = types.PruneNothing + store, err := NewStore(memdb.NewDB(), opts) require.NoError(t, err) require.Equal(t, int64(math.MaxInt64), store.Commit().Version) require.Panics(t, func() { store.Commit() }) @@ -301,18 +324,24 @@ func TestCommit(t *testing.T) { }) // setting initial version - store, err := NewStore(memdb.NewDB(), - StoreConfig{InitialVersion: 5, Pruning: types.PruneNothing, StateCommitmentDB: memdb.NewDB()}) + opts = simpleStoreConfig(t) + opts.InitialVersion = 5 + opts.Pruning = types.PruneNothing + opts.StateCommitmentDB = memdb.NewDB() + store, err := NewStore(memdb.NewDB(), opts) require.NoError(t, err) require.Equal(t, int64(5), store.Commit().Version) - store, err = NewStore(memdb.NewDB(), StoreConfig{StateCommitmentDB: memdb.NewDB()}) + opts = simpleStoreConfig(t) + store, err = NewStore(memdb.NewDB(), opts) require.NoError(t, err) store.Commit() store.stateDB = dbVersionsFails{store.stateDB} require.Panics(t, func() { store.LastCommitID() }) - store, err = NewStore(memdb.NewDB(), StoreConfig{StateCommitmentDB: memdb.NewDB()}) + opts = simpleStoreConfig(t) + opts.StateCommitmentDB = memdb.NewDB() + store, err = NewStore(memdb.NewDB(), opts) require.NoError(t, err) store.Commit() store.stateTxn = rwCrudFails{store.stateTxn} @@ -364,7 +393,7 @@ func TestPruning(t *testing.T) { // Test pruning interval // Save up to 20th version while checking history at specific version checkpoints - opts := types.PruningOptions{0, 5, 10} + pruning := types.PruningOptions{0, 5, 10} testCheckPoints := map[uint64][]uint64{ 5: []uint64{1, 2, 3, 4, 5}, 10: []uint64{5, 10}, @@ -372,7 +401,7 @@ func TestPruning(t *testing.T) { 20: []uint64{5, 10, 15, 20}, } db := memdb.NewDB() - store, err := NewStore(db, StoreConfig{Pruning: opts}) + store, err := NewStore(db, StoreConfig{Pruning: pruning}) require.NoError(t, err) for i := byte(1); i <= 20; i++ { @@ -396,13 +425,13 @@ func TestPruning(t *testing.T) { } func TestQuery(t *testing.T) { - store := newStoreWithData(t, memdb.NewDB(), nil) + path := func(skey types.StoreKey, endp string) string { return "/" + skey.Name() + endp } - k1, v1 := []byte("key1"), []byte("val1") - k2, v2 := []byte("key2"), []byte("val2") - v3 := []byte("val3") + k1, v1 := []byte("k1"), []byte("v1") + k2, v2 := []byte("k2"), []byte("v2") + v3 := []byte("v3") - ksub := []byte("key") + ksub := []byte("k") KVs0 := kv.Pairs{} KVs1 := kv.Pairs{ Pairs: []kv.Pair{ @@ -426,82 +455,88 @@ func TestQuery(t *testing.T) { valExpSub2, err := KVs2.Marshal() require.NoError(t, err) + store, err := NewStore(memdb.NewDB(), simpleStoreConfig(t)) + require.NoError(t, err) cid := store.Commit() ver := cid.Version - query := abci.RequestQuery{Path: "/key", Data: k1, Height: ver} - querySub := abci.RequestQuery{Path: "/subspace", Data: ksub, Height: ver} + query := abci.RequestQuery{Path: path(skey_1, "/key"), Data: k1, Height: ver} + querySub := abci.RequestQuery{Path: path(skey_1, "/subspace"), Data: ksub, Height: ver} // query subspace before anything set qres := store.Query(querySub) - require.True(t, qres.IsOK()) + require.True(t, qres.IsOK(), qres.Log) require.Equal(t, valExpSubEmpty, qres.Value) + sub := store.GetKVStore(skey_1) + require.NotNil(t, sub) // set data - store.Set(k1, v1) - store.Set(k2, v2) + sub.Set(k1, v1) + sub.Set(k2, v2) // set data without commit, doesn't show up qres = store.Query(query) - require.True(t, qres.IsOK()) + require.True(t, qres.IsOK(), qres.Log) require.Nil(t, qres.Value) // commit it, but still don't see on old version cid = store.Commit() qres = store.Query(query) - require.True(t, qres.IsOK()) + require.True(t, qres.IsOK(), qres.Log) require.Nil(t, qres.Value) // but yes on the new version query.Height = cid.Version qres = store.Query(query) - require.True(t, qres.IsOK()) + require.True(t, qres.IsOK(), qres.Log) require.Equal(t, v1, qres.Value) // and for the subspace + querySub.Height = cid.Version qres = store.Query(querySub) - require.True(t, qres.IsOK()) + require.True(t, qres.IsOK(), qres.Log) require.Equal(t, valExpSub1, qres.Value) // modify - store.Set(k1, v3) + sub.Set(k1, v3) cid = store.Commit() // query will return old values, as height is fixed qres = store.Query(query) - require.True(t, qres.IsOK()) + require.True(t, qres.IsOK(), qres.Log) require.Equal(t, v1, qres.Value) // update to latest in the query and we are happy query.Height = cid.Version qres = store.Query(query) - require.True(t, qres.IsOK()) + require.True(t, qres.IsOK(), qres.Log) require.Equal(t, v3, qres.Value) - query2 := abci.RequestQuery{Path: "/key", Data: k2, Height: cid.Version} + query2 := abci.RequestQuery{Path: path(skey_1, "/key"), Data: k2, Height: cid.Version} qres = store.Query(query2) - require.True(t, qres.IsOK()) + require.True(t, qres.IsOK(), qres.Log) require.Equal(t, v2, qres.Value) // and for the subspace + querySub.Height = cid.Version qres = store.Query(querySub) - require.True(t, qres.IsOK()) + require.True(t, qres.IsOK(), qres.Log) require.Equal(t, valExpSub2, qres.Value) - // default (height 0) will show latest -1 - query0 := abci.RequestQuery{Path: "/key", Data: k1} + // default (height 0) will show latest-1 + query0 := abci.RequestQuery{Path: path(skey_1, "/key"), Data: k1} qres = store.Query(query0) - require.True(t, qres.IsOK()) + require.True(t, qres.IsOK(), qres.Log) require.Equal(t, v1, qres.Value) // querying an empty store will fail - store2, err := NewStore(memdb.NewDB(), DefaultStoreConfig) + store2, err := NewStore(memdb.NewDB(), simpleStoreConfig(t)) require.NoError(t, err) qres = store2.Query(query0) require.True(t, qres.IsErr()) // default shows latest, if latest-1 does not exist - store2.Set(k1, v1) + store2.GetKVStore(skey_1).Set(k1, v1) store2.Commit() qres = store2.Query(query0) - require.True(t, qres.IsOK()) + require.True(t, qres.IsOK(), qres.Log) require.Equal(t, v1, qres.Value) store2.Close() @@ -518,69 +553,243 @@ func TestQuery(t *testing.T) { store2.Close() // query with a nil or empty key fails - badquery := abci.RequestQuery{Path: "/key", Data: []byte{}} + badquery := abci.RequestQuery{Path: path(skey_1, "/key"), Data: []byte{}} qres = store.Query(badquery) require.True(t, qres.IsErr()) badquery.Data = nil qres = store.Query(badquery) require.True(t, qres.IsErr()) // querying an invalid height will fail - badquery = abci.RequestQuery{Path: "/key", Data: k1, Height: store.LastCommitID().Version + 1} + badquery = abci.RequestQuery{Path: path(skey_1, "/key"), Data: k1, Height: store.LastCommitID().Version + 1} qres = store.Query(badquery) require.True(t, qres.IsErr()) // or an invalid path - badquery = abci.RequestQuery{Path: "/badpath", Data: k1} + badquery = abci.RequestQuery{Path: path(skey_1, "/badpath"), Data: k1} qres = store.Query(badquery) require.True(t, qres.IsErr()) // test that proofs are generated with single and separate DBs testProve := func() { - queryProve0 := abci.RequestQuery{Path: "/key", Data: k1, Prove: true} - store.Query(queryProve0) + queryProve0 := abci.RequestQuery{Path: path(skey_1, "/key"), Data: k1, Prove: true} qres = store.Query(queryProve0) - require.True(t, qres.IsOK()) + require.True(t, qres.IsOK(), qres.Log) require.Equal(t, v1, qres.Value) require.NotNil(t, qres.ProofOps) } testProve() store.Close() - store, err = NewStore(memdb.NewDB(), StoreConfig{StateCommitmentDB: memdb.NewDB()}) + opts := simpleStoreConfig(t) + opts.StateCommitmentDB = memdb.NewDB() + store, err = NewStore(memdb.NewDB(), opts) require.NoError(t, err) - store.Set(k1, v1) + store.GetKVStore(skey_1).Set(k1, v1) store.Commit() testProve() store.Close() } -type dbDeleteVersionFails struct{ dbm.DBConnection } -type dbRWCommitFails struct{ *memdb.MemDB } -type dbRWCrudFails struct{ dbm.DBConnection } -type dbSaveVersionFails struct{ *memdb.MemDB } -type dbVersionsIs struct { - dbm.DBConnection - vset dbm.VersionSet +func TestStoreConfig(t *testing.T) { + opts := DefaultStoreConfig() + // Fail with invalid types + require.Error(t, opts.ReservePrefix(skey_1.Name(), types.StoreTypeDB)) + require.Error(t, opts.ReservePrefix(skey_1.Name(), types.StoreTypeSMT)) + // Ensure that no prefix conflicts are allowed + require.NoError(t, opts.ReservePrefix(skey_1.Name(), types.StoreTypePersistent)) + require.NoError(t, opts.ReservePrefix(skey_2.Name(), types.StoreTypeMemory)) + require.NoError(t, opts.ReservePrefix(skey_3b.Name(), types.StoreTypeTransient)) + require.Error(t, opts.ReservePrefix(skey_1b.Name(), types.StoreTypePersistent)) + require.Error(t, opts.ReservePrefix(skey_2b.Name(), types.StoreTypePersistent)) + require.Error(t, opts.ReservePrefix(skey_3.Name(), types.StoreTypePersistent)) } -type dbVersionsFails struct{ dbm.DBConnection } -type rwCommitFails struct{ dbm.DBReadWriter } -type rwCrudFails struct{ dbm.DBReadWriter } - -func (dbVersionsFails) Versions() (dbm.VersionSet, error) { return nil, errors.New("dbVersionsFails") } -func (db dbVersionsIs) Versions() (dbm.VersionSet, error) { return db.vset, nil } -func (db dbRWCrudFails) ReadWriter() dbm.DBReadWriter { - return rwCrudFails{db.DBConnection.ReadWriter()} -} -func (dbSaveVersionFails) SaveVersion(uint64) error { return errors.New("dbSaveVersionFails") } -func (dbDeleteVersionFails) DeleteVersion(uint64) error { return errors.New("dbDeleteVersionFails") } -func (tx rwCommitFails) Commit() error { - tx.Discard() - return errors.New("rwCommitFails") + +func TestRootStoreBasic(t *testing.T) { + opts := DefaultStoreConfig() + err := opts.ReservePrefix(skey_1.Name(), types.StoreTypePersistent) + require.NoError(t, err) + db := memdb.NewDB() + store, err := NewStore(db, opts) + require.NoError(t, err) + + store_1 := store.GetKVStore(skey_1) + require.NotNil(t, store_1) + store_1.Set([]byte{0}, []byte{0}) + val := store_1.Get([]byte{0}) + require.Equal(t, []byte{0}, val) + store_1.Delete([]byte{0}) + val = store_1.Get([]byte{0}) + require.Equal(t, []byte(nil), val) } -func (db dbRWCommitFails) ReadWriter() dbm.DBReadWriter { - return rwCommitFails{db.MemDB.ReadWriter()} + +func TestGetVersion(t *testing.T) { + db := memdb.NewDB() + opts := storeConfig123(t) + store, err := NewStore(db, opts) + require.NoError(t, err) + + cid := store.Commit() + // opts := DefaultStoreConfig() + + view, err := store.GetVersion(cid.Version) + require.NoError(t, err) + subview := view.GetKVStore(skey_1) + require.NotNil(t, subview) + + // version view should be read-only + require.Panics(t, func() { subview.Set([]byte{1}, []byte{1}) }) + require.Panics(t, func() { subview.Delete([]byte{0}) }) + // nonexistent version shouldn't be accessible + view, err = store.GetVersion(cid.Version + 1) + require.Equal(t, ErrVersionDoesNotExist, err) + + substore := store.GetKVStore(skey_1) + require.NotNil(t, substore) + substore.Set([]byte{0}, []byte{0}) + // setting a new value shouldn't affect old version + require.False(t, subview.Has([]byte{0})) + + cid = store.Commit() + view, err = store.GetVersion(cid.Version) + require.NoError(t, err) + subview = view.GetKVStore(skey_1) + require.NotNil(t, subview) + + store.Delete([]byte{0}) + require.Equal(t, []byte{0}, subview.Get([]byte{0})) } -func (rwCrudFails) Get([]byte) ([]byte, error) { return nil, errors.New("rwCrudFails.Get") } -func (rwCrudFails) Has([]byte) (bool, error) { return false, errors.New("rwCrudFails.Has") } -func (rwCrudFails) Set([]byte, []byte) error { return errors.New("rwCrudFails.Set") } -func (rwCrudFails) Delete([]byte) error { return errors.New("rwCrudFails.Delete") } +func TestRootStoreMigration(t *testing.T) { + db := memdb.NewDB() + opts := storeConfig123(t) + store, err := NewStore(db, opts) + require.NoError(t, err) + + // write some data in all stores + k1, v1 := []byte("first"), []byte("store") + s1 := store.GetKVStore(skey_1) + require.NotNil(t, s1) + s1.Set(k1, v1) + + k2, v2 := []byte("second"), []byte("restore") + s2 := store.GetKVStore(skey_2) + require.NotNil(t, s2) + s2.Set(k2, v2) + + k3, v3 := []byte("third"), []byte("dropped") + s3 := store.GetKVStore(skey_3) + require.NotNil(t, s3) + s3.Set(k3, v3) + + require.Panics(t, func() { store.GetKVStore(skey_4) }) + + cid := store.Commit() + require.NoError(t, store.Close()) + + // Load without changes and make sure it is sensible + store, err = NewStore(db, opts) + require.NoError(t, err) + + // let's query data to see it was saved properly + s2 = store.GetKVStore(skey_2) + require.NotNil(t, s2) + require.Equal(t, v2, s2.Get(k2)) + require.NoError(t, store.Close()) + + // now, let's load with upgrades... + opts.Upgrades = []types.StoreUpgrades{ + types.StoreUpgrades{ + Added: []string{skey_4.Name()}, + Renamed: []types.StoreRename{{ + OldKey: skey_2.Name(), + NewKey: skey_2b.Name(), + }}, + Deleted: []string{skey_3.Name()}, + }, + } + restore, err := NewStore(db, opts) + require.Nil(t, err) + + // s1 was not changed + s1 = restore.GetKVStore(skey_1) + require.NotNil(t, s1) + require.Equal(t, v1, s1.Get(k1)) + + // store3 is gone + require.Panics(t, func() { s3 = restore.GetKVStore(skey_3) }) + + // store4 is mounted, with empty data + s4 := restore.GetKVStore(skey_4) + require.NotNil(t, s4) + + values := 0 + it := s4.Iterator(nil, nil) + for ; it.Valid(); it.Next() { + values += 1 + } + require.Zero(t, values) + require.NoError(t, it.Close()) + + // write something inside store4 + k4, v4 := []byte("fourth"), []byte("created") + s4.Set(k4, v4) + + // store2 is no longer mounted + require.Panics(t, func() { restore.GetKVStore(skey_2) }) + + // restore2 has the old data + rs2 := restore.GetKVStore(skey_2b) + require.NotNil(t, rs2) + require.Equal(t, v2, rs2.Get(k2)) + + // store this migrated data, and load it again without migrations + migratedID := restore.Commit() + require.Equal(t, migratedID.Version, int64(2)) + require.NoError(t, restore.Close()) + + // fail to load the migrated store with the old schema + reload, err := NewStore(db, storeConfig123(t)) + require.Error(t, err) + + // pass in a schema reflecting the migrations + migratedOpts := DefaultStoreConfig() + err = migratedOpts.ReservePrefix(skey_1.Name(), types.StoreTypePersistent) + require.NoError(t, err) + err = migratedOpts.ReservePrefix(skey_2b.Name(), types.StoreTypePersistent) + require.NoError(t, err) + err = migratedOpts.ReservePrefix(skey_4.Name(), types.StoreTypePersistent) + require.NoError(t, err) + reload, err = NewStore(db, migratedOpts) + require.Nil(t, err) + require.Equal(t, migratedID, reload.LastCommitID()) + + // query this new store + rl1 := reload.GetKVStore(skey_1) + require.NotNil(t, rl1) + require.Equal(t, v1, rl1.Get(k1)) + + rl2 := reload.GetKVStore(skey_2b) + require.NotNil(t, rl2) + require.Equal(t, v2, rl2.Get(k2)) + + rl4 := reload.GetKVStore(skey_4) + require.NotNil(t, rl4) + require.Equal(t, v4, rl4.Get(k4)) + + // load and check a view of the store at first commit + view, err := store.GetVersion(cid.Version) + require.NoError(t, err) + + s1 = view.GetKVStore(skey_1) + require.NotNil(t, s1) + require.Equal(t, v1, s1.Get(k1)) + + s2 = view.GetKVStore(skey_2) + require.NotNil(t, s2) + require.Equal(t, v2, s2.Get(k2)) + + s3 = view.GetKVStore(skey_3) + require.NotNil(t, s3) + require.Equal(t, v3, s3.Get(k3)) + + require.Panics(t, func() { view.GetKVStore(skey_4) }) +} diff --git a/store/v2/root/store_view.go b/store/v2/root/store_view.go deleted file mode 100644 index a5ed1cded9b6..000000000000 --- a/store/v2/root/store_view.go +++ /dev/null @@ -1,130 +0,0 @@ -package root - -import ( - "errors" - "io" - - dbm "github.com/cosmos/cosmos-sdk/db" - "github.com/cosmos/cosmos-sdk/db/prefix" - - util "github.com/cosmos/cosmos-sdk/internal" - "github.com/cosmos/cosmos-sdk/store/cachekv" - "github.com/cosmos/cosmos-sdk/store/listenkv" - "github.com/cosmos/cosmos-sdk/store/tracekv" - types "github.com/cosmos/cosmos-sdk/store/v2" - "github.com/cosmos/cosmos-sdk/store/v2/smt" -) - -var ErrReadOnly = errors.New("cannot modify read-only store") - -// Represents a read-only view of a store's contents at a given version. -type storeView struct { - stateView dbm.DBReader - dataBucket dbm.DBReader - indexBucket dbm.DBReader - stateCommitmentView dbm.DBReader - stateCommitmentStore *smt.Store -} - -func (s *Store) GetVersion(version int64) (ret *storeView, err error) { - stateView, err := s.stateDB.ReaderAt(uint64(version)) - if err != nil { - return - } - defer func() { - if err != nil { - err = util.CombineErrors(err, stateView.Discard(), "stateView.Discard also failed") - } - }() - - stateCommitmentView := stateView - if s.opts.StateCommitmentDB != nil { - stateCommitmentView, err = s.opts.StateCommitmentDB.ReaderAt(uint64(version)) - if err != nil { - return - } - defer func() { - if err != nil { - err = util.CombineErrors(err, stateCommitmentView.Discard(), "stateCommitmentView.Discard also failed") - } - }() - } - root, err := stateView.Get(merkleRootKey) - if err != nil { - return - } - return &storeView{ - stateView: stateView, - dataBucket: prefix.NewPrefixReader(stateView, dataPrefix), - indexBucket: prefix.NewPrefixReader(stateView, indexPrefix), - stateCommitmentView: stateCommitmentView, - stateCommitmentStore: loadSMT(dbm.ReaderAsReadWriter(stateCommitmentView), root), - }, nil -} - -func (s *storeView) GetStateCommitmentStore() *smt.Store { - return s.stateCommitmentStore -} - -// Get implements KVStore. -func (s *storeView) Get(key []byte) []byte { - val, err := s.dataBucket.Get(key) - if err != nil { - panic(err) - } - return val -} - -// Has implements KVStore. -func (s *storeView) Has(key []byte) bool { - has, err := s.dataBucket.Has(key) - if err != nil { - panic(err) - } - return has -} - -// Set implements KVStore. -func (s *storeView) Set(key []byte, value []byte) { - panic(ErrReadOnly) -} - -// Delete implements KVStore. -func (s *storeView) Delete(key []byte) { - panic(ErrReadOnly) -} - -// Iterator implements KVStore. -func (s *storeView) Iterator(start, end []byte) types.Iterator { - iter, err := s.dataBucket.Iterator(start, end) - if err != nil { - panic(err) - } - return newIterator(iter) -} - -// ReverseIterator implements KVStore. -func (s *storeView) ReverseIterator(start, end []byte) types.Iterator { - iter, err := s.dataBucket.ReverseIterator(start, end) - if err != nil { - panic(err) - } - return newIterator(iter) -} - -// GetStoreType implements Store. -func (s *storeView) GetStoreType() types.StoreType { - return types.StoreTypeDecoupled -} - -func (st *storeView) CacheWrap() types.CacheWrap { - return cachekv.NewStore(st) -} - -func (st *storeView) CacheWrapWithTrace(w io.Writer, tc types.TraceContext) types.CacheWrap { - return cachekv.NewStore(tracekv.NewStore(st, w, tc)) -} - -func (st *storeView) CacheWrapWithListeners(storeKey types.StoreKey, listeners []types.WriteListener) types.CacheWrap { - return cachekv.NewStore(listenkv.NewStore(st, storeKey, listeners)) -} diff --git a/store/v2/root/test_util.go b/store/v2/root/test_util.go new file mode 100644 index 000000000000..f41b31115d41 --- /dev/null +++ b/store/v2/root/test_util.go @@ -0,0 +1,40 @@ +package root + +import ( + "errors" + + dbm "github.com/cosmos/cosmos-sdk/db" + "github.com/cosmos/cosmos-sdk/db/memdb" +) + +type dbDeleteVersionFails struct{ dbm.DBConnection } +type dbRWCommitFails struct{ *memdb.MemDB } +type dbRWCrudFails struct{ dbm.DBConnection } +type dbSaveVersionFails struct{ *memdb.MemDB } +type dbVersionsIs struct { + dbm.DBConnection + vset dbm.VersionSet +} +type dbVersionsFails struct{ dbm.DBConnection } +type rwCommitFails struct{ dbm.DBReadWriter } +type rwCrudFails struct{ dbm.DBReadWriter } + +func (dbVersionsFails) Versions() (dbm.VersionSet, error) { return nil, errors.New("dbVersionsFails") } +func (db dbVersionsIs) Versions() (dbm.VersionSet, error) { return db.vset, nil } +func (db dbRWCrudFails) ReadWriter() dbm.DBReadWriter { + return rwCrudFails{db.DBConnection.ReadWriter()} +} +func (dbSaveVersionFails) SaveVersion(uint64) error { return errors.New("dbSaveVersionFails") } +func (dbDeleteVersionFails) DeleteVersion(uint64) error { return errors.New("dbDeleteVersionFails") } +func (tx rwCommitFails) Commit() error { + tx.Discard() + return errors.New("rwCommitFails") +} +func (db dbRWCommitFails) ReadWriter() dbm.DBReadWriter { + return rwCommitFails{db.MemDB.ReadWriter()} +} + +func (rwCrudFails) Get([]byte) ([]byte, error) { return nil, errors.New("rwCrudFails.Get") } +func (rwCrudFails) Has([]byte) (bool, error) { return false, errors.New("rwCrudFails.Has") } +func (rwCrudFails) Set([]byte, []byte) error { return errors.New("rwCrudFails.Set") } +func (rwCrudFails) Delete([]byte) error { return errors.New("rwCrudFails.Delete") } diff --git a/store/v2/root/view_store.go b/store/v2/root/view_store.go new file mode 100644 index 000000000000..7f75957f4d02 --- /dev/null +++ b/store/v2/root/view_store.go @@ -0,0 +1,155 @@ +package root + +import ( + "errors" + "io" + + dbm "github.com/cosmos/cosmos-sdk/db" + "github.com/cosmos/cosmos-sdk/db/memdb" + prefixdb "github.com/cosmos/cosmos-sdk/db/prefix" + util "github.com/cosmos/cosmos-sdk/internal" + "github.com/cosmos/cosmos-sdk/store/cachekv" + "github.com/cosmos/cosmos-sdk/store/listenkv" + "github.com/cosmos/cosmos-sdk/store/tracekv" + types "github.com/cosmos/cosmos-sdk/store/v2" + "github.com/cosmos/cosmos-sdk/store/v2/mem" + "github.com/cosmos/cosmos-sdk/store/v2/smt" + transkv "github.com/cosmos/cosmos-sdk/store/v2/transient" +) + +var ErrReadOnly = errors.New("cannot modify read-only store") + +func (s *viewStore) GetStateCommitmentStore() *smt.Store { + return s.stateCommitmentStore +} + +// Get implements KVStore. +func (s *viewStore) Get(key []byte) []byte { + val, err := s.dataBucket.Get(key) + if err != nil { + panic(err) + } + return val +} + +// Has implements KVStore. +func (s *viewStore) Has(key []byte) bool { + has, err := s.dataBucket.Has(key) + if err != nil { + panic(err) + } + return has +} + +// Set implements KVStore. +func (s *viewStore) Set(key []byte, value []byte) { + panic(ErrReadOnly) +} + +// Delete implements KVStore. +func (s *viewStore) Delete(key []byte) { + panic(ErrReadOnly) +} + +// Iterator implements KVStore. +func (s *viewStore) Iterator(start, end []byte) types.Iterator { + iter, err := s.dataBucket.Iterator(start, end) + if err != nil { + panic(err) + } + return newIterator(iter) +} + +// ReverseIterator implements KVStore. +func (s *viewStore) ReverseIterator(start, end []byte) types.Iterator { + iter, err := s.dataBucket.ReverseIterator(start, end) + if err != nil { + panic(err) + } + return newIterator(iter) +} + +// GetStoreType implements Store. +func (s *viewStore) GetStoreType() types.StoreType { + return types.StoreTypePersistent +} + +func (st *viewStore) CacheWrap() types.CacheWrap { + return cachekv.NewStore(st) +} + +func (st *viewStore) CacheWrapWithTrace(w io.Writer, tc types.TraceContext) types.CacheWrap { + return cachekv.NewStore(tracekv.NewStore(st, w, tc)) +} + +func (st *viewStore) CacheWrapWithListeners(storeKey types.StoreKey, listeners []types.WriteListener) types.CacheWrap { + return cachekv.NewStore(listenkv.NewStore(st, storeKey, listeners)) +} + +func (store *Store) getView(version int64) (ret *viewStore, err error) { + stateView, err := store.stateDB.ReaderAt(uint64(version)) + if err != nil { + return + } + defer func() { + if err != nil { + err = util.CombineErrors(err, stateView.Discard(), "stateView.Discard also failed") + } + }() + + stateCommitmentView := stateView + if store.StateCommitmentDB != nil { + stateCommitmentView, err = store.StateCommitmentDB.ReaderAt(uint64(version)) + if err != nil { + return + } + defer func() { + if err != nil { + err = util.CombineErrors(err, stateCommitmentView.Discard(), "stateCommitmentView.Discard also failed") + } + }() + } + root, err := stateView.Get(merkleRootKey) + if err != nil { + return + } + ret = &viewStore{ + stateView: stateView, + dataBucket: prefixdb.NewPrefixReader(stateView, dataPrefix), + indexBucket: prefixdb.NewPrefixReader(stateView, indexPrefix), + stateCommitmentView: stateCommitmentView, + stateCommitmentStore: loadSMT(dbm.ReaderAsReadWriter(stateCommitmentView), root), + } + // Now read this version's schema + schemaView := prefixdb.NewPrefixReader(ret.stateView, schemaPrefix) + defer func() { + if err != nil { + err = util.CombineErrors(err, schemaView.Discard(), "schemaView.Discard also failed") + } + }() + pr, err := readSavedSchema(schemaView) + if err != nil { + return + } + // The migrated contents and schema are not committed until the next store.Commit + ret.schema = pr.StoreSchema + return +} + +// if the schema indicates a mem/tran store, it's ignored +func (rv *viewStore) generic() rootGeneric { return rootGeneric{rv.schema, rv, nil, nil} } + +func (rv *viewStore) GetKVStore(key types.StoreKey) types.KVStore { + return rv.generic().getStore(key.Name()) +} + +func (rv *viewStore) CacheRootStore() types.CacheRootStore { + return &cacheStore{ + CacheKVStore: cachekv.NewStore(rv), + mem: cachekv.NewStore(mem.NewStore(memdb.NewDB())), + tran: cachekv.NewStore(transkv.NewStore(memdb.NewDB())), + schema: rv.schema, + listenerMixin: &listenerMixin{}, + traceMixin: &traceMixin{}, + } +} diff --git a/store/v2/types.go b/store/v2/types.go index f47376ba5cc4..743bb91eccc4 100644 --- a/store/v2/types.go +++ b/store/v2/types.go @@ -91,3 +91,5 @@ const StoreTypePersistent = v1.StoreTypePersistent var NewKVStoreKey = v1.NewKVStoreKey var PrefixEndBytes = v1.PrefixEndBytes +var KVStorePrefixIterator = v1.KVStorePrefixIterator +var KVStoreReversePrefixIterator = v1.KVStoreReversePrefixIterator From fa85a723d5ac33676dec06e16b2dc27558de6118 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 17 Nov 2021 00:31:31 +0800 Subject: [PATCH 13/47] RootStore uses per-substore SMTs --- db/prefix/prefix.go | 11 +- store/v2/dbadapter/store.go | 2 - store/v2/root/kv_store.go | 136 ------------- store/v2/root/store.go | 396 +++++++++++++++++++++++------------- store/v2/root/store_test.go | 73 ++++--- store/v2/root/sub_store.go | 117 +++++++++++ store/v2/root/test_util.go | 36 +++- store/v2/root/view_store.go | 90 ++++---- 8 files changed, 511 insertions(+), 350 deletions(-) delete mode 100644 store/v2/root/kv_store.go create mode 100644 store/v2/root/sub_store.go diff --git a/db/prefix/prefix.go b/db/prefix/prefix.go index 3f97a91fa92c..af4ae2aa10fd 100644 --- a/db/prefix/prefix.go +++ b/db/prefix/prefix.go @@ -46,7 +46,7 @@ func NewPrefixWriter(db dbm.DBWriter, prefix []byte) prefixW { } func prefixed(prefix, key []byte) []byte { - return append(prefix, key...) + return append(cp(prefix), key...) } // Get implements DBReader. @@ -170,6 +170,12 @@ func (pdb prefixW) Commit() error { return pdb.db.Commit() } // Discard implements DBReadWriter. func (pdb prefixW) Discard() error { return pdb.db.Discard() } +func cp(bz []byte) (ret []byte) { + ret = make([]byte, len(bz)) + copy(ret, bz) + return ret +} + // Returns a slice of the same length (big endian), but incremented by one. // Returns nil on overflow (e.g. if bz bytes are all 0xFF) // CONTRACT: len(bz) > 0 @@ -177,8 +183,7 @@ func cpIncr(bz []byte) (ret []byte) { if len(bz) == 0 { panic("cpIncr expects non-zero bz length") } - ret = make([]byte, len(bz)) - copy(ret, bz) + ret = cp(bz) for i := len(bz) - 1; i >= 0; i-- { if ret[i] < byte(0xFF) { ret[i]++ diff --git a/store/v2/dbadapter/store.go b/store/v2/dbadapter/store.go index 152314de8110..fa59d90fc5f7 100644 --- a/store/v2/dbadapter/store.go +++ b/store/v2/dbadapter/store.go @@ -62,7 +62,6 @@ func (dsa Store) Iterator(start, end []byte) types.Iterator { if err != nil { panic(err) } - return dbutil.DBToStoreIterator(iter) } @@ -72,7 +71,6 @@ func (dsa Store) ReverseIterator(start, end []byte) types.Iterator { if err != nil { panic(err) } - return dbutil.DBToStoreIterator(iter) } diff --git a/store/v2/root/kv_store.go b/store/v2/root/kv_store.go deleted file mode 100644 index e1b817c4eb77..000000000000 --- a/store/v2/root/kv_store.go +++ /dev/null @@ -1,136 +0,0 @@ -package root - -import ( - "crypto/sha256" - "io" - - dbm "github.com/cosmos/cosmos-sdk/db" - "github.com/cosmos/cosmos-sdk/db/prefix" - "github.com/cosmos/cosmos-sdk/store/cachekv" - "github.com/cosmos/cosmos-sdk/store/listenkv" - "github.com/cosmos/cosmos-sdk/store/tracekv" - "github.com/cosmos/cosmos-sdk/store/types" - "github.com/cosmos/cosmos-sdk/store/v2/smt" -) - -var ( - _ types.KVStore = (*Store)(nil) -) - -// Store is a CommitKVStore which handles state storage and commitments as separate concerns, -// optionally using separate backing key-value DBs for each. -// Allows synchronized R/W access by locking. - -// var DefaultStoreConfig = StoreConfig{Pruning: types.PruneDefault, StateCommitmentDB: nil} - -// NewStore creates a new Store, or loads one if the DB contains existing data. - -// Get implements KVStore. -func (s *Store) Get(key []byte) []byte { - s.mtx.RLock() - defer s.mtx.RUnlock() - - val, err := s.dataBucket.Get(key) - if err != nil { - panic(err) - } - return val -} - -// Has implements KVStore. -func (s *Store) Has(key []byte) bool { - s.mtx.RLock() - defer s.mtx.RUnlock() - - has, err := s.dataBucket.Has(key) - if err != nil { - panic(err) - } - return has -} - -// Set implements KVStore. -func (s *Store) Set(key, value []byte) { - s.mtx.Lock() - defer s.mtx.Unlock() - - err := s.dataBucket.Set(key, value) - if err != nil { - panic(err) - } - s.stateCommitmentStore.Set(key, value) - khash := sha256.Sum256(key) - err = s.indexBucket.Set(khash[:], key) - if err != nil { - panic(err) - } -} - -// Delete implements KVStore. -func (s *Store) Delete(key []byte) { - khash := sha256.Sum256(key) - s.mtx.Lock() - defer s.mtx.Unlock() - - s.stateCommitmentStore.Delete(key) - _ = s.indexBucket.Delete(khash[:]) - _ = s.dataBucket.Delete(key) -} - -type contentsIterator struct { - dbm.Iterator - valid bool -} - -func newIterator(source dbm.Iterator) *contentsIterator { - ret := &contentsIterator{Iterator: source} - ret.Next() - return ret -} - -func (it *contentsIterator) Next() { it.valid = it.Iterator.Next() } -func (it *contentsIterator) Valid() bool { return it.valid } - -// Iterator implements KVStore. -func (s *Store) Iterator(start, end []byte) types.Iterator { - iter, err := s.dataBucket.Iterator(start, end) - if err != nil { - panic(err) - } - return newIterator(iter) -} - -// ReverseIterator implements KVStore. -func (s *Store) ReverseIterator(start, end []byte) types.Iterator { - iter, err := s.dataBucket.ReverseIterator(start, end) - if err != nil { - panic(err) - } - return newIterator(iter) -} - -// GetStoreType implements Store. -func (s *Store) GetStoreType() types.StoreType { - return types.StoreTypePersistent -} - -func (s *Store) GetPruning() types.PruningOptions { return s.Pruning } -func (s *Store) SetPruning(po types.PruningOptions) { s.Pruning = po } - -func loadSMT(stateCommitmentTxn dbm.DBReadWriter, root []byte) *smt.Store { - merkleNodes := prefix.NewPrefixReadWriter(stateCommitmentTxn, merkleNodePrefix) - merkleValues := prefix.NewPrefixReadWriter(stateCommitmentTxn, merkleValuePrefix) - return smt.LoadStore(merkleNodes, merkleValues, root) -} - -func (s *Store) CacheWrap() types.CacheWrap { - return cachekv.NewStore(s) -} - -func (s *Store) CacheWrapWithTrace(w io.Writer, tc types.TraceContext) types.CacheWrap { - return cachekv.NewStore(tracekv.NewStore(s, w, tc)) -} - -func (s *Store) CacheWrapWithListeners(storeKey types.StoreKey, listeners []types.WriteListener) types.CacheWrap { - return cachekv.NewStore(listenkv.NewStore(s, storeKey, listeners)) -} diff --git a/store/v2/root/store.go b/store/v2/root/store.go index 0d8abaa4785d..d7d2d07032d1 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -17,6 +17,7 @@ import ( prefixdb "github.com/cosmos/cosmos-sdk/db/prefix" util "github.com/cosmos/cosmos-sdk/internal" "github.com/cosmos/cosmos-sdk/store/cachekv" + sdkmaps "github.com/cosmos/cosmos-sdk/store/internal/maps" "github.com/cosmos/cosmos-sdk/store/prefix" types "github.com/cosmos/cosmos-sdk/store/v2" "github.com/cosmos/cosmos-sdk/store/v2/mem" @@ -27,25 +28,29 @@ import ( ) var ( - _ types.KVStore = (*Store)(nil) _ types.Queryable = (*Store)(nil) _ types.CommitRootStore = (*Store)(nil) _ types.CacheRootStore = (*cacheStore)(nil) _ types.BasicRootStore = (*viewStore)(nil) + _ types.KVStore = (*substore)(nil) ) var ( - merkleRootKey = []byte{0} // Key for root hash of Merkle tree - dataPrefix = []byte{1} // Prefix for state mappings - indexPrefix = []byte{2} // Prefix for Store reverse index - merkleNodePrefix = []byte{3} // Prefix for Merkle tree nodes - merkleValuePrefix = []byte{4} // Prefix for Merkle value mappings - schemaPrefix = []byte{5} // Prefix for store keys (namespaces) -) + // Root prefixes + merkleRootKey = []byte{0} // Key for root hash of namespace tree + schemaPrefix = []byte{1} // Prefix for store keys (namespaces) + contentPrefix = []byte{2} // Prefix for store contents + + // Per-substore prefixes + substoreMerkleRootKey = []byte{0} // Key for root hashes of Merkle trees + dataPrefix = []byte{1} // Prefix for state mappings + indexPrefix = []byte{2} // Prefix for Store reverse index + merkleNodePrefix = []byte{3} // Prefix for Merkle tree nodes + merkleValuePrefix = []byte{4} // Prefix for Merkle value mappings -var ( ErrVersionDoesNotExist = errors.New("version does not exist") ErrMaximumHeight = errors.New("maximum block height reached") + ErrStoreNotFound = func(skey string) error { return fmt.Errorf("store does not exist for key: %s", skey) } ) // StoreConfig is used to define a schema and pass options to the RootStore constructor. @@ -68,31 +73,11 @@ type StoreConfig struct { // A loaded mapping of substore keys to store types type StoreSchema map[string]types.StoreType -// Builder type used to create a valid schema with no prefix conflicts -type prefixRegistry struct { - StoreSchema - reserved []string -} - -// Mixin types that will be composed into each distinct root store variant type -type listenerMixin struct { - listeners map[types.StoreKey][]types.WriteListener -} - -type traceMixin struct { - TraceWriter io.Writer - TraceContext types.TraceContext -} - // Main persistent store type type Store struct { stateDB dbm.DBConnection stateTxn dbm.DBReadWriter - dataBucket dbm.DBReadWriter - indexBucket dbm.DBReadWriter stateCommitmentTxn dbm.DBReadWriter - // State commitment (SC) KV store for current version - stateCommitmentStore *smt.Store Pruning types.PruningOptions InitialVersion uint64 @@ -104,35 +89,59 @@ type Store struct { *listenerMixin *traceMixin - mtx sync.RWMutex + substoreCache map[string]*substore + npSubstoreCache map[string]types.KVStore + + mtx sync.RWMutex + loadExisting bool PersistentCache types.RootStorePersistentCache } +type substore struct { + root *Store + key string + dataBucket dbm.DBReadWriter + indexBucket dbm.DBReadWriter + stateCommitmentStore *smt.Store +} + // Branched state type cacheStore struct { - types.CacheKVStore - mem, tran types.CacheKVStore - schema StoreSchema + source types.BasicRootStore + substores map[string]types.CacheKVStore *listenerMixin *traceMixin } // Read-only store for querying type viewStore struct { - stateView dbm.DBReader + stateView dbm.DBReader + stateCommitmentView dbm.DBReader + substoreCache map[string]*viewSubstore + schema StoreSchema +} + +type viewSubstore struct { dataBucket dbm.DBReader indexBucket dbm.DBReader - stateCommitmentView dbm.DBReader stateCommitmentStore *smt.Store +} - schema StoreSchema +// Builder type used to create a valid schema with no prefix conflicts +type prefixRegistry struct { + StoreSchema + reserved []string +} + +// Mixin types that will be composed into each distinct root store variant type +type listenerMixin struct { + listeners map[types.StoreKey][]types.WriteListener } -// Auxiliary type used only to avoid repetitive method implementations -type rootGeneric struct { - schema StoreSchema - persist, mem, tran types.KVStore +type traceMixin struct { + TraceWriter io.Writer + TraceContext types.TraceContext } // DefaultStoreConfig returns a RootStore config with an empty schema, a single backing DB, @@ -199,11 +208,14 @@ func readSavedSchema(bucket dbm.DBReader) (*prefixRegistry, error) { ret.StoreSchema[string(it.Key())] = types.StoreType(value[0]) ret.reserved = append(ret.reserved, string(it.Key())) // assume iter yields keys sorted } - it.Close() + if err = it.Close(); err != nil { + return nil, err + } return &ret, nil } -// NewStore constructs a RootStore directly from a DB connection and options. +// NewStore constructs a RootStore directly from a database. +// Creates a new store if no data exists; otherwise loads existing data. func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { versions, err := db.Versions() if err != nil { @@ -247,37 +259,22 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { stateCommitmentTxn = opts.StateCommitmentDB.ReadWriter() } - var stateCommitmentStore *smt.Store - if loadExisting { - var root []byte - root, err = stateTxn.Get(merkleRootKey) - if err != nil { - return - } - if root == nil { - err = fmt.Errorf("could not get root of SMT") - return - } - stateCommitmentStore = loadSMT(stateCommitmentTxn, root) - } else { - merkleNodes := prefixdb.NewPrefixReadWriter(stateCommitmentTxn, merkleNodePrefix) - merkleValues := prefixdb.NewPrefixReadWriter(stateCommitmentTxn, merkleValuePrefix) - stateCommitmentStore = smt.NewStore(merkleNodes, merkleValues) - } ret = &Store{ - stateDB: db, - stateTxn: stateTxn, - dataBucket: prefixdb.NewPrefixReadWriter(stateTxn, dataPrefix), - indexBucket: prefixdb.NewPrefixReadWriter(stateTxn, indexPrefix), - stateCommitmentTxn: stateCommitmentTxn, - stateCommitmentStore: stateCommitmentStore, + stateDB: db, + stateTxn: stateTxn, + StateCommitmentDB: opts.StateCommitmentDB, + stateCommitmentTxn: stateCommitmentTxn, + + substoreCache: map[string]*substore{}, + npSubstoreCache: map[string]types.KVStore{}, - Pruning: opts.Pruning, - InitialVersion: opts.InitialVersion, - StateCommitmentDB: opts.StateCommitmentDB, - PersistentCache: opts.PersistentCache, - listenerMixin: opts.listenerMixin, - traceMixin: opts.traceMixin, + listenerMixin: opts.listenerMixin, + traceMixin: opts.traceMixin, + PersistentCache: opts.PersistentCache, + + Pruning: opts.Pruning, + InitialVersion: opts.InitialVersion, + loadExisting: loadExisting, } // Now load the substore schema @@ -332,13 +329,13 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { if err != nil { return } + // NB. the migrated contents and schema are not committed until the next store.Commit for skey, typ := range reg.StoreSchema { err = schemaWriter.Set([]byte(skey), []byte{byte(typ)}) if err != nil { return } } - // The migrated contents and schema are not committed until the next store.Commit ret.mem = mem.NewStore(memdb.NewDB()) ret.tran = transkv.NewStore(memdb.NewDB()) ret.schema = reg.StoreSchema @@ -355,8 +352,12 @@ func (s *Store) Close() error { // Applies store upgrades to the DB contents. func (pr *prefixRegistry) migrate(store *Store, upgrades types.StoreUpgrades) error { - // branch state to allow mutation while iterating - branch := cachekv.NewStore(store) + // Get a view of current state to allow mutation while iterating + reader := store.stateDB.Reader() + scReader := reader + if store.StateCommitmentDB != nil { + scReader = store.StateCommitmentDB.Reader() + } for _, key := range upgrades.Deleted { sst, ix, err := pr.storeInfo(key) @@ -369,13 +370,27 @@ func (pr *prefixRegistry) migrate(store *Store, upgrades types.StoreUpgrades) er pr.reserved = append(pr.reserved[:ix], pr.reserved[ix+1:]...) delete(pr.StoreSchema, key) - sub := prefix.NewStore(store, []byte(key)) - subbranch := prefix.NewStore(branch, []byte(key)) - it := sub.Iterator(nil, nil) - for ; it.Valid(); it.Next() { - subbranch.Delete(it.Key()) + pfx := substorePrefix(key) + subReader := prefixdb.NewPrefixReader(reader, pfx) + it, err := subReader.Iterator(nil, nil) + if err != nil { + return err + } + for it.Next() { + store.stateTxn.Delete(it.Key()) } it.Close() + if store.StateCommitmentDB != nil { + subReader = prefixdb.NewPrefixReader(scReader, pfx) + it, err = subReader.Iterator(nil, nil) + if err != nil { + return err + } + for it.Next() { + store.stateCommitmentTxn.Delete(it.Key()) + } + it.Close() + } } for _, rename := range upgrades.Renamed { sst, ix, err := pr.storeInfo(rename.OldKey) @@ -392,15 +407,31 @@ func (pr *prefixRegistry) migrate(store *Store, upgrades types.StoreUpgrades) er return err } - sub := prefix.NewStore(store, []byte(rename.OldKey)) - subbranch := prefix.NewStore(branch, []byte(rename.NewKey)) - it := sub.Iterator(nil, nil) - for ; it.Valid(); it.Next() { - subbranch.Set(it.Key(), it.Value()) + oldPrefix := substorePrefix(rename.OldKey) + newPrefix := substorePrefix(rename.NewKey) + subReader := prefixdb.NewPrefixReader(reader, oldPrefix) + subWriter := prefixdb.NewPrefixWriter(store.stateTxn, newPrefix) + it, err := subReader.Iterator(nil, nil) + if err != nil { + return err + } + for it.Next() { + subWriter.Set(it.Key(), it.Value()) } it.Close() + if store.StateCommitmentDB != nil { + subReader = prefixdb.NewPrefixReader(scReader, oldPrefix) + subWriter = prefixdb.NewPrefixWriter(store.stateCommitmentTxn, newPrefix) + it, err = subReader.Iterator(nil, nil) + if err != nil { + return err + } + for it.Next() { + subWriter.Set(it.Key(), it.Value()) + } + it.Close() + } } - branch.Write() for _, key := range upgrades.Added { err := pr.ReservePrefix(key, types.StoreTypePersistent) @@ -411,12 +442,85 @@ func (pr *prefixRegistry) migrate(store *Store, upgrades types.StoreUpgrades) er return nil } -func (rs *Store) GetKVStore(key types.StoreKey) types.KVStore { - return rs.generic().getStore(key.Name()) +func substorePrefix(key string) []byte { + return append(contentPrefix, key...) +} + +func (rs *Store) GetKVStore(skey types.StoreKey) types.KVStore { + key := skey.Name() + var sub types.KVStore + if typ, has := rs.schema[key]; has { + switch typ { + case types.StoreTypeMemory: + sub = rs.mem + case types.StoreTypeTransient: + sub = rs.tran + } + if sub != nil { + if cached, has := rs.npSubstoreCache[key]; has { + return cached + } + ret := prefix.NewStore(sub, []byte(key)) + rs.npSubstoreCache[key] = ret + return ret + } + } else { + panic(ErrStoreNotFound(key)) + } + if cached, has := rs.substoreCache[key]; has { + return cached + } + ret, err := rs.getSubstore(key) + if err != nil { + panic(err) + } + rs.substoreCache[key] = ret + return ret +} + +// Gets a persistent substore +func (rs *Store) getSubstore(key string) (*substore, error) { + pfx := substorePrefix(key) + stateRW := prefixdb.NewPrefixReadWriter(rs.stateTxn, pfx) + stateCommitmentRW := prefixdb.NewPrefixReadWriter(rs.stateCommitmentTxn, pfx) + var stateCommitmentStore *smt.Store + + rootHash, err := stateRW.Get(substoreMerkleRootKey) + if err != nil { + return nil, err + } + if rootHash != nil { + stateCommitmentStore = loadSMT(stateCommitmentRW, rootHash) + } else { + merkleNodes := prefixdb.NewPrefixReadWriter(stateCommitmentRW, merkleNodePrefix) + merkleValues := prefixdb.NewPrefixReadWriter(stateCommitmentRW, merkleValuePrefix) + stateCommitmentStore = smt.NewStore(merkleNodes, merkleValues) + } + + return &substore{ + root: rs, + key: key, + dataBucket: prefixdb.NewPrefixReadWriter(stateRW, dataPrefix), + indexBucket: prefixdb.NewPrefixReadWriter(stateRW, indexPrefix), + stateCommitmentStore: stateCommitmentStore, + }, nil +} + +// resets a substore's state after commit (stateTxn discarded) +func (s *substore) refresh(rootHash []byte) { + pfx := substorePrefix(s.key) + stateRW := prefixdb.NewPrefixReadWriter(s.root.stateTxn, pfx) + stateCommitmentRW := prefixdb.NewPrefixReadWriter(s.root.stateCommitmentTxn, pfx) + s.dataBucket = prefixdb.NewPrefixReadWriter(stateRW, dataPrefix) + s.indexBucket = prefixdb.NewPrefixReadWriter(stateRW, indexPrefix) + s.stateCommitmentStore = loadSMT(stateCommitmentRW, rootHash) } // Commit implements Committer. func (s *Store) Commit() types.CommitID { + s.mtx.Lock() + defer s.mtx.Unlock() + versions, err := s.stateDB.Versions() if err != nil { panic(err) @@ -450,18 +554,40 @@ func (s *Store) Commit() types.CommitID { } s.tran.Commit() - return *cid } +func (s *Store) updateMerkleRoots() (ret map[string][]byte, err error) { + ret = map[string][]byte{} + for key, _ := range s.schema { + sub, has := s.substoreCache[key] + if !has { + sub, err = s.getSubstore(key) + if err != nil { + return + } + } + rootHash := sub.stateCommitmentStore.Root() + ret[key] = rootHash + pfx := substorePrefix(key) + stateW := prefixdb.NewPrefixReadWriter(s.stateTxn, pfx) + if err = stateW.Set(substoreMerkleRootKey, rootHash); err != nil { + return + } + } + return +} + func (s *Store) commit(target uint64) (id *types.CommitID, err error) { - root := s.stateCommitmentStore.Root() - err = s.stateTxn.Set(merkleRootKey, root) + rootHashes, err := s.updateMerkleRoots() if err != nil { return } - err = s.stateTxn.Commit() - if err != nil { + rootHash := sdkmaps.HashFromMap(rootHashes) + if err = s.stateTxn.Set(merkleRootKey, rootHash); err != nil { + return + } + if err = s.stateTxn.Commit(); err != nil { return } defer func() { @@ -484,6 +610,7 @@ func (s *Store) commit(target uint64) (id *types.CommitID, err error) { // If DBs are not separate, StateCommitment state has been commmitted & snapshotted if s.StateCommitmentDB != nil { + // if any error is encountered henceforth, we must revert the state and SC dbs defer func() { if err != nil { if delerr := s.stateDB.DeleteVersion(target); delerr != nil { @@ -510,12 +637,13 @@ func (s *Store) commit(target uint64) (id *types.CommitID, err error) { } s.stateTxn = stateTxn - s.dataBucket = prefixdb.NewPrefixReadWriter(stateTxn, dataPrefix) - s.indexBucket = prefixdb.NewPrefixReadWriter(stateTxn, indexPrefix) s.stateCommitmentTxn = stateCommitmentTxn - s.stateCommitmentStore = loadSMT(stateCommitmentTxn, root) + // the state on all living substores must be refreshed + for key, sub := range s.substoreCache { + sub.refresh(rootHashes[key]) + } - return &types.CommitID{Version: int64(target), Hash: root}, nil + return &types.CommitID{Version: int64(target), Hash: rootHash}, nil } // LastCommitID implements Committer. @@ -547,10 +675,8 @@ func (rs *Store) GetVersion(version int64) (types.BasicRootStore, error) { func (rs *Store) CacheRootStore() types.CacheRootStore { return &cacheStore{ - CacheKVStore: cachekv.NewStore(rs), - mem: cachekv.NewStore(rs.mem), - tran: cachekv.NewStore(rs.tran), - schema: rs.schema, + source: rs, + substores: map[string]types.CacheKVStore{}, listenerMixin: &listenerMixin{}, traceMixin: &traceMixin{}, } @@ -607,20 +733,23 @@ func (rs *Store) Query(req abci.RequestQuery) (res abci.ResponseQuery) { storeName, subpath, err := parsePath(req.Path) if err != nil { - return sdkerrors.QueryResult(err, false) + return sdkerrors.QueryResult(sdkerrors.Wrapf(err, "failed to parse path"), false) } view, err := rs.getView(height) if err != nil { if errors.Is(err, dbm.ErrVersionDoesNotExist) { err = sdkerrors.ErrInvalidHeight } - return sdkerrors.QueryResult(err, false) + return sdkerrors.QueryResult(sdkerrors.Wrapf(err, "failed to access height"), false) } - substore := view.generic().getStore(storeName) - if substore == nil { + if _, has := rs.schema[storeName]; !has { return sdkerrors.QueryResult(sdkerrors.Wrapf(sdkerrors.ErrUnknownRequest, "no such store: %s", storeName), false) } + substore, err := view.getSubstore(storeName) + if err != nil { + return sdkerrors.QueryResult(sdkerrors.Wrapf(err, "failed to access store: %s", storeName), false) + } switch subpath { case "/key": @@ -631,7 +760,7 @@ func (rs *Store) Query(req abci.RequestQuery) (res abci.ResponseQuery) { break } // res.ProofOps, err = view.prove(storeName, res.Key) - res.ProofOps, err = view.stateCommitmentStore.GetProof([]byte(storeName + string(res.Key))) + res.ProofOps, err = substore.stateCommitmentStore.GetProof([]byte(storeName + string(res.Key))) if err != nil { return sdkerrors.QueryResult(fmt.Errorf("Merkle proof creation failed for key: %v", res.Key), false) //nolint: stylecheck // proper name } @@ -665,49 +794,37 @@ func (rs *Store) Query(req abci.RequestQuery) (res abci.ResponseQuery) { return res } -func (rs *Store) generic() rootGeneric { return rootGeneric{rs.schema, rs, rs.mem, rs.tran} } - -func (store rootGeneric) getStore(key string) types.KVStore { - var sub types.KVStore - if typ, ok := store.schema[key]; ok { - switch typ { - case types.StoreTypePersistent: - sub = store.persist - case types.StoreTypeMemory: - sub = store.mem - case types.StoreTypeTransient: - sub = store.tran - } - } - if sub == nil { - panic(fmt.Errorf("store does not exist for key: %s", key)) +func (cs *cacheStore) GetKVStore(key types.StoreKey) types.KVStore { + ret, has := cs.substores[key.Name()] + if has { + return ret } - return prefix.NewStore(sub, []byte(key)) -} - -func (rs *cacheStore) GetKVStore(key types.StoreKey) types.KVStore { - return rs.generic().getStore(key.Name()) + ret = cachekv.NewStore(cs.source.GetKVStore(key)) + cs.substores[key.Name()] = ret + return ret } -func (rs *cacheStore) Write() { - rs.CacheKVStore.Write() - rs.mem.Write() - rs.tran.Write() +func (cs *cacheStore) Write() { + for _, sub := range cs.substores { + sub.Write() + } } // Recursively wraps the CacheRootStore in another cache store. -func (rs *cacheStore) CacheRootStore() types.CacheRootStore { +func (cs *cacheStore) CacheRootStore() types.CacheRootStore { return &cacheStore{ - CacheKVStore: cachekv.NewStore(rs), - mem: cachekv.NewStore(rs.mem), - tran: cachekv.NewStore(rs.tran), - schema: rs.schema, + source: cs, + substores: map[string]types.CacheKVStore{}, listenerMixin: &listenerMixin{}, traceMixin: &traceMixin{}, } } -func (rs *cacheStore) generic() rootGeneric { return rootGeneric{rs.schema, rs, rs.mem, rs.tran} } +func loadSMT(stateCommitmentTxn dbm.DBReadWriter, root []byte) *smt.Store { + merkleNodes := prefixdb.NewPrefixReadWriter(stateCommitmentTxn, merkleNodePrefix) + merkleValues := prefixdb.NewPrefixReadWriter(stateCommitmentTxn, merkleValuePrefix) + return smt.LoadStore(merkleNodes, merkleValues, root) +} // Returns closest index and whether it's a match func binarySearch(hay []string, ndl string) (int, bool) { @@ -765,7 +882,7 @@ func (pr *prefixRegistry) ReservePrefix(key string, typ types.StoreType) error { } func (lreg *listenerMixin) AddListeners(key types.StoreKey, listeners []types.WriteListener) { - if ls, ok := lreg.listeners[key]; ok { + if ls, has := lreg.listeners[key]; has { lreg.listeners[key] = append(ls, listeners...) } else { lreg.listeners[key] = listeners @@ -774,7 +891,7 @@ func (lreg *listenerMixin) AddListeners(key types.StoreKey, listeners []types.Wr // ListeningEnabled returns if listening is enabled for a specific KVStore func (lreg *listenerMixin) ListeningEnabled(key types.StoreKey) bool { - if ls, ok := lreg.listeners[key]; ok { + if ls, has := lreg.listeners[key]; has { return len(ls) != 0 } return false @@ -790,6 +907,9 @@ func (treg *traceMixin) SetTraceContext(tc types.TraceContext) { treg.TraceContext = tc } +func (s *Store) GetPruning() types.PruningOptions { return s.Pruning } +func (s *Store) SetPruning(po types.PruningOptions) { s.Pruning = po } + func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser, ready chan<- struct{}) error { return nil } diff --git a/store/v2/root/store_test.go b/store/v2/root/store_test.go index d2c9a38cf278..b7078fcd5384 100644 --- a/store/v2/root/store_test.go +++ b/store/v2/root/store_test.go @@ -56,7 +56,7 @@ func newSubStoreWithData(t *testing.T, db dbm.DBConnection, storeData map[string } func TestGetSetHasDelete(t *testing.T) { - root, store := newSubStoreWithData(t, memdb.NewDB(), alohaData) + _, store := newSubStoreWithData(t, memdb.NewDB(), alohaData) key := "hello" exists := store.Has([]byte(key)) @@ -81,9 +81,11 @@ func TestGetSetHasDelete(t *testing.T) { require.Panics(t, func() { store.Set(nil, []byte("value")) }, "Set(nil key) should panic") require.Panics(t, func() { store.Set([]byte{}, []byte("value")) }, "Set(empty key) should panic") require.Panics(t, func() { store.Set([]byte("key"), nil) }, "Set(nil value) should panic") - root.indexBucket = rwCrudFails{root.indexBucket} - require.Panics(t, func() { store.Set([]byte("key"), []byte("value")) }, - "Set() when index fails should panic") + sub := store.(*substore) + sub.indexBucket = rwCrudFails{sub.indexBucket, nil} + require.Panics(t, func() { + store.Set([]byte("key"), []byte("value")) + }, "Set() when index fails should panic") } func TestConstructors(t *testing.T) { @@ -125,13 +127,15 @@ func TestConstructors(t *testing.T) { merkledb.Close() // can't load existing store when we can't access the latest Merkle root hash - store, err = NewStore(db, DefaultStoreConfig()) + store, err = NewStore(db, simpleStoreConfig(t)) require.NoError(t, err) store.Commit() require.NoError(t, store.Close()) // ...whether because root is misssing w = db.Writer() - w.Delete(merkleRootKey) + s1RootKey := append(contentPrefix, substorePrefix(skey_1.Name())...) + s1RootKey = append(s1RootKey, merkleRootKey...) + w.Delete(s1RootKey) w.Commit() db.SaveNextVersion() store, err = NewStore(db, DefaultStoreConfig()) @@ -205,23 +209,30 @@ func TestIterators(t *testing.T) { func TestCommit(t *testing.T) { testBasic := func(opts StoreConfig) { // Sanity test for Merkle hashing - store, err := NewStore(memdb.NewDB(), opts) + db := memdb.NewDB() + store, err := NewStore(db, opts) require.NoError(t, err) require.Zero(t, store.LastCommitID()) idNew := store.Commit() - store.Set([]byte{0}, []byte{0}) + s1 := store.GetKVStore(skey_1) + s1.Set([]byte{0}, []byte{0}) idOne := store.Commit() require.Equal(t, idNew.Version+1, idOne.Version) require.NotEqual(t, idNew.Hash, idOne.Hash) - // Hash of emptied store is same as new store - store.Delete([]byte{0}) - idEmptied := store.Commit() - require.Equal(t, idNew.Hash, idEmptied.Hash) - - previd := idEmptied + // // Hash of emptied store is same as new store + // opts.Upgrades = []types.StoreUpgrades{ + // types.StoreUpgrades{Deleted: []string{skey_1.Name()}}, + // } + // store.Close() + // store, err = NewStore(db, opts) + // require.NoError(t, err) + // idEmptied := store.Commit() + // require.Equal(t, idNew.Hash, idEmptied.Hash) + + previd := idOne for i := byte(1); i < 5; i++ { - store.Set([]byte{i}, []byte{i}) + s1.Set([]byte{i}, []byte{i}) id := store.Commit() lastid := store.LastCommitID() require.Equal(t, id.Hash, lastid.Hash) @@ -243,8 +254,8 @@ func TestCommit(t *testing.T) { if db == nil { db = store.stateDB } - - store.Set([]byte{0}, []byte{0}) + s1 := store.GetKVStore(skey_1) + s1.Set([]byte{0}, []byte{0}) require.Panics(t, func() { store.Commit() }) require.NoError(t, store.Close()) @@ -257,7 +268,8 @@ func TestCommit(t *testing.T) { store, err := NewStore(db, opts) require.NoError(t, err) - require.Nil(t, store.Get([]byte{0})) + s1 = store.GetKVStore(skey_1) + require.Nil(t, s1.Get([]byte{0})) require.NoError(t, store.Close()) } @@ -300,7 +312,7 @@ func TestCommit(t *testing.T) { t.Run("recover after stateTxn.Set error triggers failure", func(t *testing.T) { store, err := NewStore(memdb.NewDB(), opts) require.NoError(t, err) - store.stateTxn = rwCrudFails{store.stateTxn} + store.stateTxn = rwCrudFails{store.stateTxn, merkleRootKey} testFailedCommit(t, store, nil, opts) }) @@ -344,7 +356,7 @@ func TestCommit(t *testing.T) { store, err = NewStore(memdb.NewDB(), opts) require.NoError(t, err) store.Commit() - store.stateTxn = rwCrudFails{store.stateTxn} + store.stateTxn = rwCrudFails{store.stateTxn, nil} require.Panics(t, func() { store.LastCommitID() }) } @@ -370,11 +382,15 @@ func TestPruning(t *testing.T) { for tci, tc := range testCases { dbs := []dbm.DBConnection{memdb.NewDB(), memdb.NewDB()} - store, err := NewStore(dbs[0], StoreConfig{Pruning: tc.PruningOptions, StateCommitmentDB: dbs[1]}) + opts := simpleStoreConfig(t) + opts.Pruning = tc.PruningOptions + opts.StateCommitmentDB = dbs[1] + store, err := NewStore(dbs[0], opts) require.NoError(t, err) + s1 := store.GetKVStore(skey_1) for i := byte(1); i <= 10; i++ { - store.Set([]byte{i}, []byte{i}) + s1.Set([]byte{i}, []byte{i}) cid := store.Commit() latest := uint64(i) require.Equal(t, latest, uint64(cid.Version)) @@ -393,7 +409,6 @@ func TestPruning(t *testing.T) { // Test pruning interval // Save up to 20th version while checking history at specific version checkpoints - pruning := types.PruningOptions{0, 5, 10} testCheckPoints := map[uint64][]uint64{ 5: []uint64{1, 2, 3, 4, 5}, 10: []uint64{5, 10}, @@ -401,11 +416,13 @@ func TestPruning(t *testing.T) { 20: []uint64{5, 10, 15, 20}, } db := memdb.NewDB() - store, err := NewStore(db, StoreConfig{Pruning: pruning}) + opts := simpleStoreConfig(t) + opts.Pruning = types.PruningOptions{0, 5, 10} + store, err := NewStore(db, opts) require.NoError(t, err) for i := byte(1); i <= 20; i++ { - store.Set([]byte{i}, []byte{i}) + store.GetKVStore(skey_1).Set([]byte{i}, []byte{i}) cid := store.Commit() latest := uint64(i) require.Equal(t, latest, uint64(cid.Version)) @@ -654,7 +671,7 @@ func TestGetVersion(t *testing.T) { subview = view.GetKVStore(skey_1) require.NotNil(t, subview) - store.Delete([]byte{0}) + substore.Delete([]byte{0}) require.Equal(t, []byte{0}, subview.Get([]byte{0})) } @@ -791,5 +808,7 @@ func TestRootStoreMigration(t *testing.T) { require.NotNil(t, s3) require.Equal(t, v3, s3.Get(k3)) - require.Panics(t, func() { view.GetKVStore(skey_4) }) + require.Panics(t, func() { + view.GetKVStore(skey_4) + }) } diff --git a/store/v2/root/sub_store.go b/store/v2/root/sub_store.go new file mode 100644 index 000000000000..e11e8b0d5440 --- /dev/null +++ b/store/v2/root/sub_store.go @@ -0,0 +1,117 @@ +package root + +import ( + "crypto/sha256" + "io" + "sync" + + dbm "github.com/cosmos/cosmos-sdk/db" + dbutil "github.com/cosmos/cosmos-sdk/internal/db" + "github.com/cosmos/cosmos-sdk/store/cachekv" + "github.com/cosmos/cosmos-sdk/store/listenkv" + "github.com/cosmos/cosmos-sdk/store/tracekv" + "github.com/cosmos/cosmos-sdk/store/types" +) + +// Get implements KVStore. +func (s *substore) Get(key []byte) []byte { + s.root.mtx.RLock() + defer s.root.mtx.RUnlock() + + val, err := s.dataBucket.Get(key) + if err != nil { + panic(err) + } + return val +} + +// Has implements KVStore. +func (s *substore) Has(key []byte) bool { + s.root.mtx.RLock() + defer s.root.mtx.RUnlock() + + has, err := s.dataBucket.Has(key) + if err != nil { + panic(err) + } + return has +} + +// Set implements KVStore. +func (s *substore) Set(key, value []byte) { + s.root.mtx.Lock() + defer s.root.mtx.Unlock() + + err := s.dataBucket.Set(key, value) + if err != nil { + panic(err) + } + s.stateCommitmentStore.Set(key, value) + khash := sha256.Sum256(key) + err = s.indexBucket.Set(khash[:], key) + if err != nil { + panic(err) + } +} + +// Delete implements KVStore. +func (s *substore) Delete(key []byte) { + khash := sha256.Sum256(key) + s.root.mtx.Lock() + defer s.root.mtx.Unlock() + + s.stateCommitmentStore.Delete(key) + _ = s.indexBucket.Delete(khash[:]) + _ = s.dataBucket.Delete(key) +} + +type contentsIterator struct { + types.Iterator + locker sync.Locker +} + +func (s *substore) newSubstoreIterator(source dbm.Iterator) *contentsIterator { + locker := s.root.mtx.RLocker() + locker.Lock() + return &contentsIterator{dbutil.DBToStoreIterator(source), locker} +} + +func (it *contentsIterator) Close() error { + defer it.locker.Unlock() + return it.Iterator.Close() +} + +// Iterator implements KVStore. +func (s *substore) Iterator(start, end []byte) types.Iterator { + iter, err := s.dataBucket.Iterator(start, end) + if err != nil { + panic(err) + } + return s.newSubstoreIterator(iter) +} + +// ReverseIterator implements KVStore. +func (s *substore) ReverseIterator(start, end []byte) types.Iterator { + iter, err := s.dataBucket.ReverseIterator(start, end) + if err != nil { + panic(err) + } + return s.newSubstoreIterator(iter) +} + +// GetStoreType implements Store. +func (s *substore) GetStoreType() types.StoreType { + return types.StoreTypePersistent +} + +func (s *substore) CacheWrap() types.CacheWrap { + return cachekv.NewStore(s) +} + +func (s *substore) CacheWrapWithTrace(w io.Writer, tc types.TraceContext) types.CacheWrap { + return cachekv.NewStore(tracekv.NewStore(s, w, tc)) +} + +func (s *substore) CacheWrapWithListeners(storeKey types.StoreKey, listeners []types.WriteListener) types.CacheWrap { + return cachekv.NewStore(listenkv.NewStore(s, storeKey, listeners)) +} diff --git a/store/v2/root/test_util.go b/store/v2/root/test_util.go index f41b31115d41..2fcdbd2f09ed 100644 --- a/store/v2/root/test_util.go +++ b/store/v2/root/test_util.go @@ -1,6 +1,7 @@ package root import ( + "bytes" "errors" dbm "github.com/cosmos/cosmos-sdk/db" @@ -17,12 +18,15 @@ type dbVersionsIs struct { } type dbVersionsFails struct{ dbm.DBConnection } type rwCommitFails struct{ dbm.DBReadWriter } -type rwCrudFails struct{ dbm.DBReadWriter } +type rwCrudFails struct { + dbm.DBReadWriter + onKey []byte +} func (dbVersionsFails) Versions() (dbm.VersionSet, error) { return nil, errors.New("dbVersionsFails") } func (db dbVersionsIs) Versions() (dbm.VersionSet, error) { return db.vset, nil } func (db dbRWCrudFails) ReadWriter() dbm.DBReadWriter { - return rwCrudFails{db.DBConnection.ReadWriter()} + return rwCrudFails{db.DBConnection.ReadWriter(), nil} } func (dbSaveVersionFails) SaveVersion(uint64) error { return errors.New("dbSaveVersionFails") } func (dbDeleteVersionFails) DeleteVersion(uint64) error { return errors.New("dbDeleteVersionFails") } @@ -34,7 +38,27 @@ func (db dbRWCommitFails) ReadWriter() dbm.DBReadWriter { return rwCommitFails{db.MemDB.ReadWriter()} } -func (rwCrudFails) Get([]byte) ([]byte, error) { return nil, errors.New("rwCrudFails.Get") } -func (rwCrudFails) Has([]byte) (bool, error) { return false, errors.New("rwCrudFails.Has") } -func (rwCrudFails) Set([]byte, []byte) error { return errors.New("rwCrudFails.Set") } -func (rwCrudFails) Delete([]byte) error { return errors.New("rwCrudFails.Delete") } +func (rw rwCrudFails) Get(k []byte) ([]byte, error) { + if rw.onKey == nil || bytes.Equal(rw.onKey, k) { + return nil, errors.New("rwCrudFails.Get") + } + return rw.DBReadWriter.Get(k) +} +func (rw rwCrudFails) Has(k []byte) (bool, error) { + if rw.onKey == nil || bytes.Equal(rw.onKey, k) { + return false, errors.New("rwCrudFails.Has") + } + return rw.DBReadWriter.Has(k) +} +func (rw rwCrudFails) Set(k []byte, v []byte) error { + if rw.onKey == nil || bytes.Equal(rw.onKey, k) { + return errors.New("rwCrudFails.Set") + } + return rw.DBReadWriter.Set(k, v) +} +func (rw rwCrudFails) Delete(k []byte) error { + if rw.onKey == nil || bytes.Equal(rw.onKey, k) { + return errors.New("rwCrudFails.Delete") + } + return rw.DBReadWriter.Delete(k) +} diff --git a/store/v2/root/view_store.go b/store/v2/root/view_store.go index 7f75957f4d02..ae02296d6dd2 100644 --- a/store/v2/root/view_store.go +++ b/store/v2/root/view_store.go @@ -5,26 +5,24 @@ import ( "io" dbm "github.com/cosmos/cosmos-sdk/db" - "github.com/cosmos/cosmos-sdk/db/memdb" prefixdb "github.com/cosmos/cosmos-sdk/db/prefix" util "github.com/cosmos/cosmos-sdk/internal" + dbutil "github.com/cosmos/cosmos-sdk/internal/db" "github.com/cosmos/cosmos-sdk/store/cachekv" "github.com/cosmos/cosmos-sdk/store/listenkv" "github.com/cosmos/cosmos-sdk/store/tracekv" types "github.com/cosmos/cosmos-sdk/store/v2" - "github.com/cosmos/cosmos-sdk/store/v2/mem" "github.com/cosmos/cosmos-sdk/store/v2/smt" - transkv "github.com/cosmos/cosmos-sdk/store/v2/transient" ) var ErrReadOnly = errors.New("cannot modify read-only store") -func (s *viewStore) GetStateCommitmentStore() *smt.Store { +func (s *viewSubstore) GetStateCommitmentStore() *smt.Store { return s.stateCommitmentStore } // Get implements KVStore. -func (s *viewStore) Get(key []byte) []byte { +func (s *viewSubstore) Get(key []byte) []byte { val, err := s.dataBucket.Get(key) if err != nil { panic(err) @@ -33,7 +31,7 @@ func (s *viewStore) Get(key []byte) []byte { } // Has implements KVStore. -func (s *viewStore) Has(key []byte) bool { +func (s *viewSubstore) Has(key []byte) bool { has, err := s.dataBucket.Has(key) if err != nil { panic(err) @@ -42,47 +40,47 @@ func (s *viewStore) Has(key []byte) bool { } // Set implements KVStore. -func (s *viewStore) Set(key []byte, value []byte) { +func (s *viewSubstore) Set(key []byte, value []byte) { panic(ErrReadOnly) } // Delete implements KVStore. -func (s *viewStore) Delete(key []byte) { +func (s *viewSubstore) Delete(key []byte) { panic(ErrReadOnly) } // Iterator implements KVStore. -func (s *viewStore) Iterator(start, end []byte) types.Iterator { +func (s *viewSubstore) Iterator(start, end []byte) types.Iterator { iter, err := s.dataBucket.Iterator(start, end) if err != nil { panic(err) } - return newIterator(iter) + return dbutil.DBToStoreIterator(iter) } // ReverseIterator implements KVStore. -func (s *viewStore) ReverseIterator(start, end []byte) types.Iterator { +func (s *viewSubstore) ReverseIterator(start, end []byte) types.Iterator { iter, err := s.dataBucket.ReverseIterator(start, end) if err != nil { panic(err) } - return newIterator(iter) + return dbutil.DBToStoreIterator(iter) } // GetStoreType implements Store. -func (s *viewStore) GetStoreType() types.StoreType { +func (s *viewSubstore) GetStoreType() types.StoreType { return types.StoreTypePersistent } -func (st *viewStore) CacheWrap() types.CacheWrap { +func (st *viewSubstore) CacheWrap() types.CacheWrap { return cachekv.NewStore(st) } -func (st *viewStore) CacheWrapWithTrace(w io.Writer, tc types.TraceContext) types.CacheWrap { +func (st *viewSubstore) CacheWrapWithTrace(w io.Writer, tc types.TraceContext) types.CacheWrap { return cachekv.NewStore(tracekv.NewStore(st, w, tc)) } -func (st *viewStore) CacheWrapWithListeners(storeKey types.StoreKey, listeners []types.WriteListener) types.CacheWrap { +func (st *viewSubstore) CacheWrapWithListeners(storeKey types.StoreKey, listeners []types.WriteListener) types.CacheWrap { return cachekv.NewStore(listenkv.NewStore(st, storeKey, listeners)) } @@ -109,19 +107,8 @@ func (store *Store) getView(version int64) (ret *viewStore, err error) { } }() } - root, err := stateView.Get(merkleRootKey) - if err != nil { - return - } - ret = &viewStore{ - stateView: stateView, - dataBucket: prefixdb.NewPrefixReader(stateView, dataPrefix), - indexBucket: prefixdb.NewPrefixReader(stateView, indexPrefix), - stateCommitmentView: stateCommitmentView, - stateCommitmentStore: loadSMT(dbm.ReaderAsReadWriter(stateCommitmentView), root), - } // Now read this version's schema - schemaView := prefixdb.NewPrefixReader(ret.stateView, schemaPrefix) + schemaView := prefixdb.NewPrefixReader(stateView, schemaPrefix) defer func() { if err != nil { err = util.CombineErrors(err, schemaView.Discard(), "schemaView.Discard also failed") @@ -132,23 +119,50 @@ func (store *Store) getView(version int64) (ret *viewStore, err error) { return } // The migrated contents and schema are not committed until the next store.Commit - ret.schema = pr.StoreSchema + ret = &viewStore{ + stateView: stateView, + stateCommitmentView: stateCommitmentView, + substoreCache: map[string]*viewSubstore{}, + schema: pr.StoreSchema, + } return } -// if the schema indicates a mem/tran store, it's ignored -func (rv *viewStore) generic() rootGeneric { return rootGeneric{rv.schema, rv, nil, nil} } +func (vs *viewStore) GetKVStore(skey types.StoreKey) types.KVStore { + key := skey.Name() + if _, has := vs.schema[key]; !has { + panic(ErrStoreNotFound(key)) + } + if cached, has := vs.substoreCache[key]; has { + return cached + } + ret, err := vs.getSubstore(key) + if err != nil { + panic(err) + } + vs.substoreCache[key] = ret + return ret +} -func (rv *viewStore) GetKVStore(key types.StoreKey) types.KVStore { - return rv.generic().getStore(key.Name()) +func (vs *viewStore) getSubstore(key string) (*viewSubstore, error) { + pfx := substorePrefix(key) + stateR := prefixdb.NewPrefixReader(vs.stateView, pfx) + stateCommitmentR := prefixdb.NewPrefixReader(vs.stateCommitmentView, pfx) + rootHash, err := stateR.Get(merkleRootKey) + if err != nil { + return nil, err + } + return &viewSubstore{ + dataBucket: prefixdb.NewPrefixReader(stateR, dataPrefix), + indexBucket: prefixdb.NewPrefixReader(stateR, indexPrefix), + stateCommitmentStore: loadSMT(dbm.ReaderAsReadWriter(stateCommitmentR), rootHash), + }, nil } -func (rv *viewStore) CacheRootStore() types.CacheRootStore { +func (vs *viewStore) CacheRootStore() types.CacheRootStore { return &cacheStore{ - CacheKVStore: cachekv.NewStore(rv), - mem: cachekv.NewStore(mem.NewStore(memdb.NewDB())), - tran: cachekv.NewStore(transkv.NewStore(memdb.NewDB())), - schema: rv.schema, + source: vs, + substores: map[string]types.CacheKVStore{}, listenerMixin: &listenerMixin{}, traceMixin: &traceMixin{}, } From d889e075f8f00a6dd460a17ac504ee9bf8e252e9 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 18 Nov 2021 01:35:29 +0800 Subject: [PATCH 14/47] docs + changelog --- CHANGELOG.md | 1 + docs/core/store.md | 8 +++----- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ef67961d2a50..9b40475d5e58 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -54,6 +54,7 @@ Ref: https://keepachangelog.com/en/1.0.0/ * [\#10379](https://github.com/cosmos/cosmos-sdk/pull/10379) Add validation to `x/upgrade` CLI `software-upgrade` command `--plan-info` value. * [\#10561](https://github.com/cosmos/cosmos-sdk/pull/10561) Add configurable IAVL cache size to app.toml * [\10507](https://github.com/cosmos/cosmos-sdk/pull/10507) Add middleware for tx priority. +* [\#10430](https://github.com/cosmos/cosmos-sdk/pull/10430) ADR-040: Add `RootStore` implementation ### Improvements diff --git a/docs/core/store.md b/docs/core/store.md index 34fcb57f4e52..54c636ba2cf6 100644 --- a/docs/core/store.md +++ b/docs/core/store.md @@ -242,15 +242,13 @@ An interface providing only the basic CRUD functionality (`Get`, `Set`, `Has`, a ### Root Store -`root.RootStore` is the new default persistent store, which internally decouples the concerns of state storage and state commitment. Values are stored directly in the backing key-value database (the "storage" bucket), while the value's hash is mapped in a separate store which is able to generate a cryptographic commitment (the "state commitment" bucket, implmented with `smt.Store`). +`RootStore` is the new interface for the main client store, replacing the function of `MultiStore`. It internally decouples the concerns of state storage and state commitment: values are stored and read directly from the backing key-value database, but are also mapped in a logically separate *state-commitment* store which generates cryptographic proofs. -This can optionally be configured to use different backend databases for each bucket. - - +Implemented in `store/v2/root`. This can optionally be configured to use different backend databases for each bucket, e.g., `badgerdb` for the state storage DB and `memdb` for the state commitment DB. State commitment is implemented with an `smt.Store`. ### SMT Store -A `BasicKVStore` which is used to partially expose functions of an underlying store (for instance, to allow access to the commitment store in `flat.Store`). +Maps values into a Sparse Merkle Tree, and supports a `BasicKVStore` interface as well as methods for cryptographic proof generation. ## Next {hide} From b2007bfc6c3094a873c53205ecf108f6b66dd418 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 18 Nov 2021 11:20:45 +0800 Subject: [PATCH 15/47] Revert changes to prefix store --- store/prefix/store.go | 16 ++-------------- 1 file changed, 2 insertions(+), 14 deletions(-) diff --git a/store/prefix/store.go b/store/prefix/store.go index 941c920f48c7..295278a0a853 100644 --- a/store/prefix/store.go +++ b/store/prefix/store.go @@ -36,8 +36,8 @@ func cloneAppend(bz []byte, tail []byte) (res []byte) { } func (s Store) key(key []byte) (res []byte) { - if len(key) == 0 { - panic("nil or empty key on Store") + if key == nil { + panic("nil key on Store") } res = cloneAppend(s.prefix, key) return @@ -89,12 +89,6 @@ func (s Store) Delete(key []byte) { // Implements KVStore // Check https://github.com/tendermint/tendermint/blob/master/libs/db/prefix_db.go#L106 func (s Store) Iterator(start, end []byte) types.Iterator { - if start != nil && len(start) == 0 { - panic("empty start key") - } - if end != nil && len(end) == 0 { - panic("empty end key") - } newstart := cloneAppend(s.prefix, start) var newend []byte @@ -112,12 +106,6 @@ func (s Store) Iterator(start, end []byte) types.Iterator { // ReverseIterator implements KVStore // Check https://github.com/tendermint/tendermint/blob/master/libs/db/prefix_db.go#L129 func (s Store) ReverseIterator(start, end []byte) types.Iterator { - if start != nil && len(start) == 0 { - panic("empty start key") - } - if end != nil && len(end) == 0 { - panic("empty end key") - } newstart := cloneAppend(s.prefix, start) var newend []byte From f1caf0f2b86b07e8a64d9afdd74cca31736e8be1 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 18 Nov 2021 11:30:44 +0800 Subject: [PATCH 16/47] try to fix rocksdb test case for CI --- db/rocksdb/db_test.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/db/rocksdb/db_test.go b/db/rocksdb/db_test.go index 313bfde5b437..d28aa1cae5f8 100644 --- a/db/rocksdb/db_test.go +++ b/db/rocksdb/db_test.go @@ -2,6 +2,7 @@ package rocksdb import ( "os" + "path/filepath" "testing" "github.com/stretchr/testify/require" @@ -53,11 +54,11 @@ func TestRevertRecovery(t *testing.T) { require.NoError(t, txn.Set([]byte{2}, []byte{2})) require.NoError(t, txn.Commit()) - // make checkpoints dir temporarily unreadable to trigger an error - require.NoError(t, os.Chmod(db.checkpointsDir(), 0000)) + // move checkpoints dir temporarily to trigger an error + hideDir := filepath.Join(dir, "hide_checkpoints") + require.NoError(t, os.Rename(db.checkpointsDir(), hideDir)) require.Error(t, db.Revert()) - - require.NoError(t, os.Chmod(db.checkpointsDir(), 0755)) + require.NoError(t, os.Rename(hideDir, db.checkpointsDir())) db, err = NewDB(dir) require.NoError(t, err) } From d2f0adaeb090cd11bf84999507c13400c6dd149f Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 18 Nov 2021 11:50:57 +0800 Subject: [PATCH 17/47] add checks to testcase --- db/rocksdb/db_test.go | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/db/rocksdb/db_test.go b/db/rocksdb/db_test.go index d28aa1cae5f8..5e8bae425102 100644 --- a/db/rocksdb/db_test.go +++ b/db/rocksdb/db_test.go @@ -47,10 +47,12 @@ func TestRevertRecovery(t *testing.T) { dir := t.TempDir() db, err := NewDB(dir) require.NoError(t, err) - _, err = db.SaveNextVersion() - require.NoError(t, err) txn := db.Writer() require.NoError(t, txn.Set([]byte{1}, []byte{1})) + require.NoError(t, txn.Commit()) + _, err = db.SaveNextVersion() + require.NoError(t, err) + txn = db.Writer() require.NoError(t, txn.Set([]byte{2}, []byte{2})) require.NoError(t, txn.Commit()) @@ -59,6 +61,15 @@ func TestRevertRecovery(t *testing.T) { require.NoError(t, os.Rename(db.checkpointsDir(), hideDir)) require.Error(t, db.Revert()) require.NoError(t, os.Rename(hideDir, db.checkpointsDir())) + db, err = NewDB(dir) require.NoError(t, err) + view := db.Reader() + val, err := view.Get([]byte{1}) + require.NoError(t, err) + require.Equal(t, []byte{1}, val) + val, err = view.Get([]byte{2}) + require.NoError(t, err) + require.Nil(t, val) + view.Discard() } From 2ae149fec121ed77b84685a6e95b06acf8e58b57 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 18 Nov 2021 11:53:26 +0800 Subject: [PATCH 18/47] nit --- store/v2/root/store.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/store/v2/root/store.go b/store/v2/root/store.go index d7d2d07032d1..a2b0cc4d7954 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -50,9 +50,12 @@ var ( ErrVersionDoesNotExist = errors.New("version does not exist") ErrMaximumHeight = errors.New("maximum block height reached") - ErrStoreNotFound = func(skey string) error { return fmt.Errorf("store does not exist for key: %s", skey) } ) +func ErrStoreNotFound(skey string) error { + return fmt.Errorf("store does not exist for key: %s", skey) +} + // StoreConfig is used to define a schema and pass options to the RootStore constructor. type StoreConfig struct { // Version pruning options for backing DBs. From 62f1fc7054c571bcee36003c77ac7a1c9bfc8f41 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 18 Nov 2021 22:07:44 +0800 Subject: [PATCH 19/47] rootstore cleanup --- store/v2/root/store.go | 52 +++++++++++++++++++----------------------- 1 file changed, 24 insertions(+), 28 deletions(-) diff --git a/store/v2/root/store.go b/store/v2/root/store.go index a2b0cc4d7954..e4e8ac72051f 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -80,30 +80,27 @@ type StoreSchema map[string]types.StoreType type Store struct { stateDB dbm.DBConnection stateTxn dbm.DBReadWriter + StateCommitmentDB dbm.DBConnection stateCommitmentTxn dbm.DBReadWriter - Pruning types.PruningOptions - InitialVersion uint64 - StateCommitmentDB dbm.DBConnection - schema StoreSchema mem *mem.Store tran *transkv.Store + mtx sync.RWMutex + + Pruning types.PruningOptions + InitialVersion uint64 *listenerMixin *traceMixin + PersistentCache types.RootStorePersistentCache substoreCache map[string]*substore npSubstoreCache map[string]types.KVStore - - mtx sync.RWMutex - loadExisting bool - - PersistentCache types.RootStorePersistentCache } type substore struct { root *Store - key string + name string dataBucket dbm.DBReadWriter indexBucket dbm.DBReadWriter stateCommitmentStore *smt.Store @@ -224,14 +221,12 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { if err != nil { return } - loadExisting := false // If the DB is not empty, attempt to load existing data if saved := versions.Count(); saved != 0 { if opts.InitialVersion != 0 && versions.Last() < opts.InitialVersion { return nil, fmt.Errorf("latest saved version is less than initial version: %v < %v", versions.Last(), opts.InitialVersion) } - loadExisting = true } err = db.Revert() if err != nil { @@ -277,7 +272,6 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { Pruning: opts.Pruning, InitialVersion: opts.InitialVersion, - loadExisting: loadExisting, } // Now load the substore schema @@ -502,7 +496,7 @@ func (rs *Store) getSubstore(key string) (*substore, error) { return &substore{ root: rs, - key: key, + name: key, dataBucket: prefixdb.NewPrefixReadWriter(stateRW, dataPrefix), indexBucket: prefixdb.NewPrefixReadWriter(stateRW, indexPrefix), stateCommitmentStore: stateCommitmentStore, @@ -511,7 +505,7 @@ func (rs *Store) getSubstore(key string) (*substore, error) { // resets a substore's state after commit (stateTxn discarded) func (s *substore) refresh(rootHash []byte) { - pfx := substorePrefix(s.key) + pfx := substorePrefix(s.name) stateRW := prefixdb.NewPrefixReadWriter(s.root.stateTxn, pfx) stateCommitmentRW := prefixdb.NewPrefixReadWriter(s.root.stateCommitmentTxn, pfx) s.dataBucket = prefixdb.NewPrefixReadWriter(stateRW, dataPrefix) @@ -560,7 +554,7 @@ func (s *Store) Commit() types.CommitID { return *cid } -func (s *Store) updateMerkleRoots() (ret map[string][]byte, err error) { +func (s *Store) getMerkleRoots() (ret map[string][]byte, err error) { ret = map[string][]byte{} for key, _ := range s.schema { sub, has := s.substoreCache[key] @@ -570,23 +564,25 @@ func (s *Store) updateMerkleRoots() (ret map[string][]byte, err error) { return } } - rootHash := sub.stateCommitmentStore.Root() - ret[key] = rootHash - pfx := substorePrefix(key) - stateW := prefixdb.NewPrefixReadWriter(s.stateTxn, pfx) - if err = stateW.Set(substoreMerkleRootKey, rootHash); err != nil { - return - } + ret[key] = sub.stateCommitmentStore.Root() } return } func (s *Store) commit(target uint64) (id *types.CommitID, err error) { - rootHashes, err := s.updateMerkleRoots() + storeHashes, err := s.getMerkleRoots() if err != nil { return } - rootHash := sdkmaps.HashFromMap(rootHashes) + // Update substore Merkle roots + for key, storeHash := range storeHashes { + pfx := substorePrefix(key) + stateW := prefixdb.NewPrefixReadWriter(s.stateTxn, pfx) + if err = stateW.Set(substoreMerkleRootKey, storeHash); err != nil { + return + } + } + rootHash := sdkmaps.HashFromMap(storeHashes) if err = s.stateTxn.Set(merkleRootKey, rootHash); err != nil { return } @@ -611,7 +607,7 @@ func (s *Store) commit(target uint64) (id *types.CommitID, err error) { }() stateCommitmentTxn := stateTxn - // If DBs are not separate, StateCommitment state has been commmitted & snapshotted + // If DBs are not separate, StateCommitment state has been committed & snapshotted if s.StateCommitmentDB != nil { // if any error is encountered henceforth, we must revert the state and SC dbs defer func() { @@ -643,7 +639,7 @@ func (s *Store) commit(target uint64) (id *types.CommitID, err error) { s.stateCommitmentTxn = stateCommitmentTxn // the state on all living substores must be refreshed for key, sub := range s.substoreCache { - sub.refresh(rootHashes[key]) + sub.refresh(storeHashes[key]) } return &types.CommitID{Version: int64(target), Hash: rootHash}, nil @@ -762,7 +758,7 @@ func (rs *Store) Query(req abci.RequestQuery) (res abci.ResponseQuery) { if !req.Prove { break } - // res.ProofOps, err = view.prove(storeName, res.Key) + // TODO: actual IBC compatible proof. This is a placeholder so unit tests can pass res.ProofOps, err = substore.stateCommitmentStore.GetProof([]byte(storeName + string(res.Key))) if err != nil { return sdkerrors.QueryResult(fmt.Errorf("Merkle proof creation failed for key: %v", res.Key), false) //nolint: stylecheck // proper name From 2677b2ab91886214fa00e9ab300a86fa6743a61b Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 25 Nov 2021 01:50:24 +0800 Subject: [PATCH 20/47] cleanup, comments --- db/memdb/db.go | 4 +++- store/README.md | 2 +- store/v2/root/store_test.go | 24 ++++++++++++------------ 3 files changed, 16 insertions(+), 14 deletions(-) diff --git a/db/memdb/db.go b/db/memdb/db.go index b656d60edadb..ddae7f74db89 100644 --- a/db/memdb/db.go +++ b/db/memdb/db.go @@ -25,7 +25,9 @@ const ( // // Versioning is implemented by maintaining references to copy-on-write clones of the backing btree. // -// TODO: Currently transactions do not detect write conflicts, so writers cannot be used concurrently. +// Note: Currently, transactions do not detect write conflicts, so multiple writers cannot be +// safely committed to overlapping domains. Because of this, the number of open writers is +// limited to 1. type MemDB struct { btree *btree.BTree // Main contents mtx sync.RWMutex // Guards version history diff --git a/store/README.md b/store/README.md index 8541f69673ad..578d1e5f8dd1 100644 --- a/store/README.md +++ b/store/README.md @@ -13,7 +13,7 @@ type Store struct { ### Get -`Store.Get()` checks `Store.cache` first in order to find if there is any cached value associated with the key. If the value exists, the function returns it. If not, the function calls `Store.parent.Get()`, sets the key-value pair to the `Store.cache`, and returns it. +`Store.Get()` checks `Store.cache` first in order to find if there is any cached value associated with the key. If the value exists, the function returns it. If not, the function calls `Store.parent.Get()`, sets the key-value pair in the `Store.cache`, and returns it. ### Set diff --git a/store/v2/root/store_test.go b/store/v2/root/store_test.go index b7078fcd5384..385c8b162fdb 100644 --- a/store/v2/root/store_test.go +++ b/store/v2/root/store_test.go @@ -441,9 +441,9 @@ func TestPruning(t *testing.T) { } } -func TestQuery(t *testing.T) { - path := func(skey types.StoreKey, endp string) string { return "/" + skey.Name() + endp } +func queryPath(skey types.StoreKey, endp string) string { return "/" + skey.Name() + endp } +func TestQuery(t *testing.T) { k1, v1 := []byte("k1"), []byte("v1") k2, v2 := []byte("k2"), []byte("v2") v3 := []byte("v3") @@ -476,8 +476,8 @@ func TestQuery(t *testing.T) { require.NoError(t, err) cid := store.Commit() ver := cid.Version - query := abci.RequestQuery{Path: path(skey_1, "/key"), Data: k1, Height: ver} - querySub := abci.RequestQuery{Path: path(skey_1, "/subspace"), Data: ksub, Height: ver} + query := abci.RequestQuery{Path: queryPath(skey_1, "/key"), Data: k1, Height: ver} + querySub := abci.RequestQuery{Path: queryPath(skey_1, "/subspace"), Data: ksub, Height: ver} // query subspace before anything set qres := store.Query(querySub) @@ -527,7 +527,7 @@ func TestQuery(t *testing.T) { require.True(t, qres.IsOK(), qres.Log) require.Equal(t, v3, qres.Value) - query2 := abci.RequestQuery{Path: path(skey_1, "/key"), Data: k2, Height: cid.Version} + query2 := abci.RequestQuery{Path: queryPath(skey_1, "/key"), Data: k2, Height: cid.Version} qres = store.Query(query2) require.True(t, qres.IsOK(), qres.Log) require.Equal(t, v2, qres.Value) @@ -538,7 +538,7 @@ func TestQuery(t *testing.T) { require.Equal(t, valExpSub2, qres.Value) // default (height 0) will show latest-1 - query0 := abci.RequestQuery{Path: path(skey_1, "/key"), Data: k1} + query0 := abci.RequestQuery{Path: queryPath(skey_1, "/key"), Data: k1} qres = store.Query(query0) require.True(t, qres.IsOK(), qres.Log) require.Equal(t, v1, qres.Value) @@ -570,24 +570,24 @@ func TestQuery(t *testing.T) { store2.Close() // query with a nil or empty key fails - badquery := abci.RequestQuery{Path: path(skey_1, "/key"), Data: []byte{}} + badquery := abci.RequestQuery{Path: queryPath(skey_1, "/key"), Data: []byte{}} qres = store.Query(badquery) require.True(t, qres.IsErr()) badquery.Data = nil qres = store.Query(badquery) require.True(t, qres.IsErr()) // querying an invalid height will fail - badquery = abci.RequestQuery{Path: path(skey_1, "/key"), Data: k1, Height: store.LastCommitID().Version + 1} + badquery = abci.RequestQuery{Path: queryPath(skey_1, "/key"), Data: k1, Height: store.LastCommitID().Version + 1} qres = store.Query(badquery) require.True(t, qres.IsErr()) // or an invalid path - badquery = abci.RequestQuery{Path: path(skey_1, "/badpath"), Data: k1} + badquery = abci.RequestQuery{Path: queryPath(skey_1, "/badpath"), Data: k1} qres = store.Query(badquery) require.True(t, qres.IsErr()) // test that proofs are generated with single and separate DBs testProve := func() { - queryProve0 := abci.RequestQuery{Path: path(skey_1, "/key"), Data: k1, Prove: true} + queryProve0 := abci.RequestQuery{Path: queryPath(skey_1, "/key"), Data: k1, Prove: true} qres = store.Query(queryProve0) require.True(t, qres.IsOK(), qres.Log) require.Equal(t, v1, qres.Value) @@ -734,7 +734,7 @@ func TestRootStoreMigration(t *testing.T) { // store3 is gone require.Panics(t, func() { s3 = restore.GetKVStore(skey_3) }) - // store4 is mounted, with empty data + // store4 is valid s4 := restore.GetKVStore(skey_4) require.NotNil(t, s4) @@ -750,7 +750,7 @@ func TestRootStoreMigration(t *testing.T) { k4, v4 := []byte("fourth"), []byte("created") s4.Set(k4, v4) - // store2 is no longer mounted + // store2 is no longer valid require.Panics(t, func() { restore.GetKVStore(skey_2) }) // restore2 has the old data From 0d5c7692ab50a36bff6c70d43fdfcc2a62817f79 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 25 Nov 2021 16:03:41 +0800 Subject: [PATCH 21/47] PR revisions - godoc, cleanup --- db/prefix/prefix.go | 10 ++++++++-- store/v2/dbadapter/store.go | 3 --- store/v2/dbadapter/store_test.go | 7 +++---- 3 files changed, 11 insertions(+), 9 deletions(-) diff --git a/db/prefix/prefix.go b/db/prefix/prefix.go index af4ae2aa10fd..66f90ef7b24e 100644 --- a/db/prefix/prefix.go +++ b/db/prefix/prefix.go @@ -24,6 +24,8 @@ var _ dbm.DBReader = (*prefixR)(nil) var _ dbm.DBReadWriter = (*prefixRW)(nil) var _ dbm.DBWriter = (*prefixW)(nil) +// NewPrefixReader returns a DBReader that only has access to the subset of DB keys +// that contain the given prefix. func NewPrefixReader(db dbm.DBReader, prefix []byte) prefixR { return prefixR{ prefix: prefix, @@ -31,6 +33,8 @@ func NewPrefixReader(db dbm.DBReader, prefix []byte) prefixR { } } +// NewPrefixReadWriter returns a DBReader that only has access to the subset of DB keys +// that contain the given prefix. func NewPrefixReadWriter(db dbm.DBReadWriter, prefix []byte) prefixRW { return prefixRW{ prefix: prefix, @@ -38,6 +42,8 @@ func NewPrefixReadWriter(db dbm.DBReadWriter, prefix []byte) prefixRW { } } +// NewPrefixWriter returns a DBWriter that reads/writes only from the subset of DB keys +// that contain the given prefix func NewPrefixWriter(db dbm.DBWriter, prefix []byte) prefixW { return prefixW{ prefix: prefix, @@ -156,7 +162,7 @@ func (pdb prefixW) Set(key []byte, value []byte) error { return pdb.db.Set(prefixed(pdb.prefix, key), value) } -// Delete implements DBReadWriter. +// Delete implements DBWriter. func (pdb prefixW) Delete(key []byte) error { if len(key) == 0 { return dbm.ErrKeyEmpty @@ -164,7 +170,7 @@ func (pdb prefixW) Delete(key []byte) error { return pdb.db.Delete(prefixed(pdb.prefix, key)) } -// Close implements DBReadWriter. +// Close implements DBWriter. func (pdb prefixW) Commit() error { return pdb.db.Commit() } // Discard implements DBReadWriter. diff --git a/store/v2/dbadapter/store.go b/store/v2/dbadapter/store.go index fa59d90fc5f7..1cbd6c83585b 100644 --- a/store/v2/dbadapter/store.go +++ b/store/v2/dbadapter/store.go @@ -3,8 +3,6 @@ package dbadapter import ( "io" - // dbm "github.com/tendermint/tm-db" - dbm "github.com/cosmos/cosmos-sdk/db" dbutil "github.com/cosmos/cosmos-sdk/internal/db" "github.com/cosmos/cosmos-sdk/store/cachekv" @@ -13,7 +11,6 @@ import ( "github.com/cosmos/cosmos-sdk/store/types" ) -// dbm.DB implements KVStore so we can CacheKVStore it. var _ types.KVStore = Store{} // Wrapper type for dbm.Db with implementation of KVStore diff --git a/store/v2/dbadapter/store_test.go b/store/v2/dbadapter/store_test.go index 9f8ac71b25cf..cc78d8ac850c 100644 --- a/store/v2/dbadapter/store_test.go +++ b/store/v2/dbadapter/store_test.go @@ -5,13 +5,12 @@ import ( "errors" "testing" - "github.com/cosmos/cosmos-sdk/store/cachekv" - "github.com/golang/mock/gomock" "github.com/stretchr/testify/require" - "github.com/cosmos/cosmos-sdk/store/dbadapter" - "github.com/cosmos/cosmos-sdk/store/types" + "github.com/cosmos/cosmos-sdk/store/cachekv" + types "github.com/cosmos/cosmos-sdk/store/v2" + "github.com/cosmos/cosmos-sdk/store/v2/dbadapter" "github.com/cosmos/cosmos-sdk/tests/mocks" ) From 35843aae3ad92f80ed9c789fed8ac260a0003f4c Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 25 Nov 2021 18:22:24 +0800 Subject: [PATCH 22/47] tests + mocks for v2/dbadapter --- Makefile | 1 + store/v2/dbadapter/store_test.go | 35 +- tests/mocks/db/types.go | 745 +++++++++++++++++++++++++++++++ 3 files changed, 772 insertions(+), 9 deletions(-) create mode 100644 tests/mocks/db/types.go diff --git a/Makefile b/Makefile index 990d22ae8354..973807324df3 100644 --- a/Makefile +++ b/Makefile @@ -132,6 +132,7 @@ mockgen_cmd=go run github.com/golang/mock/mockgen mocks: $(MOCKS_DIR) $(mockgen_cmd) -source=client/account_retriever.go -package mocks -destination tests/mocks/account_retriever.go $(mockgen_cmd) -package mocks -destination tests/mocks/tendermint_tm_db_DB.go github.com/tendermint/tm-db DB + $(mockgen_cmd) -source db/types.go -package mocks -destination tests/mocks/db/types.go $(mockgen_cmd) -source=types/module/module.go -package mocks -destination tests/mocks/types_module_module.go $(mockgen_cmd) -source=types/invariant.go -package mocks -destination tests/mocks/types_invariant.go $(mockgen_cmd) -source=types/router.go -package mocks -destination tests/mocks/types_router.go diff --git a/store/v2/dbadapter/store_test.go b/store/v2/dbadapter/store_test.go index cc78d8ac850c..a5ca75108479 100644 --- a/store/v2/dbadapter/store_test.go +++ b/store/v2/dbadapter/store_test.go @@ -11,7 +11,7 @@ import ( "github.com/cosmos/cosmos-sdk/store/cachekv" types "github.com/cosmos/cosmos-sdk/store/v2" "github.com/cosmos/cosmos-sdk/store/v2/dbadapter" - "github.com/cosmos/cosmos-sdk/tests/mocks" + mocks "github.com/cosmos/cosmos-sdk/tests/mocks/db" ) var errFoo = errors.New("dummy") @@ -20,7 +20,7 @@ func TestAccessors(t *testing.T) { mockCtrl := gomock.NewController(t) defer mockCtrl.Finish() - mockDB := mocks.NewMockDB(mockCtrl) + mockDB := mocks.NewMockDBReadWriter(mockCtrl) store := dbadapter.Store{mockDB} key := []byte("test") value := []byte("testvalue") @@ -34,6 +34,7 @@ func TestAccessors(t *testing.T) { retFoo := []byte("xxx") mockDB.EXPECT().Get(gomock.Eq(key)).Times(1).Return(retFoo, nil) require.True(t, bytes.Equal(retFoo, store.Get(key))) + require.Equal(t, []byte{1, 2, 3}, []byte{1, 2, 3}) mockDB.EXPECT().Get(gomock.Eq(key)).Times(1).Return(nil, errFoo) require.Panics(t, func() { store.Get(key) }) @@ -58,24 +59,40 @@ func TestAccessors(t *testing.T) { mockDB.EXPECT().Delete(gomock.Eq(key)).Times(1).Return(errFoo) require.Panics(t, func() { store.Delete(key) }) +} + +func TestIterators(t *testing.T) { + mockCtrl := gomock.NewController(t) + defer mockCtrl.Finish() + + mockDB := mocks.NewMockDBReadWriter(mockCtrl) + store := dbadapter.Store{mockDB} + key := []byte("test") + value := []byte("testvalue") - start, end := []byte("start"), []byte("end") - mockDB.EXPECT().Iterator(gomock.Eq(start), gomock.Eq(end)).Times(1).Return(nil, nil) - require.NotPanics(t, func() { store.Iterator(start, end) }) + start, end := key, []byte("test_end") mockDB.EXPECT().Iterator(gomock.Eq(start), gomock.Eq(end)).Times(1).Return(nil, errFoo) require.Panics(t, func() { store.Iterator(start, end) }) - mockDB.EXPECT().ReverseIterator(gomock.Eq(start), gomock.Eq(end)).Times(1).Return(nil, nil) - require.NotPanics(t, func() { store.ReverseIterator(start, end) }) - mockDB.EXPECT().ReverseIterator(gomock.Eq(start), gomock.Eq(end)).Times(1).Return(nil, errFoo) require.Panics(t, func() { store.ReverseIterator(start, end) }) + + mockIter := mocks.NewMockIterator(mockCtrl) + mockIter.EXPECT().Next().Times(1).Return(true) + mockIter.EXPECT().Key().Times(1).Return(key) + mockIter.EXPECT().Value().Times(1).Return(value) + + mockDB.EXPECT().Iterator(gomock.Eq(start), gomock.Eq(end)).Times(1).Return(mockIter, nil) + iter := store.Iterator(start, end) + + require.Equal(t, key, iter.Key()) + require.Equal(t, value, iter.Value()) } func TestCacheWraps(t *testing.T) { mockCtrl := gomock.NewController(t) - mockDB := mocks.NewMockDB(mockCtrl) + mockDB := mocks.NewMockDBReadWriter(mockCtrl) store := dbadapter.Store{mockDB} cacheWrapper := store.CacheWrap() diff --git a/tests/mocks/db/types.go b/tests/mocks/db/types.go new file mode 100644 index 000000000000..269094f7d0d6 --- /dev/null +++ b/tests/mocks/db/types.go @@ -0,0 +1,745 @@ +// Code generated by MockGen. DO NOT EDIT. +// Source: db/types.go + +// Package mocks is a generated GoMock package. +package mocks + +import ( + reflect "reflect" + + db "github.com/cosmos/cosmos-sdk/db" + gomock "github.com/golang/mock/gomock" +) + +// MockDBConnection is a mock of DBConnection interface. +type MockDBConnection struct { + ctrl *gomock.Controller + recorder *MockDBConnectionMockRecorder +} + +// MockDBConnectionMockRecorder is the mock recorder for MockDBConnection. +type MockDBConnectionMockRecorder struct { + mock *MockDBConnection +} + +// NewMockDBConnection creates a new mock instance. +func NewMockDBConnection(ctrl *gomock.Controller) *MockDBConnection { + mock := &MockDBConnection{ctrl: ctrl} + mock.recorder = &MockDBConnectionMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockDBConnection) EXPECT() *MockDBConnectionMockRecorder { + return m.recorder +} + +// Close mocks base method. +func (m *MockDBConnection) Close() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Close") + ret0, _ := ret[0].(error) + return ret0 +} + +// Close indicates an expected call of Close. +func (mr *MockDBConnectionMockRecorder) Close() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockDBConnection)(nil).Close)) +} + +// DeleteVersion mocks base method. +func (m *MockDBConnection) DeleteVersion(arg0 uint64) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DeleteVersion", arg0) + ret0, _ := ret[0].(error) + return ret0 +} + +// DeleteVersion indicates an expected call of DeleteVersion. +func (mr *MockDBConnectionMockRecorder) DeleteVersion(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteVersion", reflect.TypeOf((*MockDBConnection)(nil).DeleteVersion), arg0) +} + +// ReadWriter mocks base method. +func (m *MockDBConnection) ReadWriter() db.DBReadWriter { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ReadWriter") + ret0, _ := ret[0].(db.DBReadWriter) + return ret0 +} + +// ReadWriter indicates an expected call of ReadWriter. +func (mr *MockDBConnectionMockRecorder) ReadWriter() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ReadWriter", reflect.TypeOf((*MockDBConnection)(nil).ReadWriter)) +} + +// Reader mocks base method. +func (m *MockDBConnection) Reader() db.DBReader { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Reader") + ret0, _ := ret[0].(db.DBReader) + return ret0 +} + +// Reader indicates an expected call of Reader. +func (mr *MockDBConnectionMockRecorder) Reader() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Reader", reflect.TypeOf((*MockDBConnection)(nil).Reader)) +} + +// ReaderAt mocks base method. +func (m *MockDBConnection) ReaderAt(arg0 uint64) (db.DBReader, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ReaderAt", arg0) + ret0, _ := ret[0].(db.DBReader) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ReaderAt indicates an expected call of ReaderAt. +func (mr *MockDBConnectionMockRecorder) ReaderAt(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ReaderAt", reflect.TypeOf((*MockDBConnection)(nil).ReaderAt), arg0) +} + +// Revert mocks base method. +func (m *MockDBConnection) Revert() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Revert") + ret0, _ := ret[0].(error) + return ret0 +} + +// Revert indicates an expected call of Revert. +func (mr *MockDBConnectionMockRecorder) Revert() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Revert", reflect.TypeOf((*MockDBConnection)(nil).Revert)) +} + +// SaveNextVersion mocks base method. +func (m *MockDBConnection) SaveNextVersion() (uint64, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "SaveNextVersion") + ret0, _ := ret[0].(uint64) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// SaveNextVersion indicates an expected call of SaveNextVersion. +func (mr *MockDBConnectionMockRecorder) SaveNextVersion() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SaveNextVersion", reflect.TypeOf((*MockDBConnection)(nil).SaveNextVersion)) +} + +// SaveVersion mocks base method. +func (m *MockDBConnection) SaveVersion(arg0 uint64) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "SaveVersion", arg0) + ret0, _ := ret[0].(error) + return ret0 +} + +// SaveVersion indicates an expected call of SaveVersion. +func (mr *MockDBConnectionMockRecorder) SaveVersion(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SaveVersion", reflect.TypeOf((*MockDBConnection)(nil).SaveVersion), arg0) +} + +// Versions mocks base method. +func (m *MockDBConnection) Versions() (db.VersionSet, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Versions") + ret0, _ := ret[0].(db.VersionSet) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Versions indicates an expected call of Versions. +func (mr *MockDBConnectionMockRecorder) Versions() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Versions", reflect.TypeOf((*MockDBConnection)(nil).Versions)) +} + +// Writer mocks base method. +func (m *MockDBConnection) Writer() db.DBWriter { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Writer") + ret0, _ := ret[0].(db.DBWriter) + return ret0 +} + +// Writer indicates an expected call of Writer. +func (mr *MockDBConnectionMockRecorder) Writer() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Writer", reflect.TypeOf((*MockDBConnection)(nil).Writer)) +} + +// MockDBReader is a mock of DBReader interface. +type MockDBReader struct { + ctrl *gomock.Controller + recorder *MockDBReaderMockRecorder +} + +// MockDBReaderMockRecorder is the mock recorder for MockDBReader. +type MockDBReaderMockRecorder struct { + mock *MockDBReader +} + +// NewMockDBReader creates a new mock instance. +func NewMockDBReader(ctrl *gomock.Controller) *MockDBReader { + mock := &MockDBReader{ctrl: ctrl} + mock.recorder = &MockDBReaderMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockDBReader) EXPECT() *MockDBReaderMockRecorder { + return m.recorder +} + +// Discard mocks base method. +func (m *MockDBReader) Discard() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Discard") + ret0, _ := ret[0].(error) + return ret0 +} + +// Discard indicates an expected call of Discard. +func (mr *MockDBReaderMockRecorder) Discard() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Discard", reflect.TypeOf((*MockDBReader)(nil).Discard)) +} + +// Get mocks base method. +func (m *MockDBReader) Get(arg0 []byte) ([]byte, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Get", arg0) + ret0, _ := ret[0].([]byte) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Get indicates an expected call of Get. +func (mr *MockDBReaderMockRecorder) Get(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Get", reflect.TypeOf((*MockDBReader)(nil).Get), arg0) +} + +// Has mocks base method. +func (m *MockDBReader) Has(key []byte) (bool, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Has", key) + ret0, _ := ret[0].(bool) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Has indicates an expected call of Has. +func (mr *MockDBReaderMockRecorder) Has(key interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Has", reflect.TypeOf((*MockDBReader)(nil).Has), key) +} + +// Iterator mocks base method. +func (m *MockDBReader) Iterator(start, end []byte) (db.Iterator, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Iterator", start, end) + ret0, _ := ret[0].(db.Iterator) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Iterator indicates an expected call of Iterator. +func (mr *MockDBReaderMockRecorder) Iterator(start, end interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Iterator", reflect.TypeOf((*MockDBReader)(nil).Iterator), start, end) +} + +// ReverseIterator mocks base method. +func (m *MockDBReader) ReverseIterator(start, end []byte) (db.Iterator, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ReverseIterator", start, end) + ret0, _ := ret[0].(db.Iterator) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ReverseIterator indicates an expected call of ReverseIterator. +func (mr *MockDBReaderMockRecorder) ReverseIterator(start, end interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ReverseIterator", reflect.TypeOf((*MockDBReader)(nil).ReverseIterator), start, end) +} + +// MockDBWriter is a mock of DBWriter interface. +type MockDBWriter struct { + ctrl *gomock.Controller + recorder *MockDBWriterMockRecorder +} + +// MockDBWriterMockRecorder is the mock recorder for MockDBWriter. +type MockDBWriterMockRecorder struct { + mock *MockDBWriter +} + +// NewMockDBWriter creates a new mock instance. +func NewMockDBWriter(ctrl *gomock.Controller) *MockDBWriter { + mock := &MockDBWriter{ctrl: ctrl} + mock.recorder = &MockDBWriterMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockDBWriter) EXPECT() *MockDBWriterMockRecorder { + return m.recorder +} + +// Commit mocks base method. +func (m *MockDBWriter) Commit() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Commit") + ret0, _ := ret[0].(error) + return ret0 +} + +// Commit indicates an expected call of Commit. +func (mr *MockDBWriterMockRecorder) Commit() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Commit", reflect.TypeOf((*MockDBWriter)(nil).Commit)) +} + +// Delete mocks base method. +func (m *MockDBWriter) Delete(arg0 []byte) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Delete", arg0) + ret0, _ := ret[0].(error) + return ret0 +} + +// Delete indicates an expected call of Delete. +func (mr *MockDBWriterMockRecorder) Delete(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Delete", reflect.TypeOf((*MockDBWriter)(nil).Delete), arg0) +} + +// Discard mocks base method. +func (m *MockDBWriter) Discard() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Discard") + ret0, _ := ret[0].(error) + return ret0 +} + +// Discard indicates an expected call of Discard. +func (mr *MockDBWriterMockRecorder) Discard() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Discard", reflect.TypeOf((*MockDBWriter)(nil).Discard)) +} + +// Set mocks base method. +func (m *MockDBWriter) Set(arg0, arg1 []byte) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Set", arg0, arg1) + ret0, _ := ret[0].(error) + return ret0 +} + +// Set indicates an expected call of Set. +func (mr *MockDBWriterMockRecorder) Set(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Set", reflect.TypeOf((*MockDBWriter)(nil).Set), arg0, arg1) +} + +// MockDBReadWriter is a mock of DBReadWriter interface. +type MockDBReadWriter struct { + ctrl *gomock.Controller + recorder *MockDBReadWriterMockRecorder +} + +// MockDBReadWriterMockRecorder is the mock recorder for MockDBReadWriter. +type MockDBReadWriterMockRecorder struct { + mock *MockDBReadWriter +} + +// NewMockDBReadWriter creates a new mock instance. +func NewMockDBReadWriter(ctrl *gomock.Controller) *MockDBReadWriter { + mock := &MockDBReadWriter{ctrl: ctrl} + mock.recorder = &MockDBReadWriterMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockDBReadWriter) EXPECT() *MockDBReadWriterMockRecorder { + return m.recorder +} + +// Commit mocks base method. +func (m *MockDBReadWriter) Commit() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Commit") + ret0, _ := ret[0].(error) + return ret0 +} + +// Commit indicates an expected call of Commit. +func (mr *MockDBReadWriterMockRecorder) Commit() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Commit", reflect.TypeOf((*MockDBReadWriter)(nil).Commit)) +} + +// Delete mocks base method. +func (m *MockDBReadWriter) Delete(arg0 []byte) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Delete", arg0) + ret0, _ := ret[0].(error) + return ret0 +} + +// Delete indicates an expected call of Delete. +func (mr *MockDBReadWriterMockRecorder) Delete(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Delete", reflect.TypeOf((*MockDBReadWriter)(nil).Delete), arg0) +} + +// Discard mocks base method. +func (m *MockDBReadWriter) Discard() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Discard") + ret0, _ := ret[0].(error) + return ret0 +} + +// Discard indicates an expected call of Discard. +func (mr *MockDBReadWriterMockRecorder) Discard() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Discard", reflect.TypeOf((*MockDBReadWriter)(nil).Discard)) +} + +// Get mocks base method. +func (m *MockDBReadWriter) Get(arg0 []byte) ([]byte, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Get", arg0) + ret0, _ := ret[0].([]byte) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Get indicates an expected call of Get. +func (mr *MockDBReadWriterMockRecorder) Get(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Get", reflect.TypeOf((*MockDBReadWriter)(nil).Get), arg0) +} + +// Has mocks base method. +func (m *MockDBReadWriter) Has(key []byte) (bool, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Has", key) + ret0, _ := ret[0].(bool) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Has indicates an expected call of Has. +func (mr *MockDBReadWriterMockRecorder) Has(key interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Has", reflect.TypeOf((*MockDBReadWriter)(nil).Has), key) +} + +// Iterator mocks base method. +func (m *MockDBReadWriter) Iterator(start, end []byte) (db.Iterator, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Iterator", start, end) + ret0, _ := ret[0].(db.Iterator) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// Iterator indicates an expected call of Iterator. +func (mr *MockDBReadWriterMockRecorder) Iterator(start, end interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Iterator", reflect.TypeOf((*MockDBReadWriter)(nil).Iterator), start, end) +} + +// ReverseIterator mocks base method. +func (m *MockDBReadWriter) ReverseIterator(start, end []byte) (db.Iterator, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ReverseIterator", start, end) + ret0, _ := ret[0].(db.Iterator) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ReverseIterator indicates an expected call of ReverseIterator. +func (mr *MockDBReadWriterMockRecorder) ReverseIterator(start, end interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ReverseIterator", reflect.TypeOf((*MockDBReadWriter)(nil).ReverseIterator), start, end) +} + +// Set mocks base method. +func (m *MockDBReadWriter) Set(arg0, arg1 []byte) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Set", arg0, arg1) + ret0, _ := ret[0].(error) + return ret0 +} + +// Set indicates an expected call of Set. +func (mr *MockDBReadWriterMockRecorder) Set(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Set", reflect.TypeOf((*MockDBReadWriter)(nil).Set), arg0, arg1) +} + +// MockIterator is a mock of Iterator interface. +type MockIterator struct { + ctrl *gomock.Controller + recorder *MockIteratorMockRecorder +} + +// MockIteratorMockRecorder is the mock recorder for MockIterator. +type MockIteratorMockRecorder struct { + mock *MockIterator +} + +// NewMockIterator creates a new mock instance. +func NewMockIterator(ctrl *gomock.Controller) *MockIterator { + mock := &MockIterator{ctrl: ctrl} + mock.recorder = &MockIteratorMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockIterator) EXPECT() *MockIteratorMockRecorder { + return m.recorder +} + +// Close mocks base method. +func (m *MockIterator) Close() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Close") + ret0, _ := ret[0].(error) + return ret0 +} + +// Close indicates an expected call of Close. +func (mr *MockIteratorMockRecorder) Close() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockIterator)(nil).Close)) +} + +// Domain mocks base method. +func (m *MockIterator) Domain() ([]byte, []byte) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Domain") + ret0, _ := ret[0].([]byte) + ret1, _ := ret[1].([]byte) + return ret0, ret1 +} + +// Domain indicates an expected call of Domain. +func (mr *MockIteratorMockRecorder) Domain() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Domain", reflect.TypeOf((*MockIterator)(nil).Domain)) +} + +// Error mocks base method. +func (m *MockIterator) Error() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Error") + ret0, _ := ret[0].(error) + return ret0 +} + +// Error indicates an expected call of Error. +func (mr *MockIteratorMockRecorder) Error() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Error", reflect.TypeOf((*MockIterator)(nil).Error)) +} + +// Key mocks base method. +func (m *MockIterator) Key() []byte { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Key") + ret0, _ := ret[0].([]byte) + return ret0 +} + +// Key indicates an expected call of Key. +func (mr *MockIteratorMockRecorder) Key() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Key", reflect.TypeOf((*MockIterator)(nil).Key)) +} + +// Next mocks base method. +func (m *MockIterator) Next() bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Next") + ret0, _ := ret[0].(bool) + return ret0 +} + +// Next indicates an expected call of Next. +func (mr *MockIteratorMockRecorder) Next() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Next", reflect.TypeOf((*MockIterator)(nil).Next)) +} + +// Value mocks base method. +func (m *MockIterator) Value() []byte { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Value") + ret0, _ := ret[0].([]byte) + return ret0 +} + +// Value indicates an expected call of Value. +func (mr *MockIteratorMockRecorder) Value() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Value", reflect.TypeOf((*MockIterator)(nil).Value)) +} + +// MockVersionSet is a mock of VersionSet interface. +type MockVersionSet struct { + ctrl *gomock.Controller + recorder *MockVersionSetMockRecorder +} + +// MockVersionSetMockRecorder is the mock recorder for MockVersionSet. +type MockVersionSetMockRecorder struct { + mock *MockVersionSet +} + +// NewMockVersionSet creates a new mock instance. +func NewMockVersionSet(ctrl *gomock.Controller) *MockVersionSet { + mock := &MockVersionSet{ctrl: ctrl} + mock.recorder = &MockVersionSetMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockVersionSet) EXPECT() *MockVersionSetMockRecorder { + return m.recorder +} + +// Count mocks base method. +func (m *MockVersionSet) Count() int { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Count") + ret0, _ := ret[0].(int) + return ret0 +} + +// Count indicates an expected call of Count. +func (mr *MockVersionSetMockRecorder) Count() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Count", reflect.TypeOf((*MockVersionSet)(nil).Count)) +} + +// Equal mocks base method. +func (m *MockVersionSet) Equal(arg0 db.VersionSet) bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Equal", arg0) + ret0, _ := ret[0].(bool) + return ret0 +} + +// Equal indicates an expected call of Equal. +func (mr *MockVersionSetMockRecorder) Equal(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Equal", reflect.TypeOf((*MockVersionSet)(nil).Equal), arg0) +} + +// Exists mocks base method. +func (m *MockVersionSet) Exists(arg0 uint64) bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Exists", arg0) + ret0, _ := ret[0].(bool) + return ret0 +} + +// Exists indicates an expected call of Exists. +func (mr *MockVersionSetMockRecorder) Exists(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Exists", reflect.TypeOf((*MockVersionSet)(nil).Exists), arg0) +} + +// Iterator mocks base method. +func (m *MockVersionSet) Iterator() db.VersionIterator { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Iterator") + ret0, _ := ret[0].(db.VersionIterator) + return ret0 +} + +// Iterator indicates an expected call of Iterator. +func (mr *MockVersionSetMockRecorder) Iterator() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Iterator", reflect.TypeOf((*MockVersionSet)(nil).Iterator)) +} + +// Last mocks base method. +func (m *MockVersionSet) Last() uint64 { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Last") + ret0, _ := ret[0].(uint64) + return ret0 +} + +// Last indicates an expected call of Last. +func (mr *MockVersionSetMockRecorder) Last() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Last", reflect.TypeOf((*MockVersionSet)(nil).Last)) +} + +// MockVersionIterator is a mock of VersionIterator interface. +type MockVersionIterator struct { + ctrl *gomock.Controller + recorder *MockVersionIteratorMockRecorder +} + +// MockVersionIteratorMockRecorder is the mock recorder for MockVersionIterator. +type MockVersionIteratorMockRecorder struct { + mock *MockVersionIterator +} + +// NewMockVersionIterator creates a new mock instance. +func NewMockVersionIterator(ctrl *gomock.Controller) *MockVersionIterator { + mock := &MockVersionIterator{ctrl: ctrl} + mock.recorder = &MockVersionIteratorMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockVersionIterator) EXPECT() *MockVersionIteratorMockRecorder { + return m.recorder +} + +// Next mocks base method. +func (m *MockVersionIterator) Next() bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Next") + ret0, _ := ret[0].(bool) + return ret0 +} + +// Next indicates an expected call of Next. +func (mr *MockVersionIteratorMockRecorder) Next() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Next", reflect.TypeOf((*MockVersionIterator)(nil).Next)) +} + +// Value mocks base method. +func (m *MockVersionIterator) Value() uint64 { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Value") + ret0, _ := ret[0].(uint64) + return ret0 +} + +// Value indicates an expected call of Value. +func (mr *MockVersionIteratorMockRecorder) Value() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Value", reflect.TypeOf((*MockVersionIterator)(nil).Value)) +} From 5c9eb8125708be41b0c5a1761220e6b8e078e560 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Fri, 26 Nov 2021 16:58:51 +0800 Subject: [PATCH 23/47] PR revisions --- store/v2/root/store.go | 44 ++++++++++++++++++++++++------------- store/v2/root/view_store.go | 7 +++--- store/v2/types.go | 17 ++++++++------ 3 files changed, 43 insertions(+), 25 deletions(-) diff --git a/store/v2/root/store.go b/store/v2/root/store.go index e4e8ac72051f..076bc2afbe0a 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -1,5 +1,3 @@ -// RootStore supports a subset of the StoreType values: Persistent, Memory, and Transient - package root import ( @@ -56,10 +54,11 @@ func ErrStoreNotFound(skey string) error { return fmt.Errorf("store does not exist for key: %s", skey) } -// StoreConfig is used to define a schema and pass options to the RootStore constructor. +// StoreConfig is used to define a schema and other options and pass them to the RootStore constructor. type StoreConfig struct { // Version pruning options for backing DBs. - Pruning types.PruningOptions + Pruning types.PruningOptions + // The minimum allowed version number. InitialVersion uint64 // The backing DB to use for the state commitment Merkle tree data. // If nil, Merkle data is stored in the state storage DB under a separate prefix. @@ -73,10 +72,14 @@ type StoreConfig struct { *traceMixin } -// A loaded mapping of substore keys to store types +// StoreSchema defineds a mapping of substore keys to store types type StoreSchema map[string]types.StoreType -// Main persistent store type +// Store is the main persistent store type implementing CommitRootStore. +// Substores consist of an SMT-based state commitment store and state storage. +// Substores must be reserved in the StoreConfig or defined as part of a StoreUpgrade in order to be valid. +// The state commitment store of each substore consists of a independent SMT. +// The state commitment of the root store consists of a Merkle map of all registered persistent substore names to the root hash of their corresponding SMTs. type Store struct { stateDB dbm.DBConnection stateTxn dbm.DBReadWriter @@ -88,8 +91,9 @@ type Store struct { tran *transkv.Store mtx sync.RWMutex + // Copied from StoreConfig Pruning types.PruningOptions - InitialVersion uint64 + InitialVersion uint64 // if *listenerMixin *traceMixin PersistentCache types.RootStorePersistentCache @@ -443,6 +447,7 @@ func substorePrefix(key string) []byte { return append(contentPrefix, key...) } +// GetKVStore implements BasicRootStore. func (rs *Store) GetKVStore(skey types.StoreKey) types.KVStore { key := skey.Name() var sub types.KVStore @@ -452,6 +457,9 @@ func (rs *Store) GetKVStore(skey types.StoreKey) types.KVStore { sub = rs.mem case types.StoreTypeTransient: sub = rs.tran + case types.StoreTypePersistent: + default: + panic(fmt.Errorf("StoreType not supported: %v", typ)) // should never happen } if sub != nil { if cached, has := rs.npSubstoreCache[key]; has { @@ -464,9 +472,7 @@ func (rs *Store) GetKVStore(skey types.StoreKey) types.KVStore { } else { panic(ErrStoreNotFound(key)) } - if cached, has := rs.substoreCache[key]; has { - return cached - } + // store is persistent ret, err := rs.getSubstore(key) if err != nil { panic(err) @@ -475,8 +481,11 @@ func (rs *Store) GetKVStore(skey types.StoreKey) types.KVStore { return ret } -// Gets a persistent substore +// Gets a persistent substore. This reads, but does not update the substore cache func (rs *Store) getSubstore(key string) (*substore, error) { + if cached, has := rs.substoreCache[key]; has { + return cached, nil + } pfx := substorePrefix(key) stateRW := prefixdb.NewPrefixReadWriter(rs.stateTxn, pfx) stateCommitmentRW := prefixdb.NewPrefixReadWriter(rs.stateCommitmentTxn, pfx) @@ -503,7 +512,7 @@ func (rs *Store) getSubstore(key string) (*substore, error) { }, nil } -// resets a substore's state after commit (stateTxn discarded) +// Resets a substore's state after commit (because root stateTxn has been discarded) func (s *substore) refresh(rootHash []byte) { pfx := substorePrefix(s.name) stateRW := prefixdb.NewPrefixReadWriter(s.root.stateTxn, pfx) @@ -517,7 +526,7 @@ func (s *substore) refresh(rootHash []byte) { func (s *Store) Commit() types.CommitID { s.mtx.Lock() defer s.mtx.Unlock() - + // Determine the target version versions, err := s.stateDB.Versions() if err != nil { panic(err) @@ -534,7 +543,7 @@ func (s *Store) Commit() types.CommitID { if err != nil { panic(err) } - + // Prune if necessary previous := cid.Version - 1 if s.Pruning.KeepEvery != 1 && s.Pruning.Interval != 0 && cid.Version%int64(s.Pruning.Interval) == 0 { // The range of newly prunable versions @@ -569,6 +578,7 @@ func (s *Store) getMerkleRoots() (ret map[string][]byte, err error) { return } +// Calculates root hashes and commits to DB. Does not verify target version or perform pruning. func (s *Store) commit(target uint64) (id *types.CommitID, err error) { storeHashes, err := s.getMerkleRoots() if err != nil { @@ -637,7 +647,7 @@ func (s *Store) commit(target uint64) (id *types.CommitID, err error) { s.stateTxn = stateTxn s.stateCommitmentTxn = stateCommitmentTxn - // the state on all living substores must be refreshed + // the state of all live substores must be refreshed for key, sub := range s.substoreCache { sub.refresh(storeHashes[key]) } @@ -663,15 +673,18 @@ func (s *Store) LastCommitID() types.CommitID { return types.CommitID{Version: int64(last), Hash: hash} } +// SetInitialVersion implements CommitRootStore. func (rs *Store) SetInitialVersion(version uint64) error { rs.InitialVersion = uint64(version) return nil } +// GetVersion implements CommitRootStore. func (rs *Store) GetVersion(version int64) (types.BasicRootStore, error) { return rs.getView(version) } +// CacheRootStore implements BasicRootStore. func (rs *Store) CacheRootStore() types.CacheRootStore { return &cacheStore{ source: rs, @@ -793,6 +806,7 @@ func (rs *Store) Query(req abci.RequestQuery) (res abci.ResponseQuery) { return res } +// GetKVStore implements BasicRootStore. func (cs *cacheStore) GetKVStore(key types.StoreKey) types.KVStore { ret, has := cs.substores[key.Name()] if has { diff --git a/store/v2/root/view_store.go b/store/v2/root/view_store.go index ae02296d6dd2..b0f5438ed7eb 100644 --- a/store/v2/root/view_store.go +++ b/store/v2/root/view_store.go @@ -133,9 +133,6 @@ func (vs *viewStore) GetKVStore(skey types.StoreKey) types.KVStore { if _, has := vs.schema[key]; !has { panic(ErrStoreNotFound(key)) } - if cached, has := vs.substoreCache[key]; has { - return cached - } ret, err := vs.getSubstore(key) if err != nil { panic(err) @@ -144,7 +141,11 @@ func (vs *viewStore) GetKVStore(skey types.StoreKey) types.KVStore { return ret } +// Reads but does not update substore cache func (vs *viewStore) getSubstore(key string) (*viewSubstore, error) { + if cached, has := vs.substoreCache[key]; has { + return cached, nil + } pfx := substorePrefix(key) stateR := prefixdb.NewPrefixReader(vs.stateView, pfx) stateCommitmentR := prefixdb.NewPrefixReader(vs.stateCommitmentView, pfx) diff --git a/store/v2/types.go b/store/v2/types.go index 743bb91eccc4..1ab79e16406e 100644 --- a/store/v2/types.go +++ b/store/v2/types.go @@ -7,7 +7,10 @@ import ( v1 "github.com/cosmos/cosmos-sdk/store/types" ) +// Re-export original store types + type StoreKey = v1.StoreKey +type StoreType = v1.StoreType type CommitID = v1.CommitID type StoreUpgrades = v1.StoreUpgrades type StoreRename = v1.StoreRename @@ -35,6 +38,7 @@ var ( PruneNothing = v1.PruneNothing ) +// BasicRootStore defines a minimal interface for accessing root state. type BasicRootStore interface { // Returns a KVStore which has access only to the namespace of the StoreKey. // Panics if the key is not found in the schema. @@ -43,6 +47,7 @@ type BasicRootStore interface { CacheRootStore() CacheRootStore } +// mixin interface for trace and listen methods type rootStoreTraceListen interface { TracingEnabled() bool SetTracer(w io.Writer) @@ -51,6 +56,8 @@ type rootStoreTraceListen interface { AddListeners(key StoreKey, listeners []WriteListener) } +// CommitRootStore defines a complete interface for persistent root state, including +// (read-only) access to past versions, pruning, trace/listen, and state snapshots. type CommitRootStore interface { BasicRootStore rootStoreTraceListen @@ -67,22 +74,18 @@ type CommitRootStore interface { SetInitialVersion(uint64) error } +// CacheRootStore defines a branch of the root state which can be written back to the source store. type CacheRootStore interface { BasicRootStore rootStoreTraceListen Write() } -// provides inter-block (persistent) caching capabilities for a CommitRootStore -// TODO +// RootStorePersistentCache provides inter-block (persistent) caching capabilities for a CommitRootStore. type RootStorePersistentCache = v1.MultiStorePersistentCache -//---------------------------------------- -// Store types - -type StoreType = v1.StoreType +// Re-export relevant store type values and utility functions -// Valid types const StoreTypeMemory = v1.StoreTypeMemory const StoreTypeTransient = v1.StoreTypeTransient const StoreTypeDB = v1.StoreTypeDB From 21e3745d596f4ef3d31f38c1d5c09a679ce7ff29 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Mon, 29 Nov 2021 18:12:07 +0800 Subject: [PATCH 24/47] godoc, comments --- store/v2/root/store.go | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/store/v2/root/store.go b/store/v2/root/store.go index 076bc2afbe0a..0258985fae76 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -78,8 +78,10 @@ type StoreSchema map[string]types.StoreType // Store is the main persistent store type implementing CommitRootStore. // Substores consist of an SMT-based state commitment store and state storage. // Substores must be reserved in the StoreConfig or defined as part of a StoreUpgrade in order to be valid. -// The state commitment store of each substore consists of a independent SMT. -// The state commitment of the root store consists of a Merkle map of all registered persistent substore names to the root hash of their corresponding SMTs. +// Note: +// The state commitment data and proof are structured in the same basic pattern as the MultiStore, but use an SMT rather than IAVL tree: +// * The state commitment store of each substore consists of a independent SMT. +// * The state commitment of the root store consists of a Merkle map of all registered persistent substore names to the root hash of their corresponding SMTs type Store struct { stateDB dbm.DBConnection stateTxn dbm.DBReadWriter @@ -118,7 +120,7 @@ type cacheStore struct { *traceMixin } -// Read-only store for querying +// Read-only store for querying past versions type viewStore struct { stateView dbm.DBReader stateCommitmentView dbm.DBReader @@ -448,6 +450,7 @@ func substorePrefix(key string) []byte { } // GetKVStore implements BasicRootStore. +// Returns a substore whose contents func (rs *Store) GetKVStore(skey types.StoreKey) types.KVStore { key := skey.Name() var sub types.KVStore @@ -524,6 +527,7 @@ func (s *substore) refresh(rootHash []byte) { // Commit implements Committer. func (s *Store) Commit() types.CommitID { + // Substores read-lock this mutex; lock to prevent racey invalidation of underlying txns s.mtx.Lock() defer s.mtx.Unlock() // Determine the target version @@ -817,13 +821,16 @@ func (cs *cacheStore) GetKVStore(key types.StoreKey) types.KVStore { return ret } +// Write implements CacheRootStore. func (cs *cacheStore) Write() { - for _, sub := range cs.substores { + for skey, sub := range cs.substores { sub.Write() + delete(cs.substores, skey) } } -// Recursively wraps the CacheRootStore in another cache store. +// CacheRootStore implements BasicRootStore. +// This recursively wraps the CacheRootStore in another cache store. func (cs *cacheStore) CacheRootStore() types.CacheRootStore { return &cacheStore{ source: cs, From 5f6b3554b93120a0f4f362a87b47332590458981 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 1 Dec 2021 16:51:47 +0800 Subject: [PATCH 25/47] test commit - failed revert recovery --- store/v2/root/store_test.go | 24 ++++++++++++++++++------ store/v2/root/test_util.go | 24 +++++++++++++++++++----- 2 files changed, 37 insertions(+), 11 deletions(-) diff --git a/store/v2/root/store_test.go b/store/v2/root/store_test.go index 385c8b162fdb..228a2a58d868 100644 --- a/store/v2/root/store_test.go +++ b/store/v2/root/store_test.go @@ -259,6 +259,7 @@ func TestCommit(t *testing.T) { require.Panics(t, func() { store.Commit() }) require.NoError(t, store.Close()) + // No version should be saved in the backing DB(s) versions, _ := db.Versions() require.Equal(t, 0, versions.Count()) if store.StateCommitmentDB != nil { @@ -266,6 +267,7 @@ func TestCommit(t *testing.T) { require.Equal(t, 0, versions.Count()) } + // The store should now be reloaded successfully store, err := NewStore(db, opts) require.NoError(t, err) s1 = store.GetKVStore(skey_1) @@ -276,26 +278,36 @@ func TestCommit(t *testing.T) { opts := simpleStoreConfig(t) opts.Pruning = types.PruneNothing - // Ensure storage commit is rolled back in each failure case + // Ensure Store's commit is rolled back in each failure case... t.Run("recover after failed Commit", func(t *testing.T) { store, err := NewStore(dbRWCommitFails{memdb.NewDB()}, opts) require.NoError(t, err) testFailedCommit(t, store, nil, opts) }) - t.Run("recover after failed SaveVersion", func(t *testing.T) { - store, err := NewStore(dbSaveVersionFails{memdb.NewDB()}, opts) + // If SaveVersion and Revert both fail during Store.Commit, the DB will contain + // committed data that belongs to no version: non-atomic behavior from the Store user's perspective. + // So, that data must be reverted when the store is reloaded. + t.Run("recover after failed SaveVersion and Revert", func(t *testing.T) { + var db dbm.DBConnection + db = dbSaveVersionFails{memdb.NewDB()} + // Revert should succeed in initial NewStore call, but fail during Commit + db = dbRevertFails{db, []bool{false, true}} + store, err := NewStore(db, opts) require.NoError(t, err) testFailedCommit(t, store, nil, opts) }) - + // Repeat the above for StateCommitmentDB t.Run("recover after failed StateCommitmentDB Commit", func(t *testing.T) { opts.StateCommitmentDB = dbRWCommitFails{memdb.NewDB()} store, err := NewStore(memdb.NewDB(), opts) require.NoError(t, err) testFailedCommit(t, store, nil, opts) }) - t.Run("recover after failed StateCommitmentDB SaveVersion", func(t *testing.T) { - opts.StateCommitmentDB = dbSaveVersionFails{memdb.NewDB()} + t.Run("recover after failed StateCommitmentDB SaveVersion and Revert", func(t *testing.T) { + var db dbm.DBConnection + db = dbSaveVersionFails{memdb.NewDB()} + db = dbRevertFails{db, []bool{false, true}} + opts.StateCommitmentDB = db store, err := NewStore(memdb.NewDB(), opts) require.NoError(t, err) testFailedCommit(t, store, nil, opts) diff --git a/store/v2/root/test_util.go b/store/v2/root/test_util.go index 2fcdbd2f09ed..777e59cc2b06 100644 --- a/store/v2/root/test_util.go +++ b/store/v2/root/test_util.go @@ -5,13 +5,17 @@ import ( "errors" dbm "github.com/cosmos/cosmos-sdk/db" - "github.com/cosmos/cosmos-sdk/db/memdb" ) type dbDeleteVersionFails struct{ dbm.DBConnection } -type dbRWCommitFails struct{ *memdb.MemDB } +type dbRWCommitFails struct{ dbm.DBConnection } type dbRWCrudFails struct{ dbm.DBConnection } -type dbSaveVersionFails struct{ *memdb.MemDB } +type dbSaveVersionFails struct{ dbm.DBConnection } +type dbRevertFails struct { + dbm.DBConnection + // order of calls to fail on (eg. [1, 0] => first call fails; second succeeds) + failOn []bool +} type dbVersionsIs struct { dbm.DBConnection vset dbm.VersionSet @@ -28,14 +32,24 @@ func (db dbVersionsIs) Versions() (dbm.VersionSet, error) { return db.vset, nil func (db dbRWCrudFails) ReadWriter() dbm.DBReadWriter { return rwCrudFails{db.DBConnection.ReadWriter(), nil} } -func (dbSaveVersionFails) SaveVersion(uint64) error { return errors.New("dbSaveVersionFails") } +func (dbSaveVersionFails) SaveVersion(uint64) error { return errors.New("dbSaveVersionFails") } +func (db dbRevertFails) Revert() error { + fail := false + if len(db.failOn) > 0 { + fail, db.failOn = db.failOn[0], db.failOn[1:] + } + if fail { + return errors.New("dbRevertFails") + } + return db.DBConnection.Revert() +} func (dbDeleteVersionFails) DeleteVersion(uint64) error { return errors.New("dbDeleteVersionFails") } func (tx rwCommitFails) Commit() error { tx.Discard() return errors.New("rwCommitFails") } func (db dbRWCommitFails) ReadWriter() dbm.DBReadWriter { - return rwCommitFails{db.MemDB.ReadWriter()} + return rwCommitFails{db.DBConnection.ReadWriter()} } func (rw rwCrudFails) Get(k []byte) ([]byte, error) { From 856b5faa04fd3ce561948758f52c6fbb98db60d9 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 1 Dec 2021 17:48:41 +0800 Subject: [PATCH 26/47] update docs --- docs/core/store.md | 40 ++++++++++++++++++++++++++++++++-------- 1 file changed, 32 insertions(+), 8 deletions(-) diff --git a/docs/core/store.md b/docs/core/store.md index 54c636ba2cf6..99123a5a3f87 100644 --- a/docs/core/store.md +++ b/docs/core/store.md @@ -232,24 +232,48 @@ Additional information about state streaming configuration can be found in the [ When `KVStore.Set` or `KVStore.Delete` methods are called, `listenkv.Store` automatically writes the operations to the set of `Store.listeners`. -## New Store package (`store/v2`) +# New Store package (`store/v2`) The SDK is in the process of transitioning to use the types listed here as the default interface for state storage. At the time of writing, these cannot be used within an application and are not directly compatible with the `CommitMultiStore` and related types. -### `BasicKVStore` interface +These types use the new `db` sub-module of Cosmos-SDK (`github.com/cosmos/cosmos-sdk/db`), rather than TM-DB (`github.com/tendermint/tm-db`). + +See [ADR-040](../architecture/adr-040-storage-and-smt-state-commitments.md) for the motivations and design specifications of the change. + +## `BasicKVStore` interface An interface providing only the basic CRUD functionality (`Get`, `Set`, `Has`, and `Delete` methods), without iteration or caching. This is used to partially expose components of a larger store, such as a `flat.Store`. -### Root Store +## Root Store + +This is the new interface for the main client store, replacing the function of `MultiStore`. There are a few significant differences in behavior compared with `MultiStore`: + * Commits are atomic and are performed on the entire store state; individual substores cannot be committed separately and cannot have different version numbers. + * The store's current version and version history track that of the backing `db.DBConnection`. Past versions are accessible read-only. + * The set of valid substores is defined in at initialization and cannot be updated dynamically in an existing store instance. + +### `CommitRootStore` + +This is the main interface for persisent application state, analogous to `CommitMultiStore`. + * Past versions are accessed with `GetVersion`, which returns a `BasicRootStore`. + * Substores are accessed with `GetKVStore`. Trying to get a substore that was not defined at initialization will cause a panic. + * `Close` must be called to release the DB resources being used by the store. + +### `BasicRootStore` + +A minimal interface that only allows accessing substores. Note: substores returned by `BasicRootStore.GetKVStore` are read-only. + +### Implementation (`root.Store`) + +The canonical implementation of `RootStore` is in `store/v2/root`. It internally decouples the concerns of state storage and state commitment: values are stored in, and read directly from, the backing key-value database, but are also mapped in a logically separate store which generates cryptographic proofs (the *state-commitment* store). -`RootStore` is the new interface for the main client store, replacing the function of `MultiStore`. It internally decouples the concerns of state storage and state commitment: values are stored and read directly from the backing key-value database, but are also mapped in a logically separate *state-commitment* store which generates cryptographic proofs. +The state-commitment component of each substore is implemented as an independent `smt.Store`. Internally, each substore is allocated in a separate partition within the backing DB, such that commits apply to the state of all substores. Likewise, past version state includes the state of all substore storage and state-commitment stores. -Implemented in `store/v2/root`. This can optionally be configured to use different backend databases for each bucket, e.g., `badgerdb` for the state storage DB and `memdb` for the state commitment DB. State commitment is implemented with an `smt.Store`. +This store can optionally be configured to use different backend databases for each bucket (e.g., `badgerdb` for the state storage DB and `memdb` for the state-commitment DB). -### SMT Store +## SMT Store -Maps values into a Sparse Merkle Tree, and supports a `BasicKVStore` interface as well as methods for cryptographic proof generation. +`store/v2/smt.Store` maps values into a Sparse Merkle Tree (SMT), and supports a `BasicKVStore` interface as well as methods for cryptographic proof generation. -## Next {hide} +# Next {hide} Learn about [encoding](./encoding.md) {hide} From d28890b43c2298c433110d680aaf3898c6104d89 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 1 Dec 2021 18:27:43 +0800 Subject: [PATCH 27/47] cleanup --- store/v2/root/store.go | 6 +-- store/v2/root/view_store.go | 9 ---- store/v2/types.go | 97 +++++++++++++++++++++---------------- 3 files changed, 57 insertions(+), 55 deletions(-) diff --git a/store/v2/root/store.go b/store/v2/root/store.go index 0258985fae76..fdef6ee2ff36 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -20,7 +20,7 @@ import ( types "github.com/cosmos/cosmos-sdk/store/v2" "github.com/cosmos/cosmos-sdk/store/v2/mem" "github.com/cosmos/cosmos-sdk/store/v2/smt" - transkv "github.com/cosmos/cosmos-sdk/store/v2/transient" + "github.com/cosmos/cosmos-sdk/store/v2/transient" sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" "github.com/cosmos/cosmos-sdk/types/kv" ) @@ -90,7 +90,7 @@ type Store struct { schema StoreSchema mem *mem.Store - tran *transkv.Store + tran *transient.Store mtx sync.RWMutex // Copied from StoreConfig @@ -340,7 +340,7 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { } } ret.mem = mem.NewStore(memdb.NewDB()) - ret.tran = transkv.NewStore(memdb.NewDB()) + ret.tran = transient.NewStore(memdb.NewDB()) ret.schema = reg.StoreSchema return } diff --git a/store/v2/root/view_store.go b/store/v2/root/view_store.go index b0f5438ed7eb..85eaeb6e7cfe 100644 --- a/store/v2/root/view_store.go +++ b/store/v2/root/view_store.go @@ -159,12 +159,3 @@ func (vs *viewStore) getSubstore(key string) (*viewSubstore, error) { stateCommitmentStore: loadSMT(dbm.ReaderAsReadWriter(stateCommitmentR), rootHash), }, nil } - -func (vs *viewStore) CacheRootStore() types.CacheRootStore { - return &cacheStore{ - source: vs, - substores: map[string]types.CacheKVStore{}, - listenerMixin: &listenerMixin{}, - traceMixin: &traceMixin{}, - } -} diff --git a/store/v2/types.go b/store/v2/types.go index 1ab79e16406e..7e9db7697e23 100644 --- a/store/v2/types.go +++ b/store/v2/types.go @@ -7,35 +7,55 @@ import ( v1 "github.com/cosmos/cosmos-sdk/store/types" ) -// Re-export original store types - -type StoreKey = v1.StoreKey -type StoreType = v1.StoreType -type CommitID = v1.CommitID -type StoreUpgrades = v1.StoreUpgrades -type StoreRename = v1.StoreRename -type Iterator = v1.Iterator -type PruningOptions = v1.PruningOptions - -type TraceContext = v1.TraceContext -type WriteListener = v1.WriteListener - -type BasicKVStore = v1.BasicKVStore -type KVStore = v1.KVStore -type Committer = v1.Committer -type CommitKVStore = v1.CommitKVStore -type CacheKVStore = v1.CacheKVStore -type Queryable = v1.Queryable -type CacheWrap = v1.CacheWrap - -type KVStoreKey = v1.KVStoreKey -type MemoryStoreKey = v1.MemoryStoreKey -type TransientStoreKey = v1.TransientStoreKey +// Re-export relevant original store types +type ( + StoreKey = v1.StoreKey + StoreType = v1.StoreType + CommitID = v1.CommitID + StoreUpgrades = v1.StoreUpgrades + StoreRename = v1.StoreRename + Iterator = v1.Iterator + PruningOptions = v1.PruningOptions + + TraceContext = v1.TraceContext + WriteListener = v1.WriteListener + + BasicKVStore = v1.BasicKVStore + KVStore = v1.KVStore + Committer = v1.Committer + CommitKVStore = v1.CommitKVStore + CacheKVStore = v1.CacheKVStore + Queryable = v1.Queryable + CacheWrap = v1.CacheWrap + + KVStoreKey = v1.KVStoreKey + MemoryStoreKey = v1.MemoryStoreKey + TransientStoreKey = v1.TransientStoreKey + + KVPair = v1.KVPair + StoreKVPair = v1.StoreKVPair +) + +// Re-export relevant constants, values and utility functions +const ( + StoreTypeMemory = v1.StoreTypeMemory + StoreTypeTransient = v1.StoreTypeTransient + StoreTypeDB = v1.StoreTypeDB + StoreTypeSMT = v1.StoreTypeSMT + StoreTypePersistent = v1.StoreTypePersistent +) var ( PruneDefault = v1.PruneDefault PruneEverything = v1.PruneEverything PruneNothing = v1.PruneNothing + + NewKVStoreKey = v1.NewKVStoreKey + PrefixEndBytes = v1.PrefixEndBytes + KVStorePrefixIterator = v1.KVStorePrefixIterator + KVStoreReversePrefixIterator = v1.KVStoreReversePrefixIterator + + NewStoreKVPairWriteListener = v1.NewStoreKVPairWriteListener ) // BasicRootStore defines a minimal interface for accessing root state. @@ -43,8 +63,6 @@ type BasicRootStore interface { // Returns a KVStore which has access only to the namespace of the StoreKey. // Panics if the key is not found in the schema. GetKVStore(StoreKey) KVStore - // Returns a branched whose modifications are later merged back in. - CacheRootStore() CacheRootStore } // mixin interface for trace and listen methods @@ -61,16 +79,17 @@ type rootStoreTraceListen interface { type CommitRootStore interface { BasicRootStore rootStoreTraceListen + Committer + snapshottypes.Snapshotter // Gets a read-only view of the store at a specific version. // Returns an error if the version is not found. GetVersion(int64) (BasicRootStore, error) // Closes the store and all backing transactions. Close() error - - // RootStore - Committer - snapshottypes.Snapshotter // todo: PortableStore? + // Returns a branched whose modifications are later merged back in. + CacheRootStore() CacheRootStore + // Defines the minimum version number that can be saved by this store. SetInitialVersion(uint64) error } @@ -78,21 +97,13 @@ type CommitRootStore interface { type CacheRootStore interface { BasicRootStore rootStoreTraceListen + + // Returns a branched whose modifications are later merged back in. + CacheRootStore() CacheRootStore + // Write all cached changes back to the source store. Note: this overwrites any intervening changes. Write() } // RootStorePersistentCache provides inter-block (persistent) caching capabilities for a CommitRootStore. +// TODO: placeholder. Implement and redefine this type RootStorePersistentCache = v1.MultiStorePersistentCache - -// Re-export relevant store type values and utility functions - -const StoreTypeMemory = v1.StoreTypeMemory -const StoreTypeTransient = v1.StoreTypeTransient -const StoreTypeDB = v1.StoreTypeDB -const StoreTypeSMT = v1.StoreTypeSMT -const StoreTypePersistent = v1.StoreTypePersistent - -var NewKVStoreKey = v1.NewKVStoreKey -var PrefixEndBytes = v1.PrefixEndBytes -var KVStorePrefixIterator = v1.KVStorePrefixIterator -var KVStoreReversePrefixIterator = v1.KVStoreReversePrefixIterator From f56870469618eb4461b43040c123b00402dbf9a2 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 1 Dec 2021 14:21:39 +0800 Subject: [PATCH 28/47] godoc & code comments --- db/prefix/prefix.go | 6 +++++- store/v2/root/doc.go | 19 +++++++++++++++++++ store/v2/root/store.go | 3 +++ 3 files changed, 27 insertions(+), 1 deletion(-) create mode 100644 store/v2/root/doc.go diff --git a/db/prefix/prefix.go b/db/prefix/prefix.go index 66f90ef7b24e..38a8e0034746 100644 --- a/db/prefix/prefix.go +++ b/db/prefix/prefix.go @@ -1,20 +1,24 @@ +// Prefixed DB reader/writer types let you namespace multiple DBs within a single DB. + package prefix import ( dbm "github.com/cosmos/cosmos-sdk/db" ) -// Prefix Reader/Writer lets you namespace multiple DBs within a single DB. +// prefixed Reader type prefixR struct { db dbm.DBReader prefix []byte } +// prefixed ReadWriter type prefixRW struct { db dbm.DBReadWriter prefix []byte } +// prefixed Writer type prefixW struct { db dbm.DBWriter prefix []byte diff --git a/store/v2/root/doc.go b/store/v2/root/doc.go new file mode 100644 index 000000000000..0d49e765521c --- /dev/null +++ b/store/v2/root/doc.go @@ -0,0 +1,19 @@ +// This package provides concrete implementations of the store/v2 "RootStore" types, including +// CommitRootStore, CacheRootStore, and BasicRootStore (as read-only stores at past versions). +// +// Substores are declared as part of a schema within StoreOptions. +// The schema cannot be changed once a CommitRootStore is initialized, and changes to the schema must be done +// by migrating via StoreOptions.Upgrades. If a past version is accessed, it will be loaded with the past schema. +// Stores may be declared as StoreTypePersistent, StoreTypeMemory (not persisted after close), or +// StoreTypeTransient (not persisted across commits). Non-persistent substores cannot be migrated or accessed +// in past versions. +// +// A declared persistent substore is initially empty and stores nothing in the backing DB until a value is set. +// A non-empty store is stored within a prefixed subdomain of the backing DB (using db/prefix). +// If the RootStore is configured to use a separate DBConnection for StateCommitmentDB, it will store the +// state commitment (SC) store (as an SMT) in subdomains there, and the "flat" state is stored in the main DB. +// Each substore's SC is allocated as an independent SMT, and query proofs contain two components: a proof +// of a key's (non)existence within the substore SMT, and a proof of the substore's existence within the +// RootStore (using the Merkle map proof spec (TendermintSpec)). + +package root diff --git a/store/v2/root/store.go b/store/v2/root/store.go index fdef6ee2ff36..a36521454749 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -234,6 +234,9 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { versions.Last(), opts.InitialVersion) } } + // To abide by atomicity constraints, revert the DB to the last saved version, in case it contains + // committed data in the "working" version. + // This should only happen if Store.Commit previously failed. err = db.Revert() if err != nil { return From 5a5364767c482245a6cca43d1a2a00809738ae73 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 1 Dec 2021 18:18:49 +0800 Subject: [PATCH 29/47] PR revisions --- db/prefix/prefix.go | 2 +- store/v2/root/cache_store.go | 36 ++++++++++++++++++++++++++++++++++++ store/v2/root/store.go | 31 ------------------------------- store/v2/smt/store.go | 3 ++- 4 files changed, 39 insertions(+), 33 deletions(-) create mode 100644 store/v2/root/cache_store.go diff --git a/db/prefix/prefix.go b/db/prefix/prefix.go index 38a8e0034746..52f044d18b0f 100644 --- a/db/prefix/prefix.go +++ b/db/prefix/prefix.go @@ -186,7 +186,7 @@ func cp(bz []byte) (ret []byte) { return ret } -// Returns a slice of the same length (big endian), but incremented by one. +// Returns a new slice of the same length (big endian), but incremented by one. // Returns nil on overflow (e.g. if bz bytes are all 0xFF) // CONTRACT: len(bz) > 0 func cpIncr(bz []byte) (ret []byte) { diff --git a/store/v2/root/cache_store.go b/store/v2/root/cache_store.go new file mode 100644 index 000000000000..57ccb28931de --- /dev/null +++ b/store/v2/root/cache_store.go @@ -0,0 +1,36 @@ +package root + +import ( + "github.com/cosmos/cosmos-sdk/store/cachekv" + types "github.com/cosmos/cosmos-sdk/store/v2" +) + +// GetKVStore implements BasicRootStore. +func (cs *cacheStore) GetKVStore(key types.StoreKey) types.KVStore { + ret, has := cs.substores[key.Name()] + if has { + return ret + } + ret = cachekv.NewStore(cs.source.GetKVStore(key)) + cs.substores[key.Name()] = ret + return ret +} + +// Write implements CacheRootStore. +func (cs *cacheStore) Write() { + for skey, sub := range cs.substores { + sub.Write() + delete(cs.substores, skey) + } +} + +// CacheRootStore implements BasicRootStore. +// This recursively wraps the CacheRootStore in another cache store. +func (cs *cacheStore) CacheRootStore() types.CacheRootStore { + return &cacheStore{ + source: cs, + substores: map[string]types.CacheKVStore{}, + listenerMixin: &listenerMixin{}, + traceMixin: &traceMixin{}, + } +} diff --git a/store/v2/root/store.go b/store/v2/root/store.go index a36521454749..7d49535d6fb1 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -14,7 +14,6 @@ import ( "github.com/cosmos/cosmos-sdk/db/memdb" prefixdb "github.com/cosmos/cosmos-sdk/db/prefix" util "github.com/cosmos/cosmos-sdk/internal" - "github.com/cosmos/cosmos-sdk/store/cachekv" sdkmaps "github.com/cosmos/cosmos-sdk/store/internal/maps" "github.com/cosmos/cosmos-sdk/store/prefix" types "github.com/cosmos/cosmos-sdk/store/v2" @@ -813,36 +812,6 @@ func (rs *Store) Query(req abci.RequestQuery) (res abci.ResponseQuery) { return res } -// GetKVStore implements BasicRootStore. -func (cs *cacheStore) GetKVStore(key types.StoreKey) types.KVStore { - ret, has := cs.substores[key.Name()] - if has { - return ret - } - ret = cachekv.NewStore(cs.source.GetKVStore(key)) - cs.substores[key.Name()] = ret - return ret -} - -// Write implements CacheRootStore. -func (cs *cacheStore) Write() { - for skey, sub := range cs.substores { - sub.Write() - delete(cs.substores, skey) - } -} - -// CacheRootStore implements BasicRootStore. -// This recursively wraps the CacheRootStore in another cache store. -func (cs *cacheStore) CacheRootStore() types.CacheRootStore { - return &cacheStore{ - source: cs, - substores: map[string]types.CacheKVStore{}, - listenerMixin: &listenerMixin{}, - traceMixin: &traceMixin{}, - } -} - func loadSMT(stateCommitmentTxn dbm.DBReadWriter, root []byte) *smt.Store { merkleNodes := prefixdb.NewPrefixReadWriter(stateCommitmentTxn, merkleNodePrefix) merkleValues := prefixdb.NewPrefixReadWriter(stateCommitmentTxn, merkleValuePrefix) diff --git a/store/v2/smt/store.go b/store/v2/smt/store.go index 953a08b297ed..b63d0e65ecd5 100644 --- a/store/v2/smt/store.go +++ b/store/v2/smt/store.go @@ -26,7 +26,8 @@ type Store struct { tree *smt.SparseMerkleTree } -// MapStore that wraps Get to raise InvalidKeyError +// An smt.MapStore that wraps Get to raise smt.InvalidKeyError; +// smt.SparseMerkleTree expects this error to be returned when a key is not found type dbMapStore struct{ dbm.DBReadWriter } func NewStore(nodes, values dbm.DBReadWriter) *Store { From 1432859dba52ff393dc54364a35eecf995b6cde0 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 1 Dec 2021 18:19:24 +0800 Subject: [PATCH 30/47] fix v2 mem & tran stores --- store/transient/store_test.go | 4 ++-- store/v2/mem/store.go | 4 +++- store/v2/mem/store_test.go | 39 ++++++++++++++++++++++++++++++++ store/v2/root/store.go | 5 ++-- store/v2/transient/store.go | 4 +++- store/v2/transient/store_test.go | 3 +-- 6 files changed, 50 insertions(+), 9 deletions(-) create mode 100644 store/v2/mem/store_test.go diff --git a/store/transient/store_test.go b/store/transient/store_test.go index 632b561b618a..a1c8ea90ba98 100644 --- a/store/transient/store_test.go +++ b/store/transient/store_test.go @@ -6,8 +6,8 @@ import ( "github.com/stretchr/testify/require" - "github.com/cosmos/cosmos-sdk/store/transient" - "github.com/cosmos/cosmos-sdk/store/types" + types "github.com/cosmos/cosmos-sdk/store/v2" + "github.com/cosmos/cosmos-sdk/store/v2/transient" ) var k, v = []byte("hello"), []byte("world") diff --git a/store/v2/mem/store.go b/store/v2/mem/store.go index 6b2665839c36..88d4697ca1d4 100644 --- a/store/v2/mem/store.go +++ b/store/v2/mem/store.go @@ -2,6 +2,7 @@ package mem import ( dbm "github.com/cosmos/cosmos-sdk/db" + "github.com/cosmos/cosmos-sdk/db/memdb" "github.com/cosmos/cosmos-sdk/store/types" "github.com/cosmos/cosmos-sdk/store/v2/dbadapter" ) @@ -19,7 +20,8 @@ type Store struct { } // NewStore constructs a new in-memory store. -func NewStore(db dbm.DBConnection) *Store { +func NewStore() *Store { + db := memdb.NewDB() return &Store{ Store: dbadapter.Store{DB: db.ReadWriter()}, conn: db, diff --git a/store/v2/mem/store_test.go b/store/v2/mem/store_test.go new file mode 100644 index 000000000000..822f5eaeb34f --- /dev/null +++ b/store/v2/mem/store_test.go @@ -0,0 +1,39 @@ +package mem_test + +import ( + "testing" + + "github.com/stretchr/testify/require" + + types "github.com/cosmos/cosmos-sdk/store/v2" + "github.com/cosmos/cosmos-sdk/store/v2/mem" +) + +func TestStore(t *testing.T) { + store := mem.NewStore() + key, value := []byte("key"), []byte("value") + + require.Equal(t, types.StoreTypeMemory, store.GetStoreType()) + + require.Nil(t, store.Get(key)) + store.Set(key, value) + require.Equal(t, value, store.Get(key)) + + newValue := []byte("newValue") + store.Set(key, newValue) + require.Equal(t, newValue, store.Get(key)) + + store.Delete(key) + require.Nil(t, store.Get(key)) +} + +func TestCommit(t *testing.T) { + store := mem.NewStore() + key, value := []byte("key"), []byte("value") + + store.Set(key, value) + id := store.Commit() + require.True(t, id.IsZero()) + require.True(t, store.LastCommitID().IsZero()) + require.Equal(t, value, store.Get(key)) +} diff --git a/store/v2/root/store.go b/store/v2/root/store.go index 7d49535d6fb1..5f9e5ef1adf9 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -11,7 +11,6 @@ import ( abci "github.com/tendermint/tendermint/abci/types" dbm "github.com/cosmos/cosmos-sdk/db" - "github.com/cosmos/cosmos-sdk/db/memdb" prefixdb "github.com/cosmos/cosmos-sdk/db/prefix" util "github.com/cosmos/cosmos-sdk/internal" sdkmaps "github.com/cosmos/cosmos-sdk/store/internal/maps" @@ -341,8 +340,8 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { return } } - ret.mem = mem.NewStore(memdb.NewDB()) - ret.tran = transient.NewStore(memdb.NewDB()) + ret.mem = mem.NewStore() + ret.tran = transient.NewStore() ret.schema = reg.StoreSchema return } diff --git a/store/v2/transient/store.go b/store/v2/transient/store.go index 51ab85cff011..1c68e15e169b 100644 --- a/store/v2/transient/store.go +++ b/store/v2/transient/store.go @@ -2,6 +2,7 @@ package transient import ( dbm "github.com/cosmos/cosmos-sdk/db" + "github.com/cosmos/cosmos-sdk/db/memdb" "github.com/cosmos/cosmos-sdk/store/types" "github.com/cosmos/cosmos-sdk/store/v2/dbadapter" ) @@ -18,7 +19,8 @@ type Store struct { } // NewStore constructs a new transient store. -func NewStore(db dbm.DBConnection) *Store { +func NewStore() *Store { + db := memdb.NewDB() return &Store{ Store: dbadapter.Store{DB: db.ReadWriter()}, conn: db, diff --git a/store/v2/transient/store_test.go b/store/v2/transient/store_test.go index e46ce085825f..4b051245be8f 100644 --- a/store/v2/transient/store_test.go +++ b/store/v2/transient/store_test.go @@ -6,7 +6,6 @@ import ( "github.com/stretchr/testify/require" - "github.com/cosmos/cosmos-sdk/db/memdb" "github.com/cosmos/cosmos-sdk/store/types" "github.com/cosmos/cosmos-sdk/store/v2/transient" ) @@ -14,7 +13,7 @@ import ( var k, v = []byte("hello"), []byte("world") func TestTransientStore(t *testing.T) { - tstore := transient.NewStore(memdb.NewDB()) + tstore := transient.NewStore() require.Nil(t, tstore.Get(k)) tstore.Set(k, v) require.Equal(t, v, tstore.Get(k)) From b046e204a0e52223955e98696f4853caf1b31bb5 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 1 Dec 2021 18:40:13 +0800 Subject: [PATCH 31/47] rm npSubstoreCache --- store/v2/root/store.go | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/store/v2/root/store.go b/store/v2/root/store.go index 5f9e5ef1adf9..8c529410f564 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -98,8 +98,7 @@ type Store struct { *traceMixin PersistentCache types.RootStorePersistentCache - substoreCache map[string]*substore - npSubstoreCache map[string]types.KVStore + substoreCache map[string]*substore } type substore struct { @@ -270,8 +269,7 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { StateCommitmentDB: opts.StateCommitmentDB, stateCommitmentTxn: stateCommitmentTxn, - substoreCache: map[string]*substore{}, - npSubstoreCache: map[string]types.KVStore{}, + substoreCache: map[string]*substore{}, listenerMixin: opts.listenerMixin, traceMixin: opts.traceMixin, @@ -466,12 +464,7 @@ func (rs *Store) GetKVStore(skey types.StoreKey) types.KVStore { panic(fmt.Errorf("StoreType not supported: %v", typ)) // should never happen } if sub != nil { - if cached, has := rs.npSubstoreCache[key]; has { - return cached - } - ret := prefix.NewStore(sub, []byte(key)) - rs.npSubstoreCache[key] = ret - return ret + return prefix.NewStore(sub, []byte(key)) } } else { panic(ErrStoreNotFound(key)) From 1648aa6ea5ae9fb9fb09294f105da660fc92de17 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 1 Dec 2021 18:42:25 +0800 Subject: [PATCH 32/47] impl, test trace & listen on rootstore --- store/v2/root/cache_store.go | 23 +++--- store/v2/root/store.go | 130 +++++++++++++++++---------------- store/v2/root/store_test.go | 137 +++++++++++++++++++++++++++++++++++ 3 files changed, 217 insertions(+), 73 deletions(-) diff --git a/store/v2/root/cache_store.go b/store/v2/root/cache_store.go index 57ccb28931de..5630acfc6db0 100644 --- a/store/v2/root/cache_store.go +++ b/store/v2/root/cache_store.go @@ -6,14 +6,16 @@ import ( ) // GetKVStore implements BasicRootStore. -func (cs *cacheStore) GetKVStore(key types.StoreKey) types.KVStore { - ret, has := cs.substores[key.Name()] - if has { - return ret +func (cs *cacheStore) GetKVStore(skey types.StoreKey) types.KVStore { + key := skey.Name() + sub, has := cs.substores[key] + if !has { + sub = cachekv.NewStore(cs.source.GetKVStore(skey)) + cs.substores[key] = sub } - ret = cachekv.NewStore(cs.source.GetKVStore(key)) - cs.substores[key.Name()] = ret - return ret + // Wrap with trace/listen if needed. Note: we don't cache this, so users must get a new substore after + // modifying tracers/listeners. + return cs.wrapTraceListen(sub, skey) } // Write implements CacheRootStore. @@ -28,9 +30,8 @@ func (cs *cacheStore) Write() { // This recursively wraps the CacheRootStore in another cache store. func (cs *cacheStore) CacheRootStore() types.CacheRootStore { return &cacheStore{ - source: cs, - substores: map[string]types.CacheKVStore{}, - listenerMixin: &listenerMixin{}, - traceMixin: &traceMixin{}, + source: cs, + substores: map[string]types.CacheKVStore{}, + traceListenMixin: newTraceListenMixin(), } } diff --git a/store/v2/root/store.go b/store/v2/root/store.go index 8c529410f564..ad1b89ac8cba 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -14,7 +14,9 @@ import ( prefixdb "github.com/cosmos/cosmos-sdk/db/prefix" util "github.com/cosmos/cosmos-sdk/internal" sdkmaps "github.com/cosmos/cosmos-sdk/store/internal/maps" + "github.com/cosmos/cosmos-sdk/store/listenkv" "github.com/cosmos/cosmos-sdk/store/prefix" + "github.com/cosmos/cosmos-sdk/store/tracekv" types "github.com/cosmos/cosmos-sdk/store/v2" "github.com/cosmos/cosmos-sdk/store/v2/mem" "github.com/cosmos/cosmos-sdk/store/v2/smt" @@ -66,8 +68,7 @@ type StoreConfig struct { PersistentCache types.RootStorePersistentCache Upgrades []types.StoreUpgrades - *listenerMixin - *traceMixin + *traceListenMixin } // StoreSchema defineds a mapping of substore keys to store types @@ -94,11 +95,10 @@ type Store struct { // Copied from StoreConfig Pruning types.PruningOptions InitialVersion uint64 // if - *listenerMixin - *traceMixin - PersistentCache types.RootStorePersistentCache + *traceListenMixin - substoreCache map[string]*substore + PersistentCache types.RootStorePersistentCache + substoreCache map[string]*substore } type substore struct { @@ -113,8 +113,7 @@ type substore struct { type cacheStore struct { source types.BasicRootStore substores map[string]types.CacheKVStore - *listenerMixin - *traceMixin + *traceListenMixin } // Read-only store for querying past versions @@ -137,16 +136,17 @@ type prefixRegistry struct { reserved []string } -// Mixin types that will be composed into each distinct root store variant type -type listenerMixin struct { - listeners map[types.StoreKey][]types.WriteListener -} - -type traceMixin struct { +// Mixin type that to compose trace & listen state into each root store variant type +type traceListenMixin struct { + listeners map[string][]types.WriteListener TraceWriter io.Writer TraceContext types.TraceContext } +func newTraceListenMixin() *traceListenMixin { + return &traceListenMixin{listeners: map[string][]types.WriteListener{}} +} + // DefaultStoreConfig returns a RootStore config with an empty schema, a single backing DB, // pruning with PruneDefault, no listeners and no tracer. func DefaultStoreConfig() StoreConfig { @@ -155,13 +155,7 @@ func DefaultStoreConfig() StoreConfig { prefixRegistry: prefixRegistry{ StoreSchema: StoreSchema{}, }, - listenerMixin: &listenerMixin{ - listeners: map[types.StoreKey][]types.WriteListener{}, - }, - traceMixin: &traceMixin{ - TraceWriter: nil, - TraceContext: nil, - }, + traceListenMixin: newTraceListenMixin(), } } @@ -271,9 +265,8 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { substoreCache: map[string]*substore{}, - listenerMixin: opts.listenerMixin, - traceMixin: opts.traceMixin, - PersistentCache: opts.PersistentCache, + traceListenMixin: opts.traceListenMixin, + PersistentCache: opts.PersistentCache, Pruning: opts.Pruning, InitialVersion: opts.InitialVersion, @@ -449,33 +442,36 @@ func substorePrefix(key string) []byte { } // GetKVStore implements BasicRootStore. -// Returns a substore whose contents func (rs *Store) GetKVStore(skey types.StoreKey) types.KVStore { key := skey.Name() - var sub types.KVStore - if typ, has := rs.schema[key]; has { - switch typ { - case types.StoreTypeMemory: - sub = rs.mem - case types.StoreTypeTransient: - sub = rs.tran - case types.StoreTypePersistent: - default: - panic(fmt.Errorf("StoreType not supported: %v", typ)) // should never happen - } - if sub != nil { - return prefix.NewStore(sub, []byte(key)) - } - } else { + var parent types.KVStore + typ, has := rs.schema[key] + if !has { panic(ErrStoreNotFound(key)) } - // store is persistent - ret, err := rs.getSubstore(key) - if err != nil { - panic(err) + switch typ { + case types.StoreTypeMemory: + parent = rs.mem + case types.StoreTypeTransient: + parent = rs.tran + case types.StoreTypePersistent: + default: + panic(fmt.Errorf("StoreType not supported: %v", typ)) // should never happen } - rs.substoreCache[key] = ret - return ret + var ret types.KVStore + if parent != nil { // store is non-persistent + ret = prefix.NewStore(parent, []byte(key)) + } else { // store is persistent + sub, err := rs.getSubstore(key) + if err != nil { + panic(err) + } + rs.substoreCache[key] = sub + ret = sub + } + // Wrap with trace/listen if needed. Note: we don't cache this, so users must get a new substore after + // modifying tracers/listeners. + return rs.wrapTraceListen(ret, skey) } // Gets a persistent substore. This reads, but does not update the substore cache @@ -685,10 +681,9 @@ func (rs *Store) GetVersion(version int64) (types.BasicRootStore, error) { // CacheRootStore implements BasicRootStore. func (rs *Store) CacheRootStore() types.CacheRootStore { return &cacheStore{ - source: rs, - substores: map[string]types.CacheKVStore{}, - listenerMixin: &listenerMixin{}, - traceMixin: &traceMixin{}, + source: rs, + substores: map[string]types.CacheKVStore{}, + traceListenMixin: newTraceListenMixin(), } } @@ -865,30 +860,41 @@ func (pr *prefixRegistry) ReservePrefix(key string, typ types.StoreType) error { return nil } -func (lreg *listenerMixin) AddListeners(key types.StoreKey, listeners []types.WriteListener) { - if ls, has := lreg.listeners[key]; has { - lreg.listeners[key] = append(ls, listeners...) +func (tlm *traceListenMixin) AddListeners(skey types.StoreKey, listeners []types.WriteListener) { + key := skey.Name() + if ls, has := tlm.listeners[key]; has { + tlm.listeners[key] = append(ls, listeners...) } else { - lreg.listeners[key] = listeners + tlm.listeners[key] = listeners } } // ListeningEnabled returns if listening is enabled for a specific KVStore -func (lreg *listenerMixin) ListeningEnabled(key types.StoreKey) bool { - if ls, has := lreg.listeners[key]; has { +func (tlm *traceListenMixin) ListeningEnabled(key types.StoreKey) bool { + if ls, has := tlm.listeners[key.Name()]; has { return len(ls) != 0 } return false } -func (treg *traceMixin) TracingEnabled() bool { - return treg.TraceWriter != nil +func (tlm *traceListenMixin) TracingEnabled() bool { + return tlm.TraceWriter != nil +} +func (tlm *traceListenMixin) SetTracer(w io.Writer) { + tlm.TraceWriter = w } -func (treg *traceMixin) SetTracer(w io.Writer) { - treg.TraceWriter = w +func (tlm *traceListenMixin) SetTraceContext(tc types.TraceContext) { + tlm.TraceContext = tc } -func (treg *traceMixin) SetTraceContext(tc types.TraceContext) { - treg.TraceContext = tc + +func (tlm *traceListenMixin) wrapTraceListen(store types.KVStore, skey types.StoreKey) types.KVStore { + if tlm.TracingEnabled() { + store = tracekv.NewStore(store, tlm.TraceWriter, tlm.TraceContext) + } + if wls, has := tlm.listeners[skey.Name()]; has && len(wls) != 0 { + store = listenkv.NewStore(store, skey, wls) + } + return store } func (s *Store) GetPruning() types.PruningOptions { return s.Pruning } diff --git a/store/v2/root/store_test.go b/store/v2/root/store_test.go index 228a2a58d868..7b5caaf67d3b 100644 --- a/store/v2/root/store_test.go +++ b/store/v2/root/store_test.go @@ -1,6 +1,7 @@ package root import ( + "bytes" "math" "testing" @@ -8,6 +9,8 @@ import ( abci "github.com/tendermint/tendermint/abci/types" + "github.com/cosmos/cosmos-sdk/codec" + codecTypes "github.com/cosmos/cosmos-sdk/codec/types" dbm "github.com/cosmos/cosmos-sdk/db" "github.com/cosmos/cosmos-sdk/db/memdb" types "github.com/cosmos/cosmos-sdk/store/v2" @@ -824,3 +827,137 @@ func TestRootStoreMigration(t *testing.T) { view.GetKVStore(skey_4) }) } + +func TestTrace(t *testing.T) { + key, value := []byte("test-key"), []byte("test-value") + tctx := types.TraceContext(map[string]interface{}{"blockHeight": 64}) + + expected_Set := "{\"operation\":\"write\",\"key\":\"dGVzdC1rZXk=\",\"value\":\"dGVzdC12YWx1ZQ==\",\"metadata\":{\"blockHeight\":64}}\n" + expected_Get := "{\"operation\":\"read\",\"key\":\"dGVzdC1rZXk=\",\"value\":\"dGVzdC12YWx1ZQ==\",\"metadata\":{\"blockHeight\":64}}\n" + expected_Get_missing := "{\"operation\":\"read\",\"key\":\"dGVzdC1rZXk=\",\"value\":\"\",\"metadata\":{\"blockHeight\":64}}\n" + expected_Delete := "{\"operation\":\"delete\",\"key\":\"dGVzdC1rZXk=\",\"value\":\"\",\"metadata\":{\"blockHeight\":64}}\n" + expected_IterKey := "{\"operation\":\"iterKey\",\"key\":\"dGVzdC1rZXk=\",\"value\":\"\",\"metadata\":{\"blockHeight\":64}}\n" + expected_IterValue := "{\"operation\":\"iterValue\",\"key\":\"\",\"value\":\"dGVzdC12YWx1ZQ==\",\"metadata\":{\"blockHeight\":64}}\n" + + db := memdb.NewDB() + opts := simpleStoreConfig(t) + require.NoError(t, opts.ReservePrefix(skey_2.Name(), types.StoreTypeMemory)) + require.NoError(t, opts.ReservePrefix(skey_3.Name(), types.StoreTypeTransient)) + + store, err := NewStore(db, opts) + require.NoError(t, err) + store.SetTraceContext(tctx) + require.False(t, store.TracingEnabled()) + + var buf bytes.Buffer + store.SetTracer(&buf) + require.True(t, store.TracingEnabled()) + + for _, skey := range []types.StoreKey{skey_1, skey_2, skey_3} { + buf.Reset() + store.GetKVStore(skey).Get(key) + require.Equal(t, expected_Get_missing, buf.String()) + + buf.Reset() + store.GetKVStore(skey).Set(key, value) + require.Equal(t, expected_Set, buf.String()) + + buf.Reset() + require.Equal(t, value, store.GetKVStore(skey).Get(key)) + require.Equal(t, expected_Get, buf.String()) + + iter := store.GetKVStore(skey).Iterator(nil, nil) + buf.Reset() + require.Equal(t, key, iter.Key()) + require.Equal(t, expected_IterKey, buf.String()) + buf.Reset() + require.Equal(t, value, iter.Value()) + require.Equal(t, expected_IterValue, buf.String()) + require.NoError(t, iter.Close()) + + buf.Reset() + store.GetKVStore(skey).Delete(key) + require.Equal(t, expected_Delete, buf.String()) + + } + store.SetTracer(nil) + require.False(t, store.TracingEnabled()) + require.NoError(t, store.Close()) +} + +func TestListeners(t *testing.T) { + kvPairs := []types.KVPair{ + {Key: []byte{1}, Value: []byte("v1")}, + {Key: []byte{2}, Value: []byte("v2")}, + {Key: []byte{3}, Value: []byte("v3")}, + } + + testCases := []struct { + key []byte + value []byte + skey types.StoreKey + }{ + { + key: kvPairs[0].Key, + value: kvPairs[0].Value, + skey: skey_1, + }, + { + key: kvPairs[1].Key, + value: kvPairs[1].Value, + skey: skey_2, + }, + { + key: kvPairs[2].Key, + value: kvPairs[2].Value, + skey: skey_3, + }, + } + + var interfaceRegistry = codecTypes.NewInterfaceRegistry() + var marshaller = codec.NewProtoCodec(interfaceRegistry) + + db := memdb.NewDB() + opts := simpleStoreConfig(t) + require.NoError(t, opts.ReservePrefix(skey_2.Name(), types.StoreTypeMemory)) + require.NoError(t, opts.ReservePrefix(skey_3.Name(), types.StoreTypeTransient)) + + store, err := NewStore(db, opts) + require.NoError(t, err) + + for i, tc := range testCases { + var buf bytes.Buffer + listener := types.NewStoreKVPairWriteListener(&buf, marshaller) + store.AddListeners(tc.skey, []types.WriteListener{listener}) + require.True(t, store.ListeningEnabled(tc.skey)) + + // Set case + expected := types.StoreKVPair{ + Key: tc.key, + Value: tc.value, + StoreKey: tc.skey.Name(), + Delete: false, + } + var kvpair types.StoreKVPair + + buf.Reset() + store.GetKVStore(tc.skey).Set(tc.key, tc.value) + require.NoError(t, marshaller.UnmarshalLengthPrefixed(buf.Bytes(), &kvpair)) + require.Equal(t, expected, kvpair, i) + + // Delete case + expected = types.StoreKVPair{ + Key: tc.key, + Value: nil, + StoreKey: tc.skey.Name(), + Delete: true, + } + kvpair = types.StoreKVPair{} + + buf.Reset() + store.GetKVStore(tc.skey).Delete(tc.key) + require.NoError(t, marshaller.UnmarshalLengthPrefixed(buf.Bytes(), &kvpair)) + require.Equal(t, expected, kvpair, i) + } + require.NoError(t, store.Close()) +} From 91581f8b3345d732de704b3f96dde17d6aea3111 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 2 Dec 2021 00:44:57 +0800 Subject: [PATCH 33/47] test fix --- store/v2/root/store_test.go | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/store/v2/root/store_test.go b/store/v2/root/store_test.go index 7b5caaf67d3b..958f8e300f95 100644 --- a/store/v2/root/store_test.go +++ b/store/v2/root/store_test.go @@ -217,21 +217,18 @@ func TestCommit(t *testing.T) { require.NoError(t, err) require.Zero(t, store.LastCommitID()) idNew := store.Commit() + + // Adding one record changes the hash s1 := store.GetKVStore(skey_1) s1.Set([]byte{0}, []byte{0}) idOne := store.Commit() require.Equal(t, idNew.Version+1, idOne.Version) require.NotEqual(t, idNew.Hash, idOne.Hash) - // // Hash of emptied store is same as new store - // opts.Upgrades = []types.StoreUpgrades{ - // types.StoreUpgrades{Deleted: []string{skey_1.Name()}}, - // } - // store.Close() - // store, err = NewStore(db, opts) - // require.NoError(t, err) - // idEmptied := store.Commit() - // require.Equal(t, idNew.Hash, idEmptied.Hash) + // Hash of emptied store is same as new store + s1.Delete([]byte{0}) + idEmptied := store.Commit() + require.Equal(t, idNew.Hash, idEmptied.Hash) previd := idOne for i := byte(1); i < 5; i++ { @@ -660,8 +657,6 @@ func TestGetVersion(t *testing.T) { require.NoError(t, err) cid := store.Commit() - // opts := DefaultStoreConfig() - view, err := store.GetVersion(cid.Version) require.NoError(t, err) subview := view.GetKVStore(skey_1) From 282ea90c4680df166c43f05c48f7811b5885b5c5 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 2 Dec 2021 00:52:53 +0800 Subject: [PATCH 34/47] rename RootStore => MultiStore --- CHANGELOG.md | 2 +- docs/core/store.md | 16 +++++++-------- store/v2/root/cache_store.go | 10 +++++----- store/v2/root/doc.go | 10 +++++----- store/v2/root/store.go | 38 ++++++++++++++++++------------------ store/v2/root/store_test.go | 4 ++-- store/v2/types.go | 26 ++++++++++++------------ 7 files changed, 53 insertions(+), 53 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9b40475d5e58..03cec141976a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -54,7 +54,7 @@ Ref: https://keepachangelog.com/en/1.0.0/ * [\#10379](https://github.com/cosmos/cosmos-sdk/pull/10379) Add validation to `x/upgrade` CLI `software-upgrade` command `--plan-info` value. * [\#10561](https://github.com/cosmos/cosmos-sdk/pull/10561) Add configurable IAVL cache size to app.toml * [\10507](https://github.com/cosmos/cosmos-sdk/pull/10507) Add middleware for tx priority. -* [\#10430](https://github.com/cosmos/cosmos-sdk/pull/10430) ADR-040: Add `RootStore` implementation +* [\#10430](https://github.com/cosmos/cosmos-sdk/pull/10430) ADR-040: Add store/v2 `MultiStore` implementation ### Improvements diff --git a/docs/core/store.md b/docs/core/store.md index 99123a5a3f87..1a688a5df054 100644 --- a/docs/core/store.md +++ b/docs/core/store.md @@ -244,27 +244,27 @@ See [ADR-040](../architecture/adr-040-storage-and-smt-state-commitments.md) for An interface providing only the basic CRUD functionality (`Get`, `Set`, `Has`, and `Delete` methods), without iteration or caching. This is used to partially expose components of a larger store, such as a `flat.Store`. -## Root Store +## MultiStore -This is the new interface for the main client store, replacing the function of `MultiStore`. There are a few significant differences in behavior compared with `MultiStore`: +This is the new interface (or set of interfaces) for the main client store, replacing the function of `MultiStore`. There are a few significant differences in behavior compared with `MultiStore`: * Commits are atomic and are performed on the entire store state; individual substores cannot be committed separately and cannot have different version numbers. * The store's current version and version history track that of the backing `db.DBConnection`. Past versions are accessible read-only. * The set of valid substores is defined in at initialization and cannot be updated dynamically in an existing store instance. -### `CommitRootStore` +### `CommitMultiStore` -This is the main interface for persisent application state, analogous to `CommitMultiStore`. - * Past versions are accessed with `GetVersion`, which returns a `BasicRootStore`. +This is the main interface for persisent application state, analogous to the original `CommitMultiStore`. + * Past versions are accessed with `GetVersion`, which returns a `BasicMultiStore`. * Substores are accessed with `GetKVStore`. Trying to get a substore that was not defined at initialization will cause a panic. * `Close` must be called to release the DB resources being used by the store. -### `BasicRootStore` +### `BasicMultiStore` -A minimal interface that only allows accessing substores. Note: substores returned by `BasicRootStore.GetKVStore` are read-only. +A minimal interface that only allows accessing substores. Note: substores returned by `BasicMultiStore.GetKVStore` are read-only. ### Implementation (`root.Store`) -The canonical implementation of `RootStore` is in `store/v2/root`. It internally decouples the concerns of state storage and state commitment: values are stored in, and read directly from, the backing key-value database, but are also mapped in a logically separate store which generates cryptographic proofs (the *state-commitment* store). +The canonical implementation of `MultiStore` is in `store/v2/root`. It internally decouples the concerns of state storage and state commitment: values are stored in, and read directly from, the backing key-value database, but are also mapped in a logically separate store which generates cryptographic proofs (the *state-commitment* store). The state-commitment component of each substore is implemented as an independent `smt.Store`. Internally, each substore is allocated in a separate partition within the backing DB, such that commits apply to the state of all substores. Likewise, past version state includes the state of all substore storage and state-commitment stores. diff --git a/store/v2/root/cache_store.go b/store/v2/root/cache_store.go index 5630acfc6db0..915ea156c201 100644 --- a/store/v2/root/cache_store.go +++ b/store/v2/root/cache_store.go @@ -5,7 +5,7 @@ import ( types "github.com/cosmos/cosmos-sdk/store/v2" ) -// GetKVStore implements BasicRootStore. +// GetKVStore implements BasicMultiStore. func (cs *cacheStore) GetKVStore(skey types.StoreKey) types.KVStore { key := skey.Name() sub, has := cs.substores[key] @@ -18,7 +18,7 @@ func (cs *cacheStore) GetKVStore(skey types.StoreKey) types.KVStore { return cs.wrapTraceListen(sub, skey) } -// Write implements CacheRootStore. +// Write implements CacheMultiStore. func (cs *cacheStore) Write() { for skey, sub := range cs.substores { sub.Write() @@ -26,9 +26,9 @@ func (cs *cacheStore) Write() { } } -// CacheRootStore implements BasicRootStore. -// This recursively wraps the CacheRootStore in another cache store. -func (cs *cacheStore) CacheRootStore() types.CacheRootStore { +// CacheMultiStore implements BasicMultiStore. +// This recursively wraps the CacheMultiStore in another cache store. +func (cs *cacheStore) CacheMultiStore() types.CacheMultiStore { return &cacheStore{ source: cs, substores: map[string]types.CacheKVStore{}, diff --git a/store/v2/root/doc.go b/store/v2/root/doc.go index 0d49e765521c..76469ab11a69 100644 --- a/store/v2/root/doc.go +++ b/store/v2/root/doc.go @@ -1,8 +1,8 @@ -// This package provides concrete implementations of the store/v2 "RootStore" types, including -// CommitRootStore, CacheRootStore, and BasicRootStore (as read-only stores at past versions). +// This package provides concrete implementations of the store/v2 "MultiStore" types, including +// CommitMultiStore, CacheMultiStore, and BasicMultiStore (as read-only stores at past versions). // // Substores are declared as part of a schema within StoreOptions. -// The schema cannot be changed once a CommitRootStore is initialized, and changes to the schema must be done +// The schema cannot be changed once a CommitMultiStore is initialized, and changes to the schema must be done // by migrating via StoreOptions.Upgrades. If a past version is accessed, it will be loaded with the past schema. // Stores may be declared as StoreTypePersistent, StoreTypeMemory (not persisted after close), or // StoreTypeTransient (not persisted across commits). Non-persistent substores cannot be migrated or accessed @@ -10,10 +10,10 @@ // // A declared persistent substore is initially empty and stores nothing in the backing DB until a value is set. // A non-empty store is stored within a prefixed subdomain of the backing DB (using db/prefix). -// If the RootStore is configured to use a separate DBConnection for StateCommitmentDB, it will store the +// If the MultiStore is configured to use a separate DBConnection for StateCommitmentDB, it will store the // state commitment (SC) store (as an SMT) in subdomains there, and the "flat" state is stored in the main DB. // Each substore's SC is allocated as an independent SMT, and query proofs contain two components: a proof // of a key's (non)existence within the substore SMT, and a proof of the substore's existence within the -// RootStore (using the Merkle map proof spec (TendermintSpec)). +// MultiStore (using the Merkle map proof spec (TendermintSpec)). package root diff --git a/store/v2/root/store.go b/store/v2/root/store.go index ad1b89ac8cba..1d94a23eaf20 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -26,11 +26,11 @@ import ( ) var ( - _ types.Queryable = (*Store)(nil) - _ types.CommitRootStore = (*Store)(nil) - _ types.CacheRootStore = (*cacheStore)(nil) - _ types.BasicRootStore = (*viewStore)(nil) - _ types.KVStore = (*substore)(nil) + _ types.Queryable = (*Store)(nil) + _ types.CommitMultiStore = (*Store)(nil) + _ types.CacheMultiStore = (*cacheStore)(nil) + _ types.BasicMultiStore = (*viewStore)(nil) + _ types.KVStore = (*substore)(nil) ) var ( @@ -54,7 +54,7 @@ func ErrStoreNotFound(skey string) error { return fmt.Errorf("store does not exist for key: %s", skey) } -// StoreConfig is used to define a schema and other options and pass them to the RootStore constructor. +// StoreConfig is used to define a schema and other options and pass them to the MultiStore constructor. type StoreConfig struct { // Version pruning options for backing DBs. Pruning types.PruningOptions @@ -65,7 +65,7 @@ type StoreConfig struct { StateCommitmentDB dbm.DBConnection prefixRegistry - PersistentCache types.RootStorePersistentCache + PersistentCache types.MultiStorePersistentCache Upgrades []types.StoreUpgrades *traceListenMixin @@ -74,7 +74,7 @@ type StoreConfig struct { // StoreSchema defineds a mapping of substore keys to store types type StoreSchema map[string]types.StoreType -// Store is the main persistent store type implementing CommitRootStore. +// Store is the main persistent store type implementing CommitMultiStore. // Substores consist of an SMT-based state commitment store and state storage. // Substores must be reserved in the StoreConfig or defined as part of a StoreUpgrade in order to be valid. // Note: @@ -97,7 +97,7 @@ type Store struct { InitialVersion uint64 // if *traceListenMixin - PersistentCache types.RootStorePersistentCache + PersistentCache types.MultiStorePersistentCache substoreCache map[string]*substore } @@ -111,7 +111,7 @@ type substore struct { // Branched state type cacheStore struct { - source types.BasicRootStore + source types.BasicMultiStore substores map[string]types.CacheKVStore *traceListenMixin } @@ -147,7 +147,7 @@ func newTraceListenMixin() *traceListenMixin { return &traceListenMixin{listeners: map[string][]types.WriteListener{}} } -// DefaultStoreConfig returns a RootStore config with an empty schema, a single backing DB, +// DefaultStoreConfig returns a MultiStore config with an empty schema, a single backing DB, // pruning with PruneDefault, no listeners and no tracer. func DefaultStoreConfig() StoreConfig { return StoreConfig{ @@ -159,7 +159,7 @@ func DefaultStoreConfig() StoreConfig { } } -// Returns true for valid store types for a RootStore schema +// Returns true for valid store types for a MultiStore schema func validSubStoreType(sst types.StoreType) bool { switch sst { case types.StoreTypePersistent: @@ -211,7 +211,7 @@ func readSavedSchema(bucket dbm.DBReader) (*prefixRegistry, error) { return &ret, nil } -// NewStore constructs a RootStore directly from a database. +// NewStore constructs a MultiStore directly from a database. // Creates a new store if no data exists; otherwise loads existing data. func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { versions, err := db.Versions() @@ -441,7 +441,7 @@ func substorePrefix(key string) []byte { return append(contentPrefix, key...) } -// GetKVStore implements BasicRootStore. +// GetKVStore implements BasicMultiStore. func (rs *Store) GetKVStore(skey types.StoreKey) types.KVStore { key := skey.Name() var parent types.KVStore @@ -667,19 +667,19 @@ func (s *Store) LastCommitID() types.CommitID { return types.CommitID{Version: int64(last), Hash: hash} } -// SetInitialVersion implements CommitRootStore. +// SetInitialVersion implements CommitMultiStore. func (rs *Store) SetInitialVersion(version uint64) error { rs.InitialVersion = uint64(version) return nil } -// GetVersion implements CommitRootStore. -func (rs *Store) GetVersion(version int64) (types.BasicRootStore, error) { +// GetVersion implements CommitMultiStore. +func (rs *Store) GetVersion(version int64) (types.BasicMultiStore, error) { return rs.getView(version) } -// CacheRootStore implements BasicRootStore. -func (rs *Store) CacheRootStore() types.CacheRootStore { +// CacheMultiStore implements BasicMultiStore. +func (rs *Store) CacheMultiStore() types.CacheMultiStore { return &cacheStore{ source: rs, substores: map[string]types.CacheKVStore{}, diff --git a/store/v2/root/store_test.go b/store/v2/root/store_test.go index 958f8e300f95..c631ad9d4cbf 100644 --- a/store/v2/root/store_test.go +++ b/store/v2/root/store_test.go @@ -632,7 +632,7 @@ func TestStoreConfig(t *testing.T) { require.Error(t, opts.ReservePrefix(skey_3.Name(), types.StoreTypePersistent)) } -func TestRootStoreBasic(t *testing.T) { +func TestMultiStoreBasic(t *testing.T) { opts := DefaultStoreConfig() err := opts.ReservePrefix(skey_1.Name(), types.StoreTypePersistent) require.NoError(t, err) @@ -685,7 +685,7 @@ func TestGetVersion(t *testing.T) { require.Equal(t, []byte{0}, subview.Get([]byte{0})) } -func TestRootStoreMigration(t *testing.T) { +func TestMultiStoreMigration(t *testing.T) { db := memdb.NewDB() opts := storeConfig123(t) store, err := NewStore(db, opts) diff --git a/store/v2/types.go b/store/v2/types.go index 7e9db7697e23..6975cbdc49c3 100644 --- a/store/v2/types.go +++ b/store/v2/types.go @@ -58,8 +58,8 @@ var ( NewStoreKVPairWriteListener = v1.NewStoreKVPairWriteListener ) -// BasicRootStore defines a minimal interface for accessing root state. -type BasicRootStore interface { +// BasicMultiStore defines a minimal interface for accessing root state. +type BasicMultiStore interface { // Returns a KVStore which has access only to the namespace of the StoreKey. // Panics if the key is not found in the schema. GetKVStore(StoreKey) KVStore @@ -74,36 +74,36 @@ type rootStoreTraceListen interface { AddListeners(key StoreKey, listeners []WriteListener) } -// CommitRootStore defines a complete interface for persistent root state, including +// CommitMultiStore defines a complete interface for persistent root state, including // (read-only) access to past versions, pruning, trace/listen, and state snapshots. -type CommitRootStore interface { - BasicRootStore +type CommitMultiStore interface { + BasicMultiStore rootStoreTraceListen Committer snapshottypes.Snapshotter // Gets a read-only view of the store at a specific version. // Returns an error if the version is not found. - GetVersion(int64) (BasicRootStore, error) + GetVersion(int64) (BasicMultiStore, error) // Closes the store and all backing transactions. Close() error // Returns a branched whose modifications are later merged back in. - CacheRootStore() CacheRootStore + CacheMultiStore() CacheMultiStore // Defines the minimum version number that can be saved by this store. SetInitialVersion(uint64) error } -// CacheRootStore defines a branch of the root state which can be written back to the source store. -type CacheRootStore interface { - BasicRootStore +// CacheMultiStore defines a branch of the root state which can be written back to the source store. +type CacheMultiStore interface { + BasicMultiStore rootStoreTraceListen // Returns a branched whose modifications are later merged back in. - CacheRootStore() CacheRootStore + CacheMultiStore() CacheMultiStore // Write all cached changes back to the source store. Note: this overwrites any intervening changes. Write() } -// RootStorePersistentCache provides inter-block (persistent) caching capabilities for a CommitRootStore. +// MultiStorePersistentCache provides inter-block (persistent) caching capabilities for a CommitMultiStore. // TODO: placeholder. Implement and redefine this -type RootStorePersistentCache = v1.MultiStorePersistentCache +type MultiStorePersistentCache = v1.MultiStorePersistentCache From 40584b7e10ff1df8718951396bbba6a6e10cc4a4 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 2 Dec 2021 18:58:46 +0800 Subject: [PATCH 35/47] make: don't build rocksdb unless ENABLE_ROCKSDB=true --- .github/workflows/test.yml | 1 + Makefile | 31 +++++++++++++++++++++++-------- db/rocksdb/batch.go | 2 ++ db/rocksdb/db.go | 2 ++ db/rocksdb/db_test.go | 2 ++ db/rocksdb/iterator.go | 2 ++ 6 files changed, 32 insertions(+), 8 deletions(-) diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index e6294909c48c..5494d719c1dd 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -49,6 +49,7 @@ jobs: test-submodules: runs-on: ubuntu-latest container: tendermintdev/docker-tm-db-testing + env: {ENABLE_ROCKSDB: true} steps: - uses: actions/checkout@v2 - uses: actions/setup-go@v2.1.4 diff --git a/Makefile b/Makefile index 973807324df3..3c4b844eb484 100644 --- a/Makefile +++ b/Makefile @@ -14,6 +14,9 @@ HTTPS_GIT := https://github.com/cosmos/cosmos-sdk.git DOCKER := $(shell which docker) DOCKER_BUF := $(DOCKER) run --rm -v $(CURDIR):/workspace --workdir /workspace bufbuild/buf:1.0.0-rc8 PROJECT_NAME = $(shell git remote get-url origin | xargs basename -s .git) +# RocksDB is a native dependency, so we don't assume the library is installed. +# Instead, it must be explicitly enabled and we warn when it is not. +ENABLE_ROCKSDB ?= false export GO111MODULE = on @@ -61,6 +64,13 @@ ldflags = -X github.com/cosmos/cosmos-sdk/version.Name=sim \ -X "github.com/cosmos/cosmos-sdk/version.BuildTags=$(build_tags_comma_sep)" \ -X github.com/tendermint/tendermint/version.TMCoreSemVer=$(TMVERSION) +ifeq ($(ENABLE_ROCKSDB),true) + BUILD_TAGS += rocksdb_build + test_tags += rocksdb_build +else + $(warning RocksDB support is disabled; to build and test with RocksDB support, set ENABLE_ROCKSDB=true) +endif + # DB backend selection ifeq (cleveldb,$(findstring cleveldb,$(COSMOS_BUILD_OPTIONS))) ldflags += -X github.com/cosmos/cosmos-sdk/types.DBBackend=cleveldb @@ -71,6 +81,9 @@ ifeq (badgerdb,$(findstring badgerdb,$(COSMOS_BUILD_OPTIONS))) endif # handle rocksdb ifeq (rocksdb,$(findstring rocksdb,$(COSMOS_BUILD_OPTIONS))) + ifneq ($(ENABLE_ROCKSDB),true) + $(error Cannot use RocksDB backend unless ENABLE_ROCKSDB=true) + endif CGO_ENABLED=1 BUILD_TAGS += rocksdb ldflags += -X github.com/cosmos/cosmos-sdk/types.DBBackend=rocksdb @@ -205,22 +218,24 @@ TEST_TARGETS := test-unit test-unit-amino test-unit-proto test-ledger-mock test- # Test runs-specific rules. To add a new test target, just add # a new rule, customise ARGS or TEST_PACKAGES ad libitum, and # append the new rule to the TEST_TARGETS list. -test-unit: ARGS=-tags='cgo ledger test_ledger_mock norace' -test-unit-amino: ARGS=-tags='ledger test_ledger_mock test_amino norace' -test-ledger: ARGS=-tags='cgo ledger norace' -test-ledger-mock: ARGS=-tags='ledger test_ledger_mock norace' -test-race: ARGS=-race -tags='cgo ledger test_ledger_mock' +test-unit: test_tags += cgo ledger test_ledger_mock norace +test-unit-amino: test_tags += ledger test_ledger_mock test_amino norace +test-ledger: test_tags += cgo ledger norace +test-ledger-mock: test_tags += ledger test_ledger_mock norace +test-race: test_tags += cgo ledger test_ledger_mock +test-race: ARGS=-race test-race: TEST_PACKAGES=$(PACKAGES_NOSIMULATION) $(TEST_TARGETS): run-tests # check-* compiles and collects tests without running them # note: go test -c doesn't support multiple packages yet (https://github.com/golang/go/issues/15513) CHECK_TEST_TARGETS := check-test-unit check-test-unit-amino -check-test-unit: ARGS=-tags='cgo ledger test_ledger_mock norace' -check-test-unit-amino: ARGS=-tags='ledger test_ledger_mock test_amino norace' +check-test-unit: test_tags += cgo ledger test_ledger_mock norace +check-test-unit-amino: test_tags += ledger test_ledger_mock test_amino norace $(CHECK_TEST_TARGETS): EXTRA_ARGS=-run=none $(CHECK_TEST_TARGETS): run-tests +ARGS += -tags "$(test_tags)" SUB_MODULES = $(shell find . -type f -name 'go.mod' -print0 | xargs -0 -n1 dirname | sort) CURRENT_DIR = $(shell pwd) run-tests: @@ -485,7 +500,7 @@ localnet-build-dlv: localnet-build-nodes: $(DOCKER) run --rm -v $(CURDIR)/.testnets:/data cosmossdk/simd \ - testnet init-files --v 4 --starting-ip-address 192.168.10.2 --keyring-backend=test + testnet init-files --v 4 --starting-ip-address 192.168.10.2 --keyring-backend=test docker-compose up -d localnet-stop: diff --git a/db/rocksdb/batch.go b/db/rocksdb/batch.go index e78d71eaa0e5..54be2ec2add2 100644 --- a/db/rocksdb/batch.go +++ b/db/rocksdb/batch.go @@ -1,3 +1,5 @@ +//go:build rocksdb_build + package rocksdb import ( diff --git a/db/rocksdb/db.go b/db/rocksdb/db.go index 4b69172b5178..058bf9a5fc81 100644 --- a/db/rocksdb/db.go +++ b/db/rocksdb/db.go @@ -1,3 +1,5 @@ +//go:build rocksdb_build + package rocksdb import ( diff --git a/db/rocksdb/db_test.go b/db/rocksdb/db_test.go index 5e8bae425102..d1d8f1926795 100644 --- a/db/rocksdb/db_test.go +++ b/db/rocksdb/db_test.go @@ -1,3 +1,5 @@ +//go:build rocksdb_build + package rocksdb import ( diff --git a/db/rocksdb/iterator.go b/db/rocksdb/iterator.go index 2b7f7b74e48c..c0b83778e20d 100644 --- a/db/rocksdb/iterator.go +++ b/db/rocksdb/iterator.go @@ -1,3 +1,5 @@ +//go:build rocksdb_build + package rocksdb import ( From f4edb06b73898895e41bad79d0deafb9cbbd1f1b Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 2 Dec 2021 20:38:05 +0800 Subject: [PATCH 36/47] set rocksdb_build tag in CI submodule tests --- .github/workflows/test.yml | 1 - scripts/module-tests.sh | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index 5494d719c1dd..e6294909c48c 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -49,7 +49,6 @@ jobs: test-submodules: runs-on: ubuntu-latest container: tendermintdev/docker-tm-db-testing - env: {ENABLE_ROCKSDB: true} steps: - uses: actions/checkout@v2 - uses: actions/setup-go@v2.1.4 diff --git a/scripts/module-tests.sh b/scripts/module-tests.sh index 86998b5aa99f..b6be64240e3b 100644 --- a/scripts/module-tests.sh +++ b/scripts/module-tests.sh @@ -21,7 +21,7 @@ execute_mod_tests() { echo ">>> running $go_mod tests" cd $mod_dir; - go test -mod=readonly -timeout 30m -coverprofile=${root_dir}/${coverage_file}.tmp -covermode=atomic -tags='norace ledger test_ledger_mock' ./... + go test -mod=readonly -timeout 30m -coverprofile=${root_dir}/${coverage_file}.tmp -covermode=atomic -tags='norace ledger test_ledger_mock rocksdb_build' ./... local ret=$? echo "test return: " $ret; cd -; From ad227e0552c1cde887261ee6c36f3e45d9f61865 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 2 Dec 2021 20:49:52 +0800 Subject: [PATCH 37/47] Update gorocksdb replace directive to use cosmos fork --- db/go.mod | 4 ++-- db/go.sum | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/db/go.mod b/db/go.mod index b745bfd6cbeb..0010c3c5c891 100644 --- a/db/go.mod +++ b/db/go.mod @@ -31,6 +31,6 @@ require ( gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b // indirect ) -// FIXME: gorocksdb bindings for OptimisticTransactionDB are not merged upstream, so we use a fork +// Note: gorocksdb bindings for OptimisticTransactionDB are not merged upstream, so we use a fork // See https://github.com/tecbot/gorocksdb/pull/216 -replace github.com/tecbot/gorocksdb => github.com/roysc/gorocksdb v1.1.1 +replace github.com/tecbot/gorocksdb => github.com/cosmos/gorocksdb v1.1.1 diff --git a/db/go.sum b/db/go.sum index e7fef9f14ffb..ed305b1961de 100644 --- a/db/go.sum +++ b/db/go.sum @@ -14,6 +14,8 @@ github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGX github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= +github.com/cosmos/gorocksdb v1.1.1 h1:N0OqpEKXgsi2qtDm8T1+AlNMXkTm6s1jowYf7/4pH5I= +github.com/cosmos/gorocksdb v1.1.1/go.mod h1:b/U29r/CtguX3TF7mKG1Jjn4APDqh4wECshxXdiWHpA= github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= @@ -99,8 +101,6 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/roysc/gorocksdb v1.1.1 h1:5qKNwi7V/AchRMjyVf5TMCcZP70ro+VyaRmQxzpRvd4= -github.com/roysc/gorocksdb v1.1.1/go.mod h1:b/U29r/CtguX3TF7mKG1Jjn4APDqh4wECshxXdiWHpA= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= From c4d6e425ec95d7ce113d98e5cc162f6cc2dcb8a1 Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Wed, 8 Dec 2021 19:12:43 +0530 Subject: [PATCH 38/47] feat: add the snapshot sync for v2 store --- docs/core/proto-docs.md | 4353 ++++------------- .../cosmos/base/store/v1beta1/snapshot.proto | 22 +- store/types/snapshot.pb.go | 666 ++- store/v2/root/store.go | 189 +- 4 files changed, 1722 insertions(+), 3508 deletions(-) diff --git a/docs/core/proto-docs.md b/docs/core/proto-docs.md index 7dd29cfbd7d2..42008ed98768 100644 --- a/docs/core/proto-docs.md +++ b/docs/core/proto-docs.md @@ -1,5 +1,7 @@ + # Protobuf Documentation + ## Table of Contents @@ -8,14 +10,14 @@ - [BaseAccount](#cosmos.auth.v1beta1.BaseAccount) - [ModuleAccount](#cosmos.auth.v1beta1.ModuleAccount) - [Params](#cosmos.auth.v1beta1.Params) - + - [cosmos/auth/v1beta1/genesis.proto](#cosmos/auth/v1beta1/genesis.proto) - [GenesisState](#cosmos.auth.v1beta1.GenesisState) - + - [cosmos/base/query/v1beta1/pagination.proto](#cosmos/base/query/v1beta1/pagination.proto) - [PageRequest](#cosmos.base.query.v1beta1.PageRequest) - [PageResponse](#cosmos.base.query.v1beta1.PageResponse) - + - [cosmos/auth/v1beta1/query.proto](#cosmos/auth/v1beta1/query.proto) - [AddressBytesToStringRequest](#cosmos.auth.v1beta1.AddressBytesToStringRequest) - [AddressBytesToStringResponse](#cosmos.auth.v1beta1.AddressBytesToStringResponse) @@ -31,29 +33,29 @@ - [QueryModuleAccountsResponse](#cosmos.auth.v1beta1.QueryModuleAccountsResponse) - [QueryParamsRequest](#cosmos.auth.v1beta1.QueryParamsRequest) - [QueryParamsResponse](#cosmos.auth.v1beta1.QueryParamsResponse) - + - [Query](#cosmos.auth.v1beta1.Query) - + - [cosmos/authz/v1beta1/authz.proto](#cosmos/authz/v1beta1/authz.proto) - [GenericAuthorization](#cosmos.authz.v1beta1.GenericAuthorization) - [Grant](#cosmos.authz.v1beta1.Grant) - + - [cosmos/authz/v1beta1/event.proto](#cosmos/authz/v1beta1/event.proto) - [EventGrant](#cosmos.authz.v1beta1.EventGrant) - [EventRevoke](#cosmos.authz.v1beta1.EventRevoke) - + - [cosmos/authz/v1beta1/genesis.proto](#cosmos/authz/v1beta1/genesis.proto) - [GenesisState](#cosmos.authz.v1beta1.GenesisState) - [GrantAuthorization](#cosmos.authz.v1beta1.GrantAuthorization) - + - [cosmos/authz/v1beta1/query.proto](#cosmos/authz/v1beta1/query.proto) - [QueryGranterGrantsRequest](#cosmos.authz.v1beta1.QueryGranterGrantsRequest) - [QueryGranterGrantsResponse](#cosmos.authz.v1beta1.QueryGranterGrantsResponse) - [QueryGrantsRequest](#cosmos.authz.v1beta1.QueryGrantsRequest) - [QueryGrantsResponse](#cosmos.authz.v1beta1.QueryGrantsResponse) - + - [Query](#cosmos.authz.v1beta1.Query) - + - [cosmos/authz/v1beta1/tx.proto](#cosmos/authz/v1beta1/tx.proto) - [MsgExec](#cosmos.authz.v1beta1.MsgExec) - [MsgExecResponse](#cosmos.authz.v1beta1.MsgExecResponse) @@ -61,18 +63,18 @@ - [MsgGrantResponse](#cosmos.authz.v1beta1.MsgGrantResponse) - [MsgRevoke](#cosmos.authz.v1beta1.MsgRevoke) - [MsgRevokeResponse](#cosmos.authz.v1beta1.MsgRevokeResponse) - + - [Msg](#cosmos.authz.v1beta1.Msg) - + - [cosmos/base/v1beta1/coin.proto](#cosmos/base/v1beta1/coin.proto) - [Coin](#cosmos.base.v1beta1.Coin) - [DecCoin](#cosmos.base.v1beta1.DecCoin) - [DecProto](#cosmos.base.v1beta1.DecProto) - [IntProto](#cosmos.base.v1beta1.IntProto) - + - [cosmos/bank/v1beta1/authz.proto](#cosmos/bank/v1beta1/authz.proto) - [SendAuthorization](#cosmos.bank.v1beta1.SendAuthorization) - + - [cosmos/bank/v1beta1/bank.proto](#cosmos/bank/v1beta1/bank.proto) - [DenomUnit](#cosmos.bank.v1beta1.DenomUnit) - [Input](#cosmos.bank.v1beta1.Input) @@ -81,11 +83,11 @@ - [Params](#cosmos.bank.v1beta1.Params) - [SendEnabled](#cosmos.bank.v1beta1.SendEnabled) - [Supply](#cosmos.bank.v1beta1.Supply) - + - [cosmos/bank/v1beta1/genesis.proto](#cosmos/bank/v1beta1/genesis.proto) - [Balance](#cosmos.bank.v1beta1.Balance) - [GenesisState](#cosmos.bank.v1beta1.GenesisState) - + - [cosmos/bank/v1beta1/query.proto](#cosmos/bank/v1beta1/query.proto) - [DenomOwner](#cosmos.bank.v1beta1.DenomOwner) - [QueryAllBalancesRequest](#cosmos.bank.v1beta1.QueryAllBalancesRequest) @@ -104,17 +106,17 @@ - [QuerySupplyOfResponse](#cosmos.bank.v1beta1.QuerySupplyOfResponse) - [QueryTotalSupplyRequest](#cosmos.bank.v1beta1.QueryTotalSupplyRequest) - [QueryTotalSupplyResponse](#cosmos.bank.v1beta1.QueryTotalSupplyResponse) - + - [Query](#cosmos.bank.v1beta1.Query) - + - [cosmos/bank/v1beta1/tx.proto](#cosmos/bank/v1beta1/tx.proto) - [MsgMultiSend](#cosmos.bank.v1beta1.MsgMultiSend) - [MsgMultiSendResponse](#cosmos.bank.v1beta1.MsgMultiSendResponse) - [MsgSend](#cosmos.bank.v1beta1.MsgSend) - [MsgSendResponse](#cosmos.bank.v1beta1.MsgSendResponse) - + - [Msg](#cosmos.bank.v1beta1.Msg) - + - [cosmos/base/abci/v1beta1/abci.proto](#cosmos/base/abci/v1beta1/abci.proto) - [ABCIMessageLog](#cosmos.base.abci.v1beta1.ABCIMessageLog) - [Attribute](#cosmos.base.abci.v1beta1.Attribute) @@ -126,19 +128,19 @@ - [StringEvent](#cosmos.base.abci.v1beta1.StringEvent) - [TxMsgData](#cosmos.base.abci.v1beta1.TxMsgData) - [TxResponse](#cosmos.base.abci.v1beta1.TxResponse) - + - [cosmos/base/kv/v1beta1/kv.proto](#cosmos/base/kv/v1beta1/kv.proto) - [Pair](#cosmos.base.kv.v1beta1.Pair) - [Pairs](#cosmos.base.kv.v1beta1.Pairs) - + - [cosmos/base/reflection/v1beta1/reflection.proto](#cosmos/base/reflection/v1beta1/reflection.proto) - [ListAllInterfacesRequest](#cosmos.base.reflection.v1beta1.ListAllInterfacesRequest) - [ListAllInterfacesResponse](#cosmos.base.reflection.v1beta1.ListAllInterfacesResponse) - [ListImplementationsRequest](#cosmos.base.reflection.v1beta1.ListImplementationsRequest) - [ListImplementationsResponse](#cosmos.base.reflection.v1beta1.ListImplementationsResponse) - + - [ReflectionService](#cosmos.base.reflection.v1beta1.ReflectionService) - + - [cosmos/base/reflection/v2alpha1/reflection.proto](#cosmos/base/reflection/v2alpha1/reflection.proto) - [AppDescriptor](#cosmos.base.reflection.v2alpha1.AppDescriptor) - [AuthnDescriptor](#cosmos.base.reflection.v2alpha1.AuthnDescriptor) @@ -166,26 +168,28 @@ - [QueryServicesDescriptor](#cosmos.base.reflection.v2alpha1.QueryServicesDescriptor) - [SigningModeDescriptor](#cosmos.base.reflection.v2alpha1.SigningModeDescriptor) - [TxDescriptor](#cosmos.base.reflection.v2alpha1.TxDescriptor) - + - [ReflectionService](#cosmos.base.reflection.v2alpha1.ReflectionService) - + - [cosmos/base/snapshots/v1beta1/snapshot.proto](#cosmos/base/snapshots/v1beta1/snapshot.proto) - [Metadata](#cosmos.base.snapshots.v1beta1.Metadata) - [Snapshot](#cosmos.base.snapshots.v1beta1.Snapshot) - + - [cosmos/base/store/v1beta1/commit_info.proto](#cosmos/base/store/v1beta1/commit_info.proto) - [CommitID](#cosmos.base.store.v1beta1.CommitID) - [CommitInfo](#cosmos.base.store.v1beta1.CommitInfo) - [StoreInfo](#cosmos.base.store.v1beta1.StoreInfo) - + - [cosmos/base/store/v1beta1/listening.proto](#cosmos/base/store/v1beta1/listening.proto) - [StoreKVPair](#cosmos.base.store.v1beta1.StoreKVPair) - + - [cosmos/base/store/v1beta1/snapshot.proto](#cosmos/base/store/v1beta1/snapshot.proto) - [SnapshotIAVLItem](#cosmos.base.store.v1beta1.SnapshotIAVLItem) - [SnapshotItem](#cosmos.base.store.v1beta1.SnapshotItem) + - [SnapshotKVItem](#cosmos.base.store.v1beta1.SnapshotKVItem) + - [SnapshotSchema](#cosmos.base.store.v1beta1.SnapshotSchema) - [SnapshotStoreItem](#cosmos.base.store.v1beta1.SnapshotStoreItem) - + - [cosmos/base/tendermint/v1beta1/query.proto](#cosmos/base/tendermint/v1beta1/query.proto) - [GetBlockByHeightRequest](#cosmos.base.tendermint.v1beta1.GetBlockByHeightRequest) - [GetBlockByHeightResponse](#cosmos.base.tendermint.v1beta1.GetBlockByHeightResponse) @@ -202,56 +206,56 @@ - [Module](#cosmos.base.tendermint.v1beta1.Module) - [Validator](#cosmos.base.tendermint.v1beta1.Validator) - [VersionInfo](#cosmos.base.tendermint.v1beta1.VersionInfo) - + - [Service](#cosmos.base.tendermint.v1beta1.Service) - + - [cosmos/capability/v1beta1/capability.proto](#cosmos/capability/v1beta1/capability.proto) - [Capability](#cosmos.capability.v1beta1.Capability) - [CapabilityOwners](#cosmos.capability.v1beta1.CapabilityOwners) - [Owner](#cosmos.capability.v1beta1.Owner) - + - [cosmos/capability/v1beta1/genesis.proto](#cosmos/capability/v1beta1/genesis.proto) - [GenesisOwners](#cosmos.capability.v1beta1.GenesisOwners) - [GenesisState](#cosmos.capability.v1beta1.GenesisState) - + - [cosmos/crisis/v1beta1/genesis.proto](#cosmos/crisis/v1beta1/genesis.proto) - [GenesisState](#cosmos.crisis.v1beta1.GenesisState) - + - [cosmos/crisis/v1beta1/tx.proto](#cosmos/crisis/v1beta1/tx.proto) - [MsgVerifyInvariant](#cosmos.crisis.v1beta1.MsgVerifyInvariant) - [MsgVerifyInvariantResponse](#cosmos.crisis.v1beta1.MsgVerifyInvariantResponse) - + - [Msg](#cosmos.crisis.v1beta1.Msg) - + - [cosmos/crypto/ed25519/keys.proto](#cosmos/crypto/ed25519/keys.proto) - [PrivKey](#cosmos.crypto.ed25519.PrivKey) - [PubKey](#cosmos.crypto.ed25519.PubKey) - + - [cosmos/crypto/hd/v1/hd.proto](#cosmos/crypto/hd/v1/hd.proto) - [BIP44Params](#cosmos.crypto.hd.v1.BIP44Params) - + - [cosmos/crypto/keyring/v1/record.proto](#cosmos/crypto/keyring/v1/record.proto) - [Record](#cosmos.crypto.keyring.v1.Record) - [Record.Ledger](#cosmos.crypto.keyring.v1.Record.Ledger) - [Record.Local](#cosmos.crypto.keyring.v1.Record.Local) - [Record.Multi](#cosmos.crypto.keyring.v1.Record.Multi) - [Record.Offline](#cosmos.crypto.keyring.v1.Record.Offline) - + - [cosmos/crypto/multisig/keys.proto](#cosmos/crypto/multisig/keys.proto) - [LegacyAminoPubKey](#cosmos.crypto.multisig.LegacyAminoPubKey) - + - [cosmos/crypto/multisig/v1beta1/multisig.proto](#cosmos/crypto/multisig/v1beta1/multisig.proto) - [CompactBitArray](#cosmos.crypto.multisig.v1beta1.CompactBitArray) - [MultiSignature](#cosmos.crypto.multisig.v1beta1.MultiSignature) - + - [cosmos/crypto/secp256k1/keys.proto](#cosmos/crypto/secp256k1/keys.proto) - [PrivKey](#cosmos.crypto.secp256k1.PrivKey) - [PubKey](#cosmos.crypto.secp256k1.PubKey) - + - [cosmos/crypto/secp256r1/keys.proto](#cosmos/crypto/secp256r1/keys.proto) - [PrivKey](#cosmos.crypto.secp256r1.PrivKey) - [PubKey](#cosmos.crypto.secp256r1.PubKey) - + - [cosmos/distribution/v1beta1/distribution.proto](#cosmos/distribution/v1beta1/distribution.proto) - [CommunityPoolSpendProposal](#cosmos.distribution.v1beta1.CommunityPoolSpendProposal) - [CommunityPoolSpendProposalWithDeposit](#cosmos.distribution.v1beta1.CommunityPoolSpendProposalWithDeposit) @@ -265,7 +269,7 @@ - [ValidatorOutstandingRewards](#cosmos.distribution.v1beta1.ValidatorOutstandingRewards) - [ValidatorSlashEvent](#cosmos.distribution.v1beta1.ValidatorSlashEvent) - [ValidatorSlashEvents](#cosmos.distribution.v1beta1.ValidatorSlashEvents) - + - [cosmos/distribution/v1beta1/genesis.proto](#cosmos/distribution/v1beta1/genesis.proto) - [DelegatorStartingInfoRecord](#cosmos.distribution.v1beta1.DelegatorStartingInfoRecord) - [DelegatorWithdrawInfo](#cosmos.distribution.v1beta1.DelegatorWithdrawInfo) @@ -275,7 +279,7 @@ - [ValidatorHistoricalRewardsRecord](#cosmos.distribution.v1beta1.ValidatorHistoricalRewardsRecord) - [ValidatorOutstandingRewardsRecord](#cosmos.distribution.v1beta1.ValidatorOutstandingRewardsRecord) - [ValidatorSlashEventRecord](#cosmos.distribution.v1beta1.ValidatorSlashEventRecord) - + - [cosmos/distribution/v1beta1/query.proto](#cosmos/distribution/v1beta1/query.proto) - [QueryCommunityPoolRequest](#cosmos.distribution.v1beta1.QueryCommunityPoolRequest) - [QueryCommunityPoolResponse](#cosmos.distribution.v1beta1.QueryCommunityPoolResponse) @@ -295,9 +299,9 @@ - [QueryValidatorOutstandingRewardsResponse](#cosmos.distribution.v1beta1.QueryValidatorOutstandingRewardsResponse) - [QueryValidatorSlashesRequest](#cosmos.distribution.v1beta1.QueryValidatorSlashesRequest) - [QueryValidatorSlashesResponse](#cosmos.distribution.v1beta1.QueryValidatorSlashesResponse) - + - [Query](#cosmos.distribution.v1beta1.Query) - + - [cosmos/distribution/v1beta1/tx.proto](#cosmos/distribution/v1beta1/tx.proto) - [MsgFundCommunityPool](#cosmos.distribution.v1beta1.MsgFundCommunityPool) - [MsgFundCommunityPoolResponse](#cosmos.distribution.v1beta1.MsgFundCommunityPoolResponse) @@ -307,57 +311,57 @@ - [MsgWithdrawDelegatorRewardResponse](#cosmos.distribution.v1beta1.MsgWithdrawDelegatorRewardResponse) - [MsgWithdrawValidatorCommission](#cosmos.distribution.v1beta1.MsgWithdrawValidatorCommission) - [MsgWithdrawValidatorCommissionResponse](#cosmos.distribution.v1beta1.MsgWithdrawValidatorCommissionResponse) - + - [Msg](#cosmos.distribution.v1beta1.Msg) - + - [cosmos/evidence/v1beta1/evidence.proto](#cosmos/evidence/v1beta1/evidence.proto) - [Equivocation](#cosmos.evidence.v1beta1.Equivocation) - + - [cosmos/evidence/v1beta1/genesis.proto](#cosmos/evidence/v1beta1/genesis.proto) - [GenesisState](#cosmos.evidence.v1beta1.GenesisState) - + - [cosmos/evidence/v1beta1/query.proto](#cosmos/evidence/v1beta1/query.proto) - [QueryAllEvidenceRequest](#cosmos.evidence.v1beta1.QueryAllEvidenceRequest) - [QueryAllEvidenceResponse](#cosmos.evidence.v1beta1.QueryAllEvidenceResponse) - [QueryEvidenceRequest](#cosmos.evidence.v1beta1.QueryEvidenceRequest) - [QueryEvidenceResponse](#cosmos.evidence.v1beta1.QueryEvidenceResponse) - + - [Query](#cosmos.evidence.v1beta1.Query) - + - [cosmos/evidence/v1beta1/tx.proto](#cosmos/evidence/v1beta1/tx.proto) - [MsgSubmitEvidence](#cosmos.evidence.v1beta1.MsgSubmitEvidence) - [MsgSubmitEvidenceResponse](#cosmos.evidence.v1beta1.MsgSubmitEvidenceResponse) - + - [Msg](#cosmos.evidence.v1beta1.Msg) - + - [cosmos/feegrant/v1beta1/feegrant.proto](#cosmos/feegrant/v1beta1/feegrant.proto) - [AllowedMsgAllowance](#cosmos.feegrant.v1beta1.AllowedMsgAllowance) - [BasicAllowance](#cosmos.feegrant.v1beta1.BasicAllowance) - [Grant](#cosmos.feegrant.v1beta1.Grant) - [PeriodicAllowance](#cosmos.feegrant.v1beta1.PeriodicAllowance) - + - [cosmos/feegrant/v1beta1/genesis.proto](#cosmos/feegrant/v1beta1/genesis.proto) - [GenesisState](#cosmos.feegrant.v1beta1.GenesisState) - + - [cosmos/feegrant/v1beta1/query.proto](#cosmos/feegrant/v1beta1/query.proto) - [QueryAllowanceRequest](#cosmos.feegrant.v1beta1.QueryAllowanceRequest) - [QueryAllowanceResponse](#cosmos.feegrant.v1beta1.QueryAllowanceResponse) - [QueryAllowancesRequest](#cosmos.feegrant.v1beta1.QueryAllowancesRequest) - [QueryAllowancesResponse](#cosmos.feegrant.v1beta1.QueryAllowancesResponse) - + - [Query](#cosmos.feegrant.v1beta1.Query) - + - [cosmos/feegrant/v1beta1/tx.proto](#cosmos/feegrant/v1beta1/tx.proto) - [MsgGrantAllowance](#cosmos.feegrant.v1beta1.MsgGrantAllowance) - [MsgGrantAllowanceResponse](#cosmos.feegrant.v1beta1.MsgGrantAllowanceResponse) - [MsgRevokeAllowance](#cosmos.feegrant.v1beta1.MsgRevokeAllowance) - [MsgRevokeAllowanceResponse](#cosmos.feegrant.v1beta1.MsgRevokeAllowanceResponse) - + - [Msg](#cosmos.feegrant.v1beta1.Msg) - + - [cosmos/genutil/v1beta1/genesis.proto](#cosmos/genutil/v1beta1/genesis.proto) - [GenesisState](#cosmos.genutil.v1beta1.GenesisState) - + - [cosmos/gov/v1beta1/gov.proto](#cosmos/gov/v1beta1/gov.proto) - [Deposit](#cosmos.gov.v1beta1.Deposit) - [DepositParams](#cosmos.gov.v1beta1.DepositParams) @@ -368,13 +372,13 @@ - [Vote](#cosmos.gov.v1beta1.Vote) - [VotingParams](#cosmos.gov.v1beta1.VotingParams) - [WeightedVoteOption](#cosmos.gov.v1beta1.WeightedVoteOption) - + - [ProposalStatus](#cosmos.gov.v1beta1.ProposalStatus) - [VoteOption](#cosmos.gov.v1beta1.VoteOption) - + - [cosmos/gov/v1beta1/genesis.proto](#cosmos/gov/v1beta1/genesis.proto) - [GenesisState](#cosmos.gov.v1beta1.GenesisState) - + - [cosmos/gov/v1beta1/query.proto](#cosmos/gov/v1beta1/query.proto) - [QueryDepositRequest](#cosmos.gov.v1beta1.QueryDepositRequest) - [QueryDepositResponse](#cosmos.gov.v1beta1.QueryDepositResponse) @@ -392,9 +396,9 @@ - [QueryVoteResponse](#cosmos.gov.v1beta1.QueryVoteResponse) - [QueryVotesRequest](#cosmos.gov.v1beta1.QueryVotesRequest) - [QueryVotesResponse](#cosmos.gov.v1beta1.QueryVotesResponse) - + - [Query](#cosmos.gov.v1beta1.Query) - + - [cosmos/gov/v1beta1/tx.proto](#cosmos/gov/v1beta1/tx.proto) - [MsgDeposit](#cosmos.gov.v1beta1.MsgDeposit) - [MsgDepositResponse](#cosmos.gov.v1beta1.MsgDepositResponse) @@ -404,9 +408,9 @@ - [MsgVoteResponse](#cosmos.gov.v1beta1.MsgVoteResponse) - [MsgVoteWeighted](#cosmos.gov.v1beta1.MsgVoteWeighted) - [MsgVoteWeightedResponse](#cosmos.gov.v1beta1.MsgVoteWeightedResponse) - + - [Msg](#cosmos.gov.v1beta1.Msg) - + - [cosmos/gov/v1beta2/gov.proto](#cosmos/gov/v1beta2/gov.proto) - [Deposit](#cosmos.gov.v1beta2.Deposit) - [DepositParams](#cosmos.gov.v1beta2.DepositParams) @@ -416,13 +420,13 @@ - [Vote](#cosmos.gov.v1beta2.Vote) - [VotingParams](#cosmos.gov.v1beta2.VotingParams) - [WeightedVoteOption](#cosmos.gov.v1beta2.WeightedVoteOption) - + - [ProposalStatus](#cosmos.gov.v1beta2.ProposalStatus) - [VoteOption](#cosmos.gov.v1beta2.VoteOption) - + - [cosmos/gov/v1beta2/genesis.proto](#cosmos/gov/v1beta2/genesis.proto) - [GenesisState](#cosmos.gov.v1beta2.GenesisState) - + - [cosmos/gov/v1beta2/query.proto](#cosmos/gov/v1beta2/query.proto) - [QueryDepositRequest](#cosmos.gov.v1beta2.QueryDepositRequest) - [QueryDepositResponse](#cosmos.gov.v1beta2.QueryDepositResponse) @@ -440,9 +444,9 @@ - [QueryVoteResponse](#cosmos.gov.v1beta2.QueryVoteResponse) - [QueryVotesRequest](#cosmos.gov.v1beta2.QueryVotesRequest) - [QueryVotesResponse](#cosmos.gov.v1beta2.QueryVotesResponse) - + - [Query](#cosmos.gov.v1beta2.Query) - + - [cosmos/gov/v1beta2/tx.proto](#cosmos/gov/v1beta2/tx.proto) - [MsgDeposit](#cosmos.gov.v1beta2.MsgDeposit) - [MsgDepositResponse](#cosmos.gov.v1beta2.MsgDepositResponse) @@ -452,9 +456,9 @@ - [MsgVoteResponse](#cosmos.gov.v1beta2.MsgVoteResponse) - [MsgVoteWeighted](#cosmos.gov.v1beta2.MsgVoteWeighted) - [MsgVoteWeightedResponse](#cosmos.gov.v1beta2.MsgVoteWeightedResponse) - + - [Msg](#cosmos.gov.v1beta2.Msg) - + - [cosmos/group/v1beta1/types.proto](#cosmos/group/v1beta1/types.proto) - [GroupAccountInfo](#cosmos.group.v1beta1.GroupAccountInfo) - [GroupInfo](#cosmos.group.v1beta1.GroupInfo) @@ -465,12 +469,12 @@ - [Tally](#cosmos.group.v1beta1.Tally) - [ThresholdDecisionPolicy](#cosmos.group.v1beta1.ThresholdDecisionPolicy) - [Vote](#cosmos.group.v1beta1.Vote) - + - [Choice](#cosmos.group.v1beta1.Choice) - [Proposal.ExecutorResult](#cosmos.group.v1beta1.Proposal.ExecutorResult) - [Proposal.Result](#cosmos.group.v1beta1.Proposal.Result) - [Proposal.Status](#cosmos.group.v1beta1.Proposal.Status) - + - [cosmos/group/v1beta1/query.proto](#cosmos/group/v1beta1/query.proto) - [QueryGroupAccountInfoRequest](#cosmos.group.v1beta1.QueryGroupAccountInfoRequest) - [QueryGroupAccountInfoResponse](#cosmos.group.v1beta1.QueryGroupAccountInfoResponse) @@ -494,9 +498,9 @@ - [QueryVotesByProposalResponse](#cosmos.group.v1beta1.QueryVotesByProposalResponse) - [QueryVotesByVoterRequest](#cosmos.group.v1beta1.QueryVotesByVoterRequest) - [QueryVotesByVoterResponse](#cosmos.group.v1beta1.QueryVotesByVoterResponse) - + - [Query](#cosmos.group.v1beta1.Query) - + - [cosmos/group/v1beta1/tx.proto](#cosmos/group/v1beta1/tx.proto) - [MsgCreateGroupAccountRequest](#cosmos.group.v1beta1.MsgCreateGroupAccountRequest) - [MsgCreateGroupAccountResponse](#cosmos.group.v1beta1.MsgCreateGroupAccountResponse) @@ -520,18 +524,18 @@ - [MsgUpdateGroupMetadataResponse](#cosmos.group.v1beta1.MsgUpdateGroupMetadataResponse) - [MsgVoteRequest](#cosmos.group.v1beta1.MsgVoteRequest) - [MsgVoteResponse](#cosmos.group.v1beta1.MsgVoteResponse) - + - [Exec](#cosmos.group.v1beta1.Exec) - + - [Msg](#cosmos.group.v1beta1.Msg) - + - [cosmos/mint/v1beta1/mint.proto](#cosmos/mint/v1beta1/mint.proto) - [Minter](#cosmos.mint.v1beta1.Minter) - [Params](#cosmos.mint.v1beta1.Params) - + - [cosmos/mint/v1beta1/genesis.proto](#cosmos/mint/v1beta1/genesis.proto) - [GenesisState](#cosmos.mint.v1beta1.GenesisState) - + - [cosmos/mint/v1beta1/query.proto](#cosmos/mint/v1beta1/query.proto) - [QueryAnnualProvisionsRequest](#cosmos.mint.v1beta1.QueryAnnualProvisionsRequest) - [QueryAnnualProvisionsResponse](#cosmos.mint.v1beta1.QueryAnnualProvisionsResponse) @@ -539,22 +543,22 @@ - [QueryInflationResponse](#cosmos.mint.v1beta1.QueryInflationResponse) - [QueryParamsRequest](#cosmos.mint.v1beta1.QueryParamsRequest) - [QueryParamsResponse](#cosmos.mint.v1beta1.QueryParamsResponse) - + - [Query](#cosmos.mint.v1beta1.Query) - + - [cosmos/nft/v1beta1/event.proto](#cosmos/nft/v1beta1/event.proto) - [EventBurn](#cosmos.nft.v1beta1.EventBurn) - [EventMint](#cosmos.nft.v1beta1.EventMint) - [EventSend](#cosmos.nft.v1beta1.EventSend) - + - [cosmos/nft/v1beta1/nft.proto](#cosmos/nft/v1beta1/nft.proto) - [Class](#cosmos.nft.v1beta1.Class) - [NFT](#cosmos.nft.v1beta1.NFT) - + - [cosmos/nft/v1beta1/genesis.proto](#cosmos/nft/v1beta1/genesis.proto) - [Entry](#cosmos.nft.v1beta1.Entry) - [GenesisState](#cosmos.nft.v1beta1.GenesisState) - + - [cosmos/nft/v1beta1/query.proto](#cosmos/nft/v1beta1/query.proto) - [QueryBalanceRequest](#cosmos.nft.v1beta1.QueryBalanceRequest) - [QueryBalanceResponse](#cosmos.nft.v1beta1.QueryBalanceResponse) @@ -570,38 +574,38 @@ - [QueryOwnerResponse](#cosmos.nft.v1beta1.QueryOwnerResponse) - [QuerySupplyRequest](#cosmos.nft.v1beta1.QuerySupplyRequest) - [QuerySupplyResponse](#cosmos.nft.v1beta1.QuerySupplyResponse) - + - [Query](#cosmos.nft.v1beta1.Query) - + - [cosmos/nft/v1beta1/tx.proto](#cosmos/nft/v1beta1/tx.proto) - [MsgSend](#cosmos.nft.v1beta1.MsgSend) - [MsgSendResponse](#cosmos.nft.v1beta1.MsgSendResponse) - + - [Msg](#cosmos.nft.v1beta1.Msg) - + - [cosmos/params/v1beta1/params.proto](#cosmos/params/v1beta1/params.proto) - [ParamChange](#cosmos.params.v1beta1.ParamChange) - [ParameterChangeProposal](#cosmos.params.v1beta1.ParameterChangeProposal) - + - [cosmos/params/v1beta1/query.proto](#cosmos/params/v1beta1/query.proto) - [QueryParamsRequest](#cosmos.params.v1beta1.QueryParamsRequest) - [QueryParamsResponse](#cosmos.params.v1beta1.QueryParamsResponse) - [QuerySubspacesRequest](#cosmos.params.v1beta1.QuerySubspacesRequest) - [QuerySubspacesResponse](#cosmos.params.v1beta1.QuerySubspacesResponse) - [Subspace](#cosmos.params.v1beta1.Subspace) - + - [Query](#cosmos.params.v1beta1.Query) - + - [cosmos/slashing/v1beta1/slashing.proto](#cosmos/slashing/v1beta1/slashing.proto) - [Params](#cosmos.slashing.v1beta1.Params) - [ValidatorSigningInfo](#cosmos.slashing.v1beta1.ValidatorSigningInfo) - + - [cosmos/slashing/v1beta1/genesis.proto](#cosmos/slashing/v1beta1/genesis.proto) - [GenesisState](#cosmos.slashing.v1beta1.GenesisState) - [MissedBlock](#cosmos.slashing.v1beta1.MissedBlock) - [SigningInfo](#cosmos.slashing.v1beta1.SigningInfo) - [ValidatorMissedBlocks](#cosmos.slashing.v1beta1.ValidatorMissedBlocks) - + - [cosmos/slashing/v1beta1/query.proto](#cosmos/slashing/v1beta1/query.proto) - [QueryParamsRequest](#cosmos.slashing.v1beta1.QueryParamsRequest) - [QueryParamsResponse](#cosmos.slashing.v1beta1.QueryParamsResponse) @@ -609,21 +613,21 @@ - [QuerySigningInfoResponse](#cosmos.slashing.v1beta1.QuerySigningInfoResponse) - [QuerySigningInfosRequest](#cosmos.slashing.v1beta1.QuerySigningInfosRequest) - [QuerySigningInfosResponse](#cosmos.slashing.v1beta1.QuerySigningInfosResponse) - + - [Query](#cosmos.slashing.v1beta1.Query) - + - [cosmos/slashing/v1beta1/tx.proto](#cosmos/slashing/v1beta1/tx.proto) - [MsgUnjail](#cosmos.slashing.v1beta1.MsgUnjail) - [MsgUnjailResponse](#cosmos.slashing.v1beta1.MsgUnjailResponse) - + - [Msg](#cosmos.slashing.v1beta1.Msg) - + - [cosmos/staking/v1beta1/authz.proto](#cosmos/staking/v1beta1/authz.proto) - [StakeAuthorization](#cosmos.staking.v1beta1.StakeAuthorization) - [StakeAuthorization.Validators](#cosmos.staking.v1beta1.StakeAuthorization.Validators) - + - [AuthorizationType](#cosmos.staking.v1beta1.AuthorizationType) - + - [cosmos/staking/v1beta1/staking.proto](#cosmos/staking/v1beta1/staking.proto) - [Commission](#cosmos.staking.v1beta1.Commission) - [CommissionRates](#cosmos.staking.v1beta1.CommissionRates) @@ -645,13 +649,13 @@ - [UnbondingDelegationEntry](#cosmos.staking.v1beta1.UnbondingDelegationEntry) - [ValAddresses](#cosmos.staking.v1beta1.ValAddresses) - [Validator](#cosmos.staking.v1beta1.Validator) - + - [BondStatus](#cosmos.staking.v1beta1.BondStatus) - + - [cosmos/staking/v1beta1/genesis.proto](#cosmos/staking/v1beta1/genesis.proto) - [GenesisState](#cosmos.staking.v1beta1.GenesisState) - [LastValidatorPower](#cosmos.staking.v1beta1.LastValidatorPower) - + - [cosmos/staking/v1beta1/query.proto](#cosmos/staking/v1beta1/query.proto) - [QueryDelegationRequest](#cosmos.staking.v1beta1.QueryDelegationRequest) - [QueryDelegationResponse](#cosmos.staking.v1beta1.QueryDelegationResponse) @@ -681,9 +685,9 @@ - [QueryValidatorUnbondingDelegationsResponse](#cosmos.staking.v1beta1.QueryValidatorUnbondingDelegationsResponse) - [QueryValidatorsRequest](#cosmos.staking.v1beta1.QueryValidatorsRequest) - [QueryValidatorsResponse](#cosmos.staking.v1beta1.QueryValidatorsResponse) - + - [Query](#cosmos.staking.v1beta1.Query) - + - [cosmos/staking/v1beta1/tx.proto](#cosmos/staking/v1beta1/tx.proto) - [MsgBeginRedelegate](#cosmos.staking.v1beta1.MsgBeginRedelegate) - [MsgBeginRedelegateResponse](#cosmos.staking.v1beta1.MsgBeginRedelegateResponse) @@ -695,18 +699,18 @@ - [MsgEditValidatorResponse](#cosmos.staking.v1beta1.MsgEditValidatorResponse) - [MsgUndelegate](#cosmos.staking.v1beta1.MsgUndelegate) - [MsgUndelegateResponse](#cosmos.staking.v1beta1.MsgUndelegateResponse) - + - [Msg](#cosmos.staking.v1beta1.Msg) - + - [cosmos/tx/signing/v1beta1/signing.proto](#cosmos/tx/signing/v1beta1/signing.proto) - [SignatureDescriptor](#cosmos.tx.signing.v1beta1.SignatureDescriptor) - [SignatureDescriptor.Data](#cosmos.tx.signing.v1beta1.SignatureDescriptor.Data) - [SignatureDescriptor.Data.Multi](#cosmos.tx.signing.v1beta1.SignatureDescriptor.Data.Multi) - [SignatureDescriptor.Data.Single](#cosmos.tx.signing.v1beta1.SignatureDescriptor.Data.Single) - [SignatureDescriptors](#cosmos.tx.signing.v1beta1.SignatureDescriptors) - + - [SignMode](#cosmos.tx.signing.v1beta1.SignMode) - + - [cosmos/tx/v1beta1/tx.proto](#cosmos/tx/v1beta1/tx.proto) - [AuthInfo](#cosmos.tx.v1beta1.AuthInfo) - [AuxSignerData](#cosmos.tx.v1beta1.AuxSignerData) @@ -721,7 +725,7 @@ - [Tx](#cosmos.tx.v1beta1.Tx) - [TxBody](#cosmos.tx.v1beta1.TxBody) - [TxRaw](#cosmos.tx.v1beta1.TxRaw) - + - [cosmos/tx/v1beta1/service.proto](#cosmos/tx/v1beta1/service.proto) - [BroadcastTxRequest](#cosmos.tx.v1beta1.BroadcastTxRequest) - [BroadcastTxResponse](#cosmos.tx.v1beta1.BroadcastTxResponse) @@ -731,18 +735,18 @@ - [GetTxsEventResponse](#cosmos.tx.v1beta1.GetTxsEventResponse) - [SimulateRequest](#cosmos.tx.v1beta1.SimulateRequest) - [SimulateResponse](#cosmos.tx.v1beta1.SimulateResponse) - + - [BroadcastMode](#cosmos.tx.v1beta1.BroadcastMode) - [OrderBy](#cosmos.tx.v1beta1.OrderBy) - + - [Service](#cosmos.tx.v1beta1.Service) - + - [cosmos/upgrade/v1beta1/upgrade.proto](#cosmos/upgrade/v1beta1/upgrade.proto) - [CancelSoftwareUpgradeProposal](#cosmos.upgrade.v1beta1.CancelSoftwareUpgradeProposal) - [ModuleVersion](#cosmos.upgrade.v1beta1.ModuleVersion) - [Plan](#cosmos.upgrade.v1beta1.Plan) - [SoftwareUpgradeProposal](#cosmos.upgrade.v1beta1.SoftwareUpgradeProposal) - + - [cosmos/upgrade/v1beta1/query.proto](#cosmos/upgrade/v1beta1/query.proto) - [QueryAppliedPlanRequest](#cosmos.upgrade.v1beta1.QueryAppliedPlanRequest) - [QueryAppliedPlanResponse](#cosmos.upgrade.v1beta1.QueryAppliedPlanResponse) @@ -752,9 +756,9 @@ - [QueryModuleVersionsResponse](#cosmos.upgrade.v1beta1.QueryModuleVersionsResponse) - [QueryUpgradedConsensusStateRequest](#cosmos.upgrade.v1beta1.QueryUpgradedConsensusStateRequest) - [QueryUpgradedConsensusStateResponse](#cosmos.upgrade.v1beta1.QueryUpgradedConsensusStateResponse) - + - [Query](#cosmos.upgrade.v1beta1.Query) - + - [cosmos/vesting/v1beta1/vesting.proto](#cosmos/vesting/v1beta1/vesting.proto) - [BaseVestingAccount](#cosmos.vesting.v1beta1.BaseVestingAccount) - [ContinuousVestingAccount](#cosmos.vesting.v1beta1.ContinuousVestingAccount) @@ -762,33 +766,28 @@ - [Period](#cosmos.vesting.v1beta1.Period) - [PeriodicVestingAccount](#cosmos.vesting.v1beta1.PeriodicVestingAccount) - [PermanentLockedAccount](#cosmos.vesting.v1beta1.PermanentLockedAccount) - + - [cosmos/vesting/v1beta1/tx.proto](#cosmos/vesting/v1beta1/tx.proto) - [MsgCreatePeriodicVestingAccount](#cosmos.vesting.v1beta1.MsgCreatePeriodicVestingAccount) - [MsgCreatePeriodicVestingAccountResponse](#cosmos.vesting.v1beta1.MsgCreatePeriodicVestingAccountResponse) - [MsgCreateVestingAccount](#cosmos.vesting.v1beta1.MsgCreateVestingAccount) - [MsgCreateVestingAccountResponse](#cosmos.vesting.v1beta1.MsgCreateVestingAccountResponse) - - - [Msg](#cosmos.vesting.v1beta1.Msg) - -- [Scalar Value Types](#scalar-value-types) + - [Msg](#cosmos.vesting.v1beta1.Msg) +- [Scalar Value Types](#scalar-value-types)

Top

## cosmos/auth/v1beta1/auth.proto - - ### BaseAccount -BaseAccount defines a base account type. It contains all the necessary fields -for basic account functionality. Any custom account type should extend this -type for additional functionality (e.g. vesting). +BaseAccount defines a base account type. It contains all the necessary fields for basic account functionality. Any +custom account type should extend this type for additional functionality (e.g. vesting). | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -797,16 +796,11 @@ type for additional functionality (e.g. vesting). | `account_number` | [uint64](#uint64) | | | | `sequence` | [uint64](#uint64) | | | - - - - - ### ModuleAccount -ModuleAccount defines an account for modules that holds coins on a pool. +ModuleAccount defines an account for modules that holds coins on a pool. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -814,16 +808,11 @@ ModuleAccount defines an account for modules that holds coins on a pool. | `name` | [string](#string) | | | | `permissions` | [string](#string) | repeated | | - - - - - ### Params -Params defines the parameters for the auth module. +Params defines the parameters for the auth module. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -833,10 +822,6 @@ Params defines the parameters for the auth module. | `sig_verify_cost_ed25519` | [uint64](#uint64) | | | | `sig_verify_cost_secp256k1` | [uint64](#uint64) | | | - - - - @@ -852,23 +837,17 @@ Params defines the parameters for the auth module. ## cosmos/auth/v1beta1/genesis.proto - - ### GenesisState -GenesisState defines the auth module's genesis state. +GenesisState defines the auth module's genesis state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `params` | [Params](#cosmos.auth.v1beta1.Params) | | params defines all the paramaters of the module. | | `accounts` | [google.protobuf.Any](#google.protobuf.Any) | repeated | accounts are the accounts present at genesis. | - - - - @@ -884,19 +863,13 @@ GenesisState defines the auth module's genesis state. ## cosmos/base/query/v1beta1/pagination.proto - - ### PageRequest -PageRequest is to be embedded in gRPC request messages for efficient -pagination. Ex: - message SomeRequest { - Foo some_parameter = 1; - PageRequest pagination = 2; - } +PageRequest is to be embedded in gRPC request messages for efficient pagination. Ex: +message SomeRequest { Foo some_parameter = 1; PageRequest pagination = 2; } | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -908,32 +881,19 @@ pagination. Ex: Since: cosmos-sdk 0.43 | - - - - - ### PageResponse -PageResponse is to be embedded in gRPC response messages where the -corresponding request message has used PageRequest. - message SomeResponse { - repeated Bar results = 1; - PageResponse page = 2; - } +PageResponse is to be embedded in gRPC response messages where the corresponding request message has used PageRequest. +message SomeResponse { repeated Bar results = 1; PageResponse page = 2; } | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `next_key` | [bytes](#bytes) | | next_key is the key to be passed to PageRequest.key to query the next page most efficiently | | `total` | [uint64](#uint64) | | total is total number of results available if PageRequest.count_total was set, its value is undefined otherwise | - - - - @@ -949,207 +909,139 @@ corresponding request message has used PageRequest. ## cosmos/auth/v1beta1/query.proto - - ### AddressBytesToStringRequest -AddressBytesToStringRequest is the request type for AddressString rpc method +AddressBytesToStringRequest is the request type for AddressString rpc method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address_bytes` | [bytes](#bytes) | | | - - - - - ### AddressBytesToStringResponse -AddressBytesToStringResponse is the response type for AddressString rpc method +AddressBytesToStringResponse is the response type for AddressString rpc method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address_string` | [string](#string) | | | - - - - - ### AddressStringToBytesRequest -AddressStringToBytesRequest is the request type for AccountBytes rpc method +AddressStringToBytesRequest is the request type for AccountBytes rpc method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address_string` | [string](#string) | | | - - - - - ### AddressStringToBytesResponse -AddressStringToBytesResponse is the response type for AddressBytes rpc method +AddressStringToBytesResponse is the response type for AddressBytes rpc method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address_bytes` | [bytes](#bytes) | | | - - - - - ### Bech32PrefixRequest -Bech32PrefixRequest is the request type for Bech32Prefix rpc method - - - - +Bech32PrefixRequest is the request type for Bech32Prefix rpc method ### Bech32PrefixResponse -Bech32PrefixResponse is the response type for Bech32Prefix rpc method +Bech32PrefixResponse is the response type for Bech32Prefix rpc method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `bech32_prefix` | [string](#string) | | | - - - - - ### QueryAccountRequest -QueryAccountRequest is the request type for the Query/Account RPC method. +QueryAccountRequest is the request type for the Query/Account RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address` | [string](#string) | | address defines the address to query for. | - - - - - ### QueryAccountResponse -QueryAccountResponse is the response type for the Query/Account RPC method. +QueryAccountResponse is the response type for the Query/Account RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `account` | [google.protobuf.Any](#google.protobuf.Any) | | account defines the account of the corresponding address. | - - - - - ### QueryAccountsRequest + QueryAccountsRequest is the request type for the Query/Accounts RPC method. Since: cosmos-sdk 0.43 - | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryAccountsResponse + QueryAccountsResponse is the response type for the Query/Accounts RPC method. Since: cosmos-sdk 0.43 - | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `accounts` | [google.protobuf.Any](#google.protobuf.Any) | repeated | accounts are the existing accounts | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryModuleAccountsRequest -QueryModuleAccountsRequest is the request type for the Query/ModuleAccounts RPC method. - - - - +QueryModuleAccountsRequest is the request type for the Query/ModuleAccounts RPC method. ### QueryModuleAccountsResponse -QueryModuleAccountsResponse is the response type for the Query/ModuleAccounts RPC method. +QueryModuleAccountsResponse is the response type for the Query/ModuleAccounts RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `accounts` | [google.protobuf.Any](#google.protobuf.Any) | repeated | | - - - - - ### QueryParamsRequest + QueryParamsRequest is the request type for the Query/Params RPC method. + +### QueryParamsResponse - - - - - -### QueryParamsResponse -QueryParamsResponse is the response type for the Query/Params RPC method. - +QueryParamsResponse is the response type for the Query/Params RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `params` | [Params](#cosmos.auth.v1beta1.Params) | | params defines the parameters of the module. | - - - - @@ -1160,19 +1052,32 @@ QueryParamsResponse is the response type for the Query/Params RPC method. ### Query + Query defines the gRPC querier service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | | ----------- | ------------ | ------------- | ------------| ------- | -------- | | `Accounts` | [QueryAccountsRequest](#cosmos.auth.v1beta1.QueryAccountsRequest) | [QueryAccountsResponse](#cosmos.auth.v1beta1.QueryAccountsResponse) | Accounts returns all the existing accounts -Since: cosmos-sdk 0.43 | GET|/cosmos/auth/v1beta1/accounts| -| `Account` | [QueryAccountRequest](#cosmos.auth.v1beta1.QueryAccountRequest) | [QueryAccountResponse](#cosmos.auth.v1beta1.QueryAccountResponse) | Account returns account details based on address. | GET|/cosmos/auth/v1beta1/accounts/{address}| -| `Params` | [QueryParamsRequest](#cosmos.auth.v1beta1.QueryParamsRequest) | [QueryParamsResponse](#cosmos.auth.v1beta1.QueryParamsResponse) | Params queries all parameters. | GET|/cosmos/auth/v1beta1/params| -| `ModuleAccounts` | [QueryModuleAccountsRequest](#cosmos.auth.v1beta1.QueryModuleAccountsRequest) | [QueryModuleAccountsResponse](#cosmos.auth.v1beta1.QueryModuleAccountsResponse) | ModuleAccounts returns all the existing module accounts. | GET|/cosmos/auth/v1beta1/module_accounts| -| `Bech32Prefix` | [Bech32PrefixRequest](#cosmos.auth.v1beta1.Bech32PrefixRequest) | [Bech32PrefixResponse](#cosmos.auth.v1beta1.Bech32PrefixResponse) | Bech32 queries bech32Prefix | GET|/cosmos/auth/v1beta1/bech32| -| `AddressBytesToString` | [AddressBytesToStringRequest](#cosmos.auth.v1beta1.AddressBytesToStringRequest) | [AddressBytesToStringResponse](#cosmos.auth.v1beta1.AddressBytesToStringResponse) | AddressBytesToString converts Account Address bytes to string | GET|/cosmos/auth/v1beta1/bech32/{address_bytes}| -| `AddressStringToBytes` | [AddressStringToBytesRequest](#cosmos.auth.v1beta1.AddressStringToBytesRequest) | [AddressStringToBytesResponse](#cosmos.auth.v1beta1.AddressStringToBytesResponse) | AddressStringToBytes converts Address string to bytes | GET|/cosmos/auth/v1beta1/bech32/{address_string}| +Since: cosmos-sdk 0.43 | GET|/cosmos/auth/v1beta1/accounts| | `Account` +| [QueryAccountRequest](#cosmos.auth.v1beta1.QueryAccountRequest) +| [QueryAccountResponse](#cosmos.auth.v1beta1.QueryAccountResponse) | Account returns account details based on address. +| GET|/cosmos/auth/v1beta1/accounts/{address}| | `Params` +| [QueryParamsRequest](#cosmos.auth.v1beta1.QueryParamsRequest) +| [QueryParamsResponse](#cosmos.auth.v1beta1.QueryParamsResponse) | Params queries all parameters. | +GET|/cosmos/auth/v1beta1/params| | `ModuleAccounts` +| [QueryModuleAccountsRequest](#cosmos.auth.v1beta1.QueryModuleAccountsRequest) +| [QueryModuleAccountsResponse](#cosmos.auth.v1beta1.QueryModuleAccountsResponse) | ModuleAccounts returns all the +existing module accounts. | GET|/cosmos/auth/v1beta1/module_accounts| | `Bech32Prefix` +| [Bech32PrefixRequest](#cosmos.auth.v1beta1.Bech32PrefixRequest) +| [Bech32PrefixResponse](#cosmos.auth.v1beta1.Bech32PrefixResponse) | Bech32 queries bech32Prefix | +GET|/cosmos/auth/v1beta1/bech32| | `AddressBytesToString` +| [AddressBytesToStringRequest](#cosmos.auth.v1beta1.AddressBytesToStringRequest) +| [AddressBytesToStringResponse](#cosmos.auth.v1beta1.AddressBytesToStringResponse) | AddressBytesToString converts +Account Address bytes to string | GET|/cosmos/auth/v1beta1/bech32/{address_bytes}| | `AddressStringToBytes` +| [AddressStringToBytesRequest](#cosmos.auth.v1beta1.AddressStringToBytesRequest) +| [AddressStringToBytesResponse](#cosmos.auth.v1beta1.AddressStringToBytesResponse) | AddressStringToBytes converts +Address string to bytes | GET|/cosmos/auth/v1beta1/bech32/{address_string}| @@ -1182,41 +1087,31 @@ Since: cosmos-sdk 0.43 | GET|/cosmos/auth/v1beta1/accounts|

Top

## cosmos/authz/v1beta1/authz.proto -Since: cosmos-sdk 0.43 +Since: cosmos-sdk 0.43 ### GenericAuthorization -GenericAuthorization gives the grantee unrestricted permissions to execute -the provided method on behalf of the granter's account. +GenericAuthorization gives the grantee unrestricted permissions to execute the provided method on behalf of the +granter's account. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `msg` | [string](#string) | | Msg, identified by it's type URL, to grant unrestricted permissions to execute | - - - - - ### Grant -Grant gives permissions to execute -the provide method with expiration time. +Grant gives permissions to execute the provide method with expiration time. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `authorization` | [google.protobuf.Any](#google.protobuf.Any) | | | | `expiration` | [google.protobuf.Timestamp](#google.protobuf.Timestamp) | | | - - - - @@ -1231,14 +1126,14 @@ the provide method with expiration time.

Top

## cosmos/authz/v1beta1/event.proto -Since: cosmos-sdk 0.43 +Since: cosmos-sdk 0.43 ### EventGrant -EventGrant is emitted on Msg/Grant +EventGrant is emitted on Msg/Grant | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -1246,16 +1141,11 @@ EventGrant is emitted on Msg/Grant | `granter` | [string](#string) | | Granter account address | | `grantee` | [string](#string) | | Grantee account address | - - - - - ### EventRevoke -EventRevoke is emitted on Msg/Revoke +EventRevoke is emitted on Msg/Revoke | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -1263,10 +1153,6 @@ EventRevoke is emitted on Msg/Revoke | `granter` | [string](#string) | | Granter account address | | `grantee` | [string](#string) | | Grantee account address | - - - - @@ -1281,29 +1167,24 @@ EventRevoke is emitted on Msg/Revoke

Top

## cosmos/authz/v1beta1/genesis.proto -Since: cosmos-sdk 0.43 +Since: cosmos-sdk 0.43 ### GenesisState -GenesisState defines the authz module's genesis state. +GenesisState defines the authz module's genesis state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `authorization` | [GrantAuthorization](#cosmos.authz.v1beta1.GrantAuthorization) | repeated | | - - - - - ### GrantAuthorization -GrantAuthorization defines the GenesisState/GrantAuthorization type. +GrantAuthorization defines the GenesisState/GrantAuthorization type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -1312,10 +1193,6 @@ GrantAuthorization defines the GenesisState/GrantAuthorization type. | `authorization` | [google.protobuf.Any](#google.protobuf.Any) | | | | `expiration` | [google.protobuf.Timestamp](#google.protobuf.Timestamp) | | | - - - - @@ -1330,46 +1207,36 @@ GrantAuthorization defines the GenesisState/GrantAuthorization type.

Top

## cosmos/authz/v1beta1/query.proto -Since: cosmos-sdk 0.43 +Since: cosmos-sdk 0.43 ### QueryGranterGrantsRequest -QueryGranterGrantsRequest is the request type for the Query/GranterGrants RPC method. +QueryGranterGrantsRequest is the request type for the Query/GranterGrants RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `granter` | [string](#string) | | | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an pagination for the request. | - - - - - ### QueryGranterGrantsResponse -QueryGranterGrantsResponse is the response type for the Query/GranterGrants RPC method. +QueryGranterGrantsResponse is the response type for the Query/GranterGrants RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `grants` | [Grant](#cosmos.authz.v1beta1.Grant) | repeated | authorizations is a list of grants granted for grantee by granter. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines an pagination for the response. | - - - - - ### QueryGrantsRequest -QueryGrantsRequest is the request type for the Query/Grants RPC method. +QueryGrantsRequest is the request type for the Query/Grants RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -1378,26 +1245,17 @@ QueryGrantsRequest is the request type for the Query/Grants RPC method. | `msg_type_url` | [string](#string) | | Optional, msg_type_url, when set, will query only grants matching given msg type. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an pagination for the request. | - - - - - ### QueryGrantsResponse -QueryGrantsResponse is the response type for the Query/Authorizations RPC method. +QueryGrantsResponse is the response type for the Query/Authorizations RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `grants` | [Grant](#cosmos.authz.v1beta1.Grant) | repeated | authorizations is a list of grants granted for grantee by granter. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines an pagination for the response. | - - - - @@ -1408,6 +1266,7 @@ QueryGrantsResponse is the response type for the Query/Authorizations RPC method ### Query + Query defines the gRPC querier service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -1423,48 +1282,37 @@ Query defines the gRPC querier service.

Top

## cosmos/authz/v1beta1/tx.proto -Since: cosmos-sdk 0.43 +Since: cosmos-sdk 0.43 ### MsgExec -MsgExec attempts to execute the provided messages using -authorizations granted to the grantee. Each message should have only -one signer corresponding to the granter of the authorization. +MsgExec attempts to execute the provided messages using authorizations granted to the grantee. Each message should have +only one signer corresponding to the granter of the authorization. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `grantee` | [string](#string) | | | | `msgs` | [google.protobuf.Any](#google.protobuf.Any) | repeated | Authorization Msg requests to execute. Each msg must implement Authorization interface The x/authz will try to find a grant matching (msg.signers[0], grantee, MsgTypeURL(msg)) triple and validate it. | - - - - - ### MsgExecResponse -MsgExecResponse defines the Msg/MsgExecResponse response type. +MsgExecResponse defines the Msg/MsgExecResponse response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `results` | [bytes](#bytes) | repeated | | - - - - - ### MsgGrant -MsgGrant is a request type for Grant method. It declares authorization to the grantee -on behalf of the granter with the provided expiration time. +MsgGrant is a request type for Grant method. It declares authorization to the grantee on behalf of the granter with the +provided expiration time. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -1472,27 +1320,18 @@ on behalf of the granter with the provided expiration time. | `grantee` | [string](#string) | | | | `grant` | [Grant](#cosmos.authz.v1beta1.Grant) | | | - - - - - ### MsgGrantResponse -MsgGrantResponse defines the Msg/MsgGrant response type. - - - - +MsgGrantResponse defines the Msg/MsgGrant response type. ### MsgRevoke -MsgRevoke revokes any authorization with the provided sdk.Msg type on the -granter's account with that has been granted to the grantee. +MsgRevoke revokes any authorization with the provided sdk.Msg type on the granter's account with that has been granted +to the grantee. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -1500,14 +1339,10 @@ granter's account with that has been granted to the grantee. | `grantee` | [string](#string) | | | | `msg_type_url` | [string](#string) | | | - - - - - ### MsgRevokeResponse + MsgRevokeResponse defines the Msg/MsgRevokeResponse response type. @@ -1524,6 +1359,7 @@ MsgRevokeResponse defines the Msg/MsgRevokeResponse response type. ### Msg + Msg defines the authz Msg service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -1541,75 +1377,52 @@ Msg defines the authz Msg service. ## cosmos/base/v1beta1/coin.proto - - ### Coin -Coin defines a token with a denomination and an amount. -NOTE: The amount field is an Int which implements the custom method -signatures required by gogoproto. +Coin defines a token with a denomination and an amount. +NOTE: The amount field is an Int which implements the custom method signatures required by gogoproto. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `denom` | [string](#string) | | | | `amount` | [string](#string) | | | - - - - - ### DecCoin -DecCoin defines a token with a denomination and a decimal amount. -NOTE: The amount field is an Dec which implements the custom method -signatures required by gogoproto. +DecCoin defines a token with a denomination and a decimal amount. +NOTE: The amount field is an Dec which implements the custom method signatures required by gogoproto. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `denom` | [string](#string) | | | | `amount` | [string](#string) | | | - - - - - ### DecProto -DecProto defines a Protobuf wrapper around a Dec object. +DecProto defines a Protobuf wrapper around a Dec object. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `dec` | [string](#string) | | | - - - - - ### IntProto -IntProto defines a Protobuf wrapper around an Int object. +IntProto defines a Protobuf wrapper around an Int object. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `int` | [string](#string) | | | - - - - @@ -1625,25 +1438,18 @@ IntProto defines a Protobuf wrapper around an Int object. ## cosmos/bank/v1beta1/authz.proto - - ### SendAuthorization -SendAuthorization allows the grantee to spend up to spend_limit coins from -the granter's account. -Since: cosmos-sdk 0.43 +SendAuthorization allows the grantee to spend up to spend_limit coins from the granter's account. +Since: cosmos-sdk 0.43 | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `spend_limit` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | | - - - - @@ -1659,14 +1465,11 @@ Since: cosmos-sdk 0.43 ## cosmos/bank/v1beta1/bank.proto - - ### DenomUnit -DenomUnit represents a struct that describes a given -denomination unit of the basic token. +DenomUnit represents a struct that describes a given denomination unit of the basic token. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -1674,33 +1477,22 @@ denomination unit of the basic token. | `exponent` | [uint32](#uint32) | | exponent represents power of 10 exponent that one must raise the base_denom to in order to equal the given DenomUnit's denom 1 denom = 10^exponent base_denom (e.g. with a base_denom of uatom, one can create a DenomUnit of 'atom' with exponent = 6, thus: 1 atom = 10^6 uatom). | | `aliases` | [string](#string) | repeated | aliases is a list of string aliases for the given denom | - - - - - ### Input -Input models transaction input. +Input models transaction input. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address` | [string](#string) | | | | `coins` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | | - - - - - ### Metadata -Metadata represents a struct that describes -a basic token. +Metadata represents a struct that describes a basic token. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -1710,87 +1502,61 @@ a basic token. | `display` | [string](#string) | | display indicates the suggested denom that should be displayed in clients. | | `name` | [string](#string) | | name defines the name of the token (eg: Cosmos Atom) -Since: cosmos-sdk 0.43 | -| `symbol` | [string](#string) | | symbol is the token symbol usually shown on exchanges (eg: ATOM). This can be the same as the display. +Since: cosmos-sdk 0.43 | | `symbol` | [string](#string) | | symbol is the token symbol usually shown on exchanges (eg: +ATOM). This can be the same as the display. -Since: cosmos-sdk 0.43 | -| `uri` | [string](#string) | | URI to a document (on or off-chain) that contains additional information. Optional. +Since: cosmos-sdk 0.43 | | `uri` | [string](#string) | | URI to a document (on or off-chain) that contains additional +information. Optional. -Since: cosmos-sdk 0.45 | -| `uri_hash` | [string](#string) | | URIHash is a sha256 hash of a document pointed by URI. It's used to verify that the document didn't change. Optional. +Since: cosmos-sdk 0.45 | | `uri_hash` | [string](#string) | | URIHash is a sha256 hash of a document pointed by URI. +It's used to verify that the document didn't change. Optional. Since: cosmos-sdk 0.45 | - - - - - ### Output -Output models transaction outputs. +Output models transaction outputs. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address` | [string](#string) | | | | `coins` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | | - - - - - ### Params -Params defines the parameters for the bank module. +Params defines the parameters for the bank module. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `send_enabled` | [SendEnabled](#cosmos.bank.v1beta1.SendEnabled) | repeated | | | `default_send_enabled` | [bool](#bool) | | | - - - - - ### SendEnabled -SendEnabled maps coin denom to a send_enabled status (whether a denom is -sendable). +SendEnabled maps coin denom to a send_enabled status (whether a denom is sendable). | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `denom` | [string](#string) | | | | `enabled` | [bool](#bool) | | | - - - - - ### Supply -Supply represents a struct that passively keeps track of the total supply -amounts in the network. -This message is deprecated now that supply is indexed by denom. +Supply represents a struct that passively keeps track of the total supply amounts in the network. This message is +deprecated now that supply is indexed by denom. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `total` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | | - - - - @@ -1806,30 +1572,22 @@ This message is deprecated now that supply is indexed by denom. ## cosmos/bank/v1beta1/genesis.proto - - ### Balance -Balance defines an account address and balance pair used in the bank module's -genesis state. +Balance defines an account address and balance pair used in the bank module's genesis state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address` | [string](#string) | | address is the address of the balance holder. | | `coins` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | coins defines the different coins this balance holds. | - - - - - ### GenesisState -GenesisState defines the bank module's genesis state. +GenesisState defines the bank module's genesis state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -1838,10 +1596,6 @@ GenesisState defines the bank module's genesis state. | `supply` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | supply represents the total supply. If it is left empty, then supply will be calculated based on the provided balances. Otherwise, it will be used to validate that the sum of the balances equals this amount. | | `denom_metadata` | [Metadata](#cosmos.bank.v1beta1.Metadata) | repeated | denom_metadata defines the metadata of the differents coins. | - - - - @@ -1857,248 +1611,166 @@ GenesisState defines the bank module's genesis state. ## cosmos/bank/v1beta1/query.proto - - ### DenomOwner -DenomOwner defines structure representing an account that owns or holds a -particular denominated token. It contains the account address and account -balance of the denominated token. +DenomOwner defines structure representing an account that owns or holds a particular denominated token. It contains the +account address and account balance of the denominated token. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address` | [string](#string) | | address defines the address that owns a particular denomination. | | `balance` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | | balance is the balance of the denominated coin for an account. | - - - - - ### QueryAllBalancesRequest -QueryBalanceRequest is the request type for the Query/AllBalances RPC method. +QueryBalanceRequest is the request type for the Query/AllBalances RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address` | [string](#string) | | address is the address to query balances for. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryAllBalancesResponse -QueryAllBalancesResponse is the response type for the Query/AllBalances RPC -method. +QueryAllBalancesResponse is the response type for the Query/AllBalances RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `balances` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | balances is the balances of all the coins. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryBalanceRequest -QueryBalanceRequest is the request type for the Query/Balance RPC method. +QueryBalanceRequest is the request type for the Query/Balance RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address` | [string](#string) | | address is the address to query balances for. | | `denom` | [string](#string) | | denom is the coin denom to query balances for. | - - - - - ### QueryBalanceResponse -QueryBalanceResponse is the response type for the Query/Balance RPC method. +QueryBalanceResponse is the response type for the Query/Balance RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `balance` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | | balance is the balance of the coin. | - - - - - ### QueryDenomMetadataRequest -QueryDenomMetadataRequest is the request type for the Query/DenomMetadata RPC method. +QueryDenomMetadataRequest is the request type for the Query/DenomMetadata RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `denom` | [string](#string) | | denom is the coin denom to query the metadata for. | - - - - - ### QueryDenomMetadataResponse -QueryDenomMetadataResponse is the response type for the Query/DenomMetadata RPC -method. +QueryDenomMetadataResponse is the response type for the Query/DenomMetadata RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `metadata` | [Metadata](#cosmos.bank.v1beta1.Metadata) | | metadata describes and provides all the client information for the requested token. | - - - - - ### QueryDenomOwnersRequest -QueryDenomOwnersRequest defines the request type for the DenomOwners RPC query, -which queries for a paginated set of all account holders of a particular -denomination. +QueryDenomOwnersRequest defines the request type for the DenomOwners RPC query, which queries for a paginated set of all +account holders of a particular denomination. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `denom` | [string](#string) | | denom defines the coin denomination to query all account holders for. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryDenomOwnersResponse -QueryDenomOwnersResponse defines the RPC response of a DenomOwners RPC query. +QueryDenomOwnersResponse defines the RPC response of a DenomOwners RPC query. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `denom_owners` | [DenomOwner](#cosmos.bank.v1beta1.DenomOwner) | repeated | | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryDenomsMetadataRequest -QueryDenomsMetadataRequest is the request type for the Query/DenomsMetadata RPC method. +QueryDenomsMetadataRequest is the request type for the Query/DenomsMetadata RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryDenomsMetadataResponse -QueryDenomsMetadataResponse is the response type for the Query/DenomsMetadata RPC -method. +QueryDenomsMetadataResponse is the response type for the Query/DenomsMetadata RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `metadatas` | [Metadata](#cosmos.bank.v1beta1.Metadata) | repeated | metadata provides the client information for all the registered tokens. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryParamsRequest -QueryParamsRequest defines the request type for querying x/bank parameters. - - - - +QueryParamsRequest defines the request type for querying x/bank parameters. ### QueryParamsResponse -QueryParamsResponse defines the response type for querying x/bank parameters. +QueryParamsResponse defines the response type for querying x/bank parameters. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `params` | [Params](#cosmos.bank.v1beta1.Params) | | | - - - - - ### QuerySupplyOfRequest -QuerySupplyOfRequest is the request type for the Query/SupplyOf RPC method. +QuerySupplyOfRequest is the request type for the Query/SupplyOf RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `denom` | [string](#string) | | denom is the coin denom to query balances for. | - - - - - ### QuerySupplyOfResponse -QuerySupplyOfResponse is the response type for the Query/SupplyOf RPC method. +QuerySupplyOfResponse is the response type for the Query/SupplyOf RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `amount` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | | amount is the supply of the coin. | - - - - - ### QueryTotalSupplyRequest -QueryTotalSupplyRequest is the request type for the Query/TotalSupply RPC -method. +QueryTotalSupplyRequest is the request type for the Query/TotalSupply RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -2106,17 +1778,11 @@ method. Since: cosmos-sdk 0.43 | - - - - - ### QueryTotalSupplyResponse -QueryTotalSupplyResponse is the response type for the Query/TotalSupply RPC -method +QueryTotalSupplyResponse is the response type for the Query/TotalSupply RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -2139,6 +1805,7 @@ Since: cosmos-sdk 0.43 | ### Query + Query defines the gRPC querier service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -2161,39 +1828,28 @@ Query defines the gRPC querier service. ## cosmos/bank/v1beta1/tx.proto - - ### MsgMultiSend -MsgMultiSend represents an arbitrary multi-in, multi-out send message. +MsgMultiSend represents an arbitrary multi-in, multi-out send message. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `inputs` | [Input](#cosmos.bank.v1beta1.Input) | repeated | | | `outputs` | [Output](#cosmos.bank.v1beta1.Output) | repeated | | - - - - - ### MsgMultiSendResponse -MsgMultiSendResponse defines the Msg/MultiSend response type. - - - - +MsgMultiSendResponse defines the Msg/MultiSend response type. ### MsgSend -MsgSend represents a message to send coins from one account to another. +MsgSend represents a message to send coins from one account to another. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -2201,14 +1857,10 @@ MsgSend represents a message to send coins from one account to another. | `to_address` | [string](#string) | | | | `amount` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | | - - - - - ### MsgSendResponse + MsgSendResponse defines the Msg/Send response type. @@ -2225,6 +1877,7 @@ MsgSendResponse defines the Msg/Send response type. ### Msg + Msg defines the bank Msg service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -2241,13 +1894,11 @@ Msg defines the bank Msg service. ## cosmos/base/abci/v1beta1/abci.proto - - ### ABCIMessageLog -ABCIMessageLog defines a structure containing an indexed tx ABCI message log. +ABCIMessageLog defines a structure containing an indexed tx ABCI message log. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -2255,86 +1906,60 @@ ABCIMessageLog defines a structure containing an indexed tx ABCI message log. | `log` | [string](#string) | | | | `events` | [StringEvent](#cosmos.base.abci.v1beta1.StringEvent) | repeated | Events contains a slice of Event objects that were emitted during some execution. | - - - - - ### Attribute -Attribute defines an attribute wrapper where the key and value are -strings instead of raw bytes. +Attribute defines an attribute wrapper where the key and value are strings instead of raw bytes. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `key` | [string](#string) | | | | `value` | [string](#string) | | | - - - - - ### GasInfo -GasInfo defines tx execution gas context. +GasInfo defines tx execution gas context. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `gas_wanted` | [uint64](#uint64) | | GasWanted is the maximum units of work we allow this tx to perform. | | `gas_used` | [uint64](#uint64) | | GasUsed is the amount of gas actually consumed. | - - - - - ### MsgData -MsgData defines the data returned in a Result object during message -execution. +MsgData defines the data returned in a Result object during message execution. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `msg_type` | [string](#string) | | | | `data` | [bytes](#bytes) | | | - - - - - ### Result -Result is the union of ResponseFormat and ResponseCheckTx. +Result is the union of ResponseFormat and ResponseCheckTx. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | -| `data` | [bytes](#bytes) | | **Deprecated.** Data is any data returned from message or handler execution. It MUST be length prefixed in order to separate data from multiple message executions. Deprecated. This field is still populated, but prefer msg_response instead because it also contains the Msg response typeURL. | +| `data` | [bytes](#bytes) | | ** +Deprecated.** Data is any data returned from message or handler execution. It MUST be length prefixed in order to separate data from multiple message executions. Deprecated. This field is still populated, but prefer msg_response instead because it also contains the Msg response typeURL. | | `log` | [string](#string) | | Log contains the log information from message or handler execution. | | `events` | [tendermint.abci.Event](#tendermint.abci.Event) | repeated | Events contains a slice of Event objects that were emitted during message or handler execution. | | `msg_responses` | [google.protobuf.Any](#google.protobuf.Any) | repeated | msg_responses contains the Msg handler responses type packed in Anys. Since: cosmos-sdk 0.45 | - - - - - ### SearchTxsResult -SearchTxsResult defines a structure for querying txs pageable +SearchTxsResult defines a structure for querying txs pageable | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -2345,70 +1970,49 @@ SearchTxsResult defines a structure for querying txs pageable | `limit` | [uint64](#uint64) | | Max count txs per page | | `txs` | [TxResponse](#cosmos.base.abci.v1beta1.TxResponse) | repeated | List of txs in current page | - - - - - ### SimulationResponse -SimulationResponse defines the response generated when a transaction is -successfully simulated. +SimulationResponse defines the response generated when a transaction is successfully simulated. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `gas_info` | [GasInfo](#cosmos.base.abci.v1beta1.GasInfo) | | | | `result` | [Result](#cosmos.base.abci.v1beta1.Result) | | | - - - - - ### StringEvent -StringEvent defines en Event object wrapper where all the attributes -contain key/value pairs that are strings instead of raw bytes. +StringEvent defines en Event object wrapper where all the attributes contain key/value pairs that are strings instead of +raw bytes. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `type` | [string](#string) | | | | `attributes` | [Attribute](#cosmos.base.abci.v1beta1.Attribute) | repeated | | - - - - - ### TxMsgData -TxMsgData defines a list of MsgData. A transaction will have a MsgData object -for each message. +TxMsgData defines a list of MsgData. A transaction will have a MsgData object for each message. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | -| `data` | [MsgData](#cosmos.base.abci.v1beta1.MsgData) | repeated | **Deprecated.** data field is deprecated and not populated. | +| `data` | [MsgData](#cosmos.base.abci.v1beta1.MsgData) | repeated | ** +Deprecated.** data field is deprecated and not populated. | | `msg_responses` | [google.protobuf.Any](#google.protobuf.Any) | repeated | msg_responses contains the Msg handler responses packed into Anys. Since: cosmos-sdk 0.45 | - - - - - ### TxResponse -TxResponse defines a structure containing relevant tx data and metadata. The -tags are stringified and the log is JSON decoded. +TxResponse defines a structure containing relevant tx data and metadata. The tags are stringified and the log is JSON +decoded. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -2447,38 +2051,27 @@ Since: cosmos-sdk 0.42.11, 0.44.5, 0.45 | ## cosmos/base/kv/v1beta1/kv.proto - - ### Pair -Pair defines a key/value bytes tuple. +Pair defines a key/value bytes tuple. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `key` | [bytes](#bytes) | | | | `value` | [bytes](#bytes) | | | - - - - - ### Pairs -Pairs defines a repeated slice of Pair objects. +Pairs defines a repeated slice of Pair objects. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `pairs` | [Pair](#cosmos.base.kv.v1beta1.Pair) | repeated | | - - - - @@ -2494,64 +2087,42 @@ Pairs defines a repeated slice of Pair objects. ## cosmos/base/reflection/v1beta1/reflection.proto - - ### ListAllInterfacesRequest -ListAllInterfacesRequest is the request type of the ListAllInterfaces RPC. - - - - +ListAllInterfacesRequest is the request type of the ListAllInterfaces RPC. ### ListAllInterfacesResponse -ListAllInterfacesResponse is the response type of the ListAllInterfaces RPC. +ListAllInterfacesResponse is the response type of the ListAllInterfaces RPC. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `interface_names` | [string](#string) | repeated | interface_names is an array of all the registered interfaces. | - - - - - ### ListImplementationsRequest -ListImplementationsRequest is the request type of the ListImplementations -RPC. +ListImplementationsRequest is the request type of the ListImplementations RPC. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `interface_name` | [string](#string) | | interface_name defines the interface to query the implementations for. | - - - - - ### ListImplementationsResponse -ListImplementationsResponse is the response type of the ListImplementations -RPC. +ListImplementationsResponse is the response type of the ListImplementations RPC. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `implementation_message_names` | [string](#string) | repeated | | - - - - @@ -2562,6 +2133,7 @@ RPC. ### ReflectionService + ReflectionService defines a service for interface reflection. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -2577,14 +2149,14 @@ ReflectionService defines a service for interface reflection.

Top

## cosmos/base/reflection/v2alpha1/reflection.proto -Since: cosmos-sdk 0.43 +Since: cosmos-sdk 0.43 ### AppDescriptor -AppDescriptor describes a cosmos-sdk based application +AppDescriptor describes a cosmos-sdk based application | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -2595,244 +2167,160 @@ AppDescriptor describes a cosmos-sdk based application | `query_services` | [QueryServicesDescriptor](#cosmos.base.reflection.v2alpha1.QueryServicesDescriptor) | | query_services provides metadata information regarding the available queriable endpoints | | `tx` | [TxDescriptor](#cosmos.base.reflection.v2alpha1.TxDescriptor) | | tx provides metadata information regarding how to send transactions to the given application | - - - - - ### AuthnDescriptor -AuthnDescriptor provides information on how to sign transactions without relying -on the online RPCs GetTxMetadata and CombineUnsignedTxAndSignatures +AuthnDescriptor provides information on how to sign transactions without relying on the online RPCs GetTxMetadata and +CombineUnsignedTxAndSignatures | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `sign_modes` | [SigningModeDescriptor](#cosmos.base.reflection.v2alpha1.SigningModeDescriptor) | repeated | sign_modes defines the supported signature algorithm | - - - - - ### ChainDescriptor -ChainDescriptor describes chain information of the application +ChainDescriptor describes chain information of the application | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `id` | [string](#string) | | id is the chain id | - - - - - ### CodecDescriptor -CodecDescriptor describes the registered interfaces and provides metadata information on the types +CodecDescriptor describes the registered interfaces and provides metadata information on the types | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `interfaces` | [InterfaceDescriptor](#cosmos.base.reflection.v2alpha1.InterfaceDescriptor) | repeated | interfaces is a list of the registerted interfaces descriptors | - - - - - ### ConfigurationDescriptor -ConfigurationDescriptor contains metadata information on the sdk.Config +ConfigurationDescriptor contains metadata information on the sdk.Config | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `bech32_account_address_prefix` | [string](#string) | | bech32_account_address_prefix is the account address prefix | - - - - - ### GetAuthnDescriptorRequest -GetAuthnDescriptorRequest is the request used for the GetAuthnDescriptor RPC - - - - +GetAuthnDescriptorRequest is the request used for the GetAuthnDescriptor RPC ### GetAuthnDescriptorResponse -GetAuthnDescriptorResponse is the response returned by the GetAuthnDescriptor RPC +GetAuthnDescriptorResponse is the response returned by the GetAuthnDescriptor RPC | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `authn` | [AuthnDescriptor](#cosmos.base.reflection.v2alpha1.AuthnDescriptor) | | authn describes how to authenticate to the application when sending transactions | - - - - - ### GetChainDescriptorRequest -GetChainDescriptorRequest is the request used for the GetChainDescriptor RPC - - - - +GetChainDescriptorRequest is the request used for the GetChainDescriptor RPC ### GetChainDescriptorResponse -GetChainDescriptorResponse is the response returned by the GetChainDescriptor RPC +GetChainDescriptorResponse is the response returned by the GetChainDescriptor RPC | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `chain` | [ChainDescriptor](#cosmos.base.reflection.v2alpha1.ChainDescriptor) | | chain describes application chain information | - - - - - ### GetCodecDescriptorRequest -GetCodecDescriptorRequest is the request used for the GetCodecDescriptor RPC - - - - +GetCodecDescriptorRequest is the request used for the GetCodecDescriptor RPC ### GetCodecDescriptorResponse -GetCodecDescriptorResponse is the response returned by the GetCodecDescriptor RPC +GetCodecDescriptorResponse is the response returned by the GetCodecDescriptor RPC | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `codec` | [CodecDescriptor](#cosmos.base.reflection.v2alpha1.CodecDescriptor) | | codec describes the application codec such as registered interfaces and implementations | - - - - - ### GetConfigurationDescriptorRequest -GetConfigurationDescriptorRequest is the request used for the GetConfigurationDescriptor RPC - - - - +GetConfigurationDescriptorRequest is the request used for the GetConfigurationDescriptor RPC ### GetConfigurationDescriptorResponse -GetConfigurationDescriptorResponse is the response returned by the GetConfigurationDescriptor RPC +GetConfigurationDescriptorResponse is the response returned by the GetConfigurationDescriptor RPC | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `config` | [ConfigurationDescriptor](#cosmos.base.reflection.v2alpha1.ConfigurationDescriptor) | | config describes the application's sdk.Config | - - - - - ### GetQueryServicesDescriptorRequest -GetQueryServicesDescriptorRequest is the request used for the GetQueryServicesDescriptor RPC - - - - +GetQueryServicesDescriptorRequest is the request used for the GetQueryServicesDescriptor RPC ### GetQueryServicesDescriptorResponse -GetQueryServicesDescriptorResponse is the response returned by the GetQueryServicesDescriptor RPC +GetQueryServicesDescriptorResponse is the response returned by the GetQueryServicesDescriptor RPC | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `queries` | [QueryServicesDescriptor](#cosmos.base.reflection.v2alpha1.QueryServicesDescriptor) | | queries provides information on the available queryable services | - - - - - ### GetTxDescriptorRequest -GetTxDescriptorRequest is the request used for the GetTxDescriptor RPC - - - - +GetTxDescriptorRequest is the request used for the GetTxDescriptor RPC ### GetTxDescriptorResponse -GetTxDescriptorResponse is the response returned by the GetTxDescriptor RPC +GetTxDescriptorResponse is the response returned by the GetTxDescriptor RPC | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `tx` | [TxDescriptor](#cosmos.base.reflection.v2alpha1.TxDescriptor) | | tx provides information on msgs that can be forwarded to the application alongside the accepted transaction protobuf type | - - - - - ### InterfaceAcceptingMessageDescriptor -InterfaceAcceptingMessageDescriptor describes a protobuf message which contains -an interface represented as a google.protobuf.Any +InterfaceAcceptingMessageDescriptor describes a protobuf message which contains an interface represented as a +google.protobuf.Any | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `fullname` | [string](#string) | | fullname is the protobuf fullname of the type containing the interface | | `field_descriptor_names` | [string](#string) | repeated | field_descriptor_names is a list of the protobuf name (not fullname) of the field which contains the interface as google.protobuf.Any (the interface is the same, but it can be in multiple fields of the same proto message) | - - - - - ### InterfaceDescriptor -InterfaceDescriptor describes the implementation of an interface +InterfaceDescriptor describes the implementation of an interface | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -2840,65 +2328,44 @@ InterfaceDescriptor describes the implementation of an interface | `interface_accepting_messages` | [InterfaceAcceptingMessageDescriptor](#cosmos.base.reflection.v2alpha1.InterfaceAcceptingMessageDescriptor) | repeated | interface_accepting_messages contains information regarding the proto messages which contain the interface as google.protobuf.Any field | | `interface_implementers` | [InterfaceImplementerDescriptor](#cosmos.base.reflection.v2alpha1.InterfaceImplementerDescriptor) | repeated | interface_implementers is a list of the descriptors of the interface implementers | - - - - - ### InterfaceImplementerDescriptor -InterfaceImplementerDescriptor describes an interface implementer +InterfaceImplementerDescriptor describes an interface implementer | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `fullname` | [string](#string) | | fullname is the protobuf queryable name of the interface implementer | | `type_url` | [string](#string) | | type_url defines the type URL used when marshalling the type as any this is required so we can provide type safe google.protobuf.Any marshalling and unmarshalling, making sure that we don't accept just 'any' type in our interface fields | - - - - - ### MsgDescriptor -MsgDescriptor describes a cosmos-sdk message that can be delivered with a transaction +MsgDescriptor describes a cosmos-sdk message that can be delivered with a transaction | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `msg_type_url` | [string](#string) | | msg_type_url contains the TypeURL of a sdk.Msg. | - - - - - ### QueryMethodDescriptor -QueryMethodDescriptor describes a queryable method of a query service -no other info is provided beside method name and tendermint queryable path -because it would be redundant with the grpc reflection service +QueryMethodDescriptor describes a queryable method of a query service no other info is provided beside method name and +tendermint queryable path because it would be redundant with the grpc reflection service | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `name` | [string](#string) | | name is the protobuf name (not fullname) of the method | | `full_query_path` | [string](#string) | | full_query_path is the path that can be used to query this method via tendermint abci.Query | - - - - - ### QueryServiceDescriptor -QueryServiceDescriptor describes a cosmos-sdk queryable service +QueryServiceDescriptor describes a cosmos-sdk queryable service | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -2906,34 +2373,23 @@ QueryServiceDescriptor describes a cosmos-sdk queryable service | `is_module` | [bool](#bool) | | is_module describes if this service is actually exposed by an application's module | | `methods` | [QueryMethodDescriptor](#cosmos.base.reflection.v2alpha1.QueryMethodDescriptor) | repeated | methods provides a list of query service methods | - - - - - ### QueryServicesDescriptor -QueryServicesDescriptor contains the list of cosmos-sdk queriable services +QueryServicesDescriptor contains the list of cosmos-sdk queriable services | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `query_services` | [QueryServiceDescriptor](#cosmos.base.reflection.v2alpha1.QueryServiceDescriptor) | repeated | query_services is a list of cosmos-sdk QueryServiceDescriptor | - - - - - ### SigningModeDescriptor -SigningModeDescriptor provides information on a signing flow of the application -NOTE(fdymylja): here we could go as far as providing an entire flow on how -to sign a message given a SigningModeDescriptor, but it's better to think about -this another time +SigningModeDescriptor provides information on a signing flow of the application NOTE(fdymylja): here we could go as far +as providing an entire flow on how to sign a message given a SigningModeDescriptor, but it's better to think about this +another time | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -2941,26 +2397,17 @@ this another time | `number` | [int32](#int32) | | number is the unique int32 identifier for the sign_mode enum | | `authn_info_provider_method_fullname` | [string](#string) | | authn_info_provider_method_fullname defines the fullname of the method to call to get the metadata required to authenticate using the provided sign_modes | - - - - - ### TxDescriptor -TxDescriptor describes the accepted transaction type +TxDescriptor describes the accepted transaction type | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `fullname` | [string](#string) | | fullname is the protobuf fullname of the raw transaction type (for instance the tx.Tx type) it is not meant to support polymorphism of transaction types, it is supposed to be used by reflection clients to understand if they can handle a specific transaction type in an application. | | `msgs` | [MsgDescriptor](#cosmos.base.reflection.v2alpha1.MsgDescriptor) | repeated | msgs lists the accepted application messages (sdk.Msg) | - - - - @@ -2971,6 +2418,7 @@ TxDescriptor describes the accepted transaction type ### ReflectionService + ReflectionService defines a service for application reflection. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -2991,28 +2439,21 @@ ReflectionService defines a service for application reflection. ## cosmos/base/snapshots/v1beta1/snapshot.proto - - ### Metadata -Metadata contains SDK-specific snapshot metadata. +Metadata contains SDK-specific snapshot metadata. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `chunk_hashes` | [bytes](#bytes) | repeated | SHA-256 chunk hashes | - - - - - ### Snapshot -Snapshot contains Tendermint state sync snapshot info. +Snapshot contains Tendermint state sync snapshot info. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -3022,10 +2463,6 @@ Snapshot contains Tendermint state sync snapshot info. | `hash` | [bytes](#bytes) | | | | `metadata` | [Metadata](#cosmos.base.snapshots.v1beta1.Metadata) | | | - - - - @@ -3041,58 +2478,39 @@ Snapshot contains Tendermint state sync snapshot info. ## cosmos/base/store/v1beta1/commit_info.proto - - ### CommitID -CommitID defines the committment information when a specific store is -committed. +CommitID defines the committment information when a specific store is committed. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `version` | [int64](#int64) | | | | `hash` | [bytes](#bytes) | | | - - - - - ### CommitInfo -CommitInfo defines commit information used by the multi-store when committing -a version/height. +CommitInfo defines commit information used by the multi-store when committing a version/height. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `version` | [int64](#int64) | | | | `store_infos` | [StoreInfo](#cosmos.base.store.v1beta1.StoreInfo) | repeated | | - - - - - ### StoreInfo -StoreInfo defines store-specific commit information. It contains a reference -between a store name and the commit ID. +StoreInfo defines store-specific commit information. It contains a reference between a store name and the commit ID. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `name` | [string](#string) | | | | `commit_id` | [CommitID](#cosmos.base.store.v1beta1.CommitID) | | | - - - - @@ -3108,18 +2526,16 @@ between a store name and the commit ID. ## cosmos/base/store/v1beta1/listening.proto - - ### StoreKVPair + StoreKVPair is a KVStore KVPair used for listening to state changes (Sets and Deletes) It optionally includes the StoreKey for the originating KVStore and a Boolean flag to distinguish between Sets and Deletes Since: cosmos-sdk 0.43 - | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `store_key` | [string](#string) | | the store key for the KVStore this pair originates from | @@ -3127,10 +2543,6 @@ Since: cosmos-sdk 0.43 | `key` | [bytes](#bytes) | | | | `value` | [bytes](#bytes) | | | - - - - @@ -3146,13 +2558,11 @@ Since: cosmos-sdk 0.43 ## cosmos/base/store/v1beta1/snapshot.proto - - ### SnapshotIAVLItem -SnapshotIAVLItem is an exported IAVL node. +SnapshotIAVLItem is an exported IAVL node. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -3161,40 +2571,50 @@ SnapshotIAVLItem is an exported IAVL node. | `version` | [int64](#int64) | | | | `height` | [int32](#int32) | | | - - - - - ### SnapshotItem -SnapshotItem is an item contained in a rootmulti.Store snapshot. +SnapshotItem is an item contained in a rootmulti.Store snapshot. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `store` | [SnapshotStoreItem](#cosmos.base.store.v1beta1.SnapshotStoreItem) | | | | `iavl` | [SnapshotIAVLItem](#cosmos.base.store.v1beta1.SnapshotIAVLItem) | | | +| `kv` | [SnapshotKVItem](#cosmos.base.store.v1beta1.SnapshotKVItem) | | | +| `schema` | [SnapshotSchema](#cosmos.base.store.v1beta1.SnapshotSchema) | | | + +### SnapshotKVItem +SnapshotKVItem is an exported Key/Value Pair +| Field | Type | Label | Description | +| ----- | ---- | ----- | ----------- | +| `key` | [bytes](#bytes) | | | +| `value` | [bytes](#bytes) | | | + - - -### SnapshotStoreItem -SnapshotStoreItem contains metadata about a snapshotted store. +### SnapshotSchema +SnapshotSchema is an exported schema of store | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | -| `name` | [string](#string) | | | +| `key` | [bytes](#bytes) | | | +| `type` | [bytes](#bytes) | | | + +### SnapshotStoreItem +SnapshotStoreItem contains metadata about a snapshotted store. +| Field | Type | Label | Description | +| ----- | ---- | ----- | ----------- | +| `name` | [string](#string) | | | @@ -3211,85 +2631,59 @@ SnapshotStoreItem contains metadata about a snapshotted store. ## cosmos/base/tendermint/v1beta1/query.proto - - ### GetBlockByHeightRequest -GetBlockByHeightRequest is the request type for the Query/GetBlockByHeight RPC method. +GetBlockByHeightRequest is the request type for the Query/GetBlockByHeight RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `height` | [int64](#int64) | | | - - - - - ### GetBlockByHeightResponse -GetBlockByHeightResponse is the response type for the Query/GetBlockByHeight RPC method. +GetBlockByHeightResponse is the response type for the Query/GetBlockByHeight RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `block_id` | [tendermint.types.BlockID](#tendermint.types.BlockID) | | | | `block` | [tendermint.types.Block](#tendermint.types.Block) | | | - - - - - ### GetLatestBlockRequest -GetLatestBlockRequest is the request type for the Query/GetLatestBlock RPC method. - - - - +GetLatestBlockRequest is the request type for the Query/GetLatestBlock RPC method. ### GetLatestBlockResponse -GetLatestBlockResponse is the response type for the Query/GetLatestBlock RPC method. +GetLatestBlockResponse is the response type for the Query/GetLatestBlock RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `block_id` | [tendermint.types.BlockID](#tendermint.types.BlockID) | | | | `block` | [tendermint.types.Block](#tendermint.types.Block) | | | - - - - - ### GetLatestValidatorSetRequest -GetLatestValidatorSetRequest is the request type for the Query/GetValidatorSetByHeight RPC method. +GetLatestValidatorSetRequest is the request type for the Query/GetValidatorSetByHeight RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an pagination for the request. | - - - - - ### GetLatestValidatorSetResponse -GetLatestValidatorSetResponse is the response type for the Query/GetValidatorSetByHeight RPC method. +GetLatestValidatorSetResponse is the response type for the Query/GetValidatorSetByHeight RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -3297,83 +2691,55 @@ GetLatestValidatorSetResponse is the response type for the Query/GetValidatorSet | `validators` | [Validator](#cosmos.base.tendermint.v1beta1.Validator) | repeated | | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines an pagination for the response. | - - - - - ### GetNodeInfoRequest -GetNodeInfoRequest is the request type for the Query/GetNodeInfo RPC method. - - - - +GetNodeInfoRequest is the request type for the Query/GetNodeInfo RPC method. ### GetNodeInfoResponse -GetNodeInfoResponse is the response type for the Query/GetNodeInfo RPC method. +GetNodeInfoResponse is the response type for the Query/GetNodeInfo RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `node_info` | [tendermint.p2p.NodeInfo](#tendermint.p2p.NodeInfo) | | | | `application_version` | [VersionInfo](#cosmos.base.tendermint.v1beta1.VersionInfo) | | | - - - - - ### GetSyncingRequest -GetSyncingRequest is the request type for the Query/GetSyncing RPC method. - - - - +GetSyncingRequest is the request type for the Query/GetSyncing RPC method. ### GetSyncingResponse -GetSyncingResponse is the response type for the Query/GetSyncing RPC method. +GetSyncingResponse is the response type for the Query/GetSyncing RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `syncing` | [bool](#bool) | | | - - - - - ### GetValidatorSetByHeightRequest -GetValidatorSetByHeightRequest is the request type for the Query/GetValidatorSetByHeight RPC method. +GetValidatorSetByHeightRequest is the request type for the Query/GetValidatorSetByHeight RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `height` | [int64](#int64) | | | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an pagination for the request. | - - - - - ### GetValidatorSetByHeightResponse -GetValidatorSetByHeightResponse is the response type for the Query/GetValidatorSetByHeight RPC method. +GetValidatorSetByHeightResponse is the response type for the Query/GetValidatorSetByHeight RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -3381,16 +2747,11 @@ GetValidatorSetByHeightResponse is the response type for the Query/GetValidatorS | `validators` | [Validator](#cosmos.base.tendermint.v1beta1.Validator) | repeated | | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines an pagination for the response. | - - - - - ### Module -Module is the type for VersionInfo +Module is the type for VersionInfo | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -3398,16 +2759,11 @@ Module is the type for VersionInfo | `version` | [string](#string) | | module version | | `sum` | [string](#string) | | checksum | - - - - - ### Validator -Validator is the type for the validator-set. +Validator is the type for the validator-set. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -3416,16 +2772,11 @@ Validator is the type for the validator-set. | `voting_power` | [int64](#int64) | | | | `proposer_priority` | [int64](#int64) | | | - - - - - ### VersionInfo -VersionInfo is the type for the GetNodeInfoResponse message. +VersionInfo is the type for the GetNodeInfoResponse message. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -3438,11 +2789,7 @@ VersionInfo is the type for the GetNodeInfoResponse message. | `build_deps` | [Module](#cosmos.base.tendermint.v1beta1.Module) | repeated | | | `cosmos_sdk_version` | [string](#string) | | Since: cosmos-sdk 0.43 | - - - - - + @@ -3452,6 +2799,7 @@ VersionInfo is the type for the GetNodeInfoResponse message. ### Service + Service defines the gRPC querier service for tendermint queries. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -3472,56 +2820,38 @@ Service defines the gRPC querier service for tendermint queries. ## cosmos/capability/v1beta1/capability.proto - - ### Capability -Capability defines an implementation of an object capability. The index -provided to a Capability must be globally unique. +Capability defines an implementation of an object capability. The index provided to a Capability must be globally +unique. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `index` | [uint64](#uint64) | | | - - - - - ### CapabilityOwners -CapabilityOwners defines a set of owners of a single Capability. The set of -owners must be unique. +CapabilityOwners defines a set of owners of a single Capability. The set of owners must be unique. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `owners` | [Owner](#cosmos.capability.v1beta1.Owner) | repeated | | - - - - - ### Owner -Owner defines a single capability owner. An owner is defined by the name of -capability and the module name. +Owner defines a single capability owner. An owner is defined by the name of capability and the module name. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `module` | [string](#string) | | | | `name` | [string](#string) | | | - - - - @@ -3537,39 +2867,28 @@ capability and the module name. ## cosmos/capability/v1beta1/genesis.proto - - ### GenesisOwners -GenesisOwners defines the capability owners with their corresponding index. +GenesisOwners defines the capability owners with their corresponding index. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `index` | [uint64](#uint64) | | index is the index of the capability owner. | | `index_owners` | [CapabilityOwners](#cosmos.capability.v1beta1.CapabilityOwners) | | index_owners are the owners at the given index. | - - - - - ### GenesisState -GenesisState defines the capability module's genesis state. +GenesisState defines the capability module's genesis state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `index` | [uint64](#uint64) | | index is the capability global index. | | `owners` | [GenesisOwners](#cosmos.capability.v1beta1.GenesisOwners) | repeated | owners represents a map from index to owners of the capability index index key is string to allow amino marshalling. | - - - - @@ -3585,22 +2904,16 @@ GenesisState defines the capability module's genesis state. ## cosmos/crisis/v1beta1/genesis.proto - - ### GenesisState -GenesisState defines the crisis module's genesis state. +GenesisState defines the crisis module's genesis state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `constant_fee` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | | constant_fee is the fee used to verify the invariant in the crisis module. | - - - - @@ -3616,13 +2929,11 @@ GenesisState defines the crisis module's genesis state. ## cosmos/crisis/v1beta1/tx.proto - - ### MsgVerifyInvariant -MsgVerifyInvariant represents a message to verify a particular invariance. +MsgVerifyInvariant represents a message to verify a particular invariance. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -3630,14 +2941,10 @@ MsgVerifyInvariant represents a message to verify a particular invariance. | `invariant_module_name` | [string](#string) | | | | `invariant_route` | [string](#string) | | | - - - - - ### MsgVerifyInvariantResponse + MsgVerifyInvariantResponse defines the Msg/VerifyInvariant response type. @@ -3654,6 +2961,7 @@ MsgVerifyInvariantResponse defines the Msg/VerifyInvariant response type. ### Msg + Msg defines the bank Msg service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -3669,42 +2977,30 @@ Msg defines the bank Msg service. ## cosmos/crypto/ed25519/keys.proto - - ### PrivKey -Deprecated: PrivKey defines a ed25519 private key. -NOTE: ed25519 keys must not be used in SDK apps except in a tendermint validator context. +Deprecated: PrivKey defines a ed25519 private key. NOTE: ed25519 keys must not be used in SDK apps except in a +tendermint validator context. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `key` | [bytes](#bytes) | | | - - - - - ### PubKey -PubKey is an ed25519 public key for handling Tendermint keys in SDK. -It's needed for Any serialization and SDK compatibility. -It must not be used in a non Tendermint key context because it doesn't implement -ADR-28. Nevertheless, you will like to use ed25519 in app user level -then you must create a new proto message and follow ADR-28 for Address construction. +PubKey is an ed25519 public key for handling Tendermint keys in SDK. It's needed for Any serialization and SDK +compatibility. It must not be used in a non Tendermint key context because it doesn't implement ADR-28. Nevertheless, +you will like to use ed25519 in app user level then you must create a new proto message and follow ADR-28 for Address +construction. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `key` | [bytes](#bytes) | | | - - - - @@ -3720,13 +3016,11 @@ then you must create a new proto message and follow ADR-28 for Address construct ## cosmos/crypto/hd/v1/hd.proto - - ### BIP44Params -BIP44Params is used as path field in ledger item in Record. +BIP44Params is used as path field in ledger item in Record. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -3736,10 +3030,6 @@ BIP44Params is used as path field in ledger item in Record. | `change` | [bool](#bool) | | change is a constant used for public derivation. Constant 0 is used for external chain and constant 1 for internal chain. | | `address_index` | [uint32](#uint32) | | address_index is used as child index in BIP32 derivation | - - - - @@ -3755,13 +3045,11 @@ BIP44Params is used as path field in ledger item in Record. ## cosmos/crypto/keyring/v1/record.proto - - ### Record -Record is used for representing a key in the keyring. +Record is used for representing a key in the keyring. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -3772,56 +3060,37 @@ Record is used for representing a key in the keyring. | `multi` | [Record.Multi](#cosmos.crypto.keyring.v1.Record.Multi) | | Multi does not store any information. | | `offline` | [Record.Offline](#cosmos.crypto.keyring.v1.Record.Offline) | | Offline does not store any information. | - - - - - ### Record.Ledger -Ledger item +Ledger item | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `path` | [cosmos.crypto.hd.v1.BIP44Params](#cosmos.crypto.hd.v1.BIP44Params) | | | - - - - - ### Record.Local -Item is a keyring item stored in a keyring backend. -Local item +Item is a keyring item stored in a keyring backend. Local item | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `priv_key` | [google.protobuf.Any](#google.protobuf.Any) | | | | `priv_key_type` | [string](#string) | | | - - - - - ### Record.Multi -Multi item - - - - +Multi item ### Record.Offline + Offline item @@ -3843,25 +3112,18 @@ Offline item ## cosmos/crypto/multisig/keys.proto - - ### LegacyAminoPubKey -LegacyAminoPubKey specifies a public key type -which nests multiple public keys and a threshold, -it uses legacy amino address rules. +LegacyAminoPubKey specifies a public key type which nests multiple public keys and a threshold, it uses legacy amino +address rules. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `threshold` | [uint32](#uint32) | | | | `public_keys` | [google.protobuf.Any](#google.protobuf.Any) | repeated | | - - - - @@ -3877,43 +3139,29 @@ it uses legacy amino address rules. ## cosmos/crypto/multisig/v1beta1/multisig.proto - - ### CompactBitArray -CompactBitArray is an implementation of a space efficient bit array. -This is used to ensure that the encoded data takes up a minimal amount of -space after proto encoding. -This is not thread safe, and is not intended for concurrent usage. +CompactBitArray is an implementation of a space efficient bit array. This is used to ensure that the encoded data takes +up a minimal amount of space after proto encoding. This is not thread safe, and is not intended for concurrent usage. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `extra_bits_stored` | [uint32](#uint32) | | | | `elems` | [bytes](#bytes) | | | - - - - - ### MultiSignature -MultiSignature wraps the signatures from a multisig.LegacyAminoPubKey. -See cosmos.tx.v1betata1.ModeInfo.Multi for how to specify which signers -signed and with which modes. +MultiSignature wraps the signatures from a multisig.LegacyAminoPubKey. See cosmos.tx.v1betata1.ModeInfo.Multi for how to +specify which signers signed and with which modes. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `signatures` | [bytes](#bytes) | repeated | | - - - - @@ -3929,41 +3177,28 @@ signed and with which modes. ## cosmos/crypto/secp256k1/keys.proto - - ### PrivKey -PrivKey defines a secp256k1 private key. +PrivKey defines a secp256k1 private key. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `key` | [bytes](#bytes) | | | - - - - - ### PubKey -PubKey defines a secp256k1 public key -Key is the compressed form of the pubkey. The first byte depends is a 0x02 byte -if the y-coordinate is the lexicographically largest of the two associated with -the x-coordinate. Otherwise the first byte is a 0x03. -This prefix is followed with the x-coordinate. +PubKey defines a secp256k1 public key Key is the compressed form of the pubkey. The first byte depends is a 0x02 byte if +the y-coordinate is the lexicographically largest of the two associated with the x-coordinate. Otherwise the first byte +is a 0x03. This prefix is followed with the x-coordinate. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `key` | [bytes](#bytes) | | | - - - - @@ -3978,38 +3213,29 @@ This prefix is followed with the x-coordinate.

Top

## cosmos/crypto/secp256r1/keys.proto -Since: cosmos-sdk 0.43 +Since: cosmos-sdk 0.43 ### PrivKey -PrivKey defines a secp256r1 ECDSA private key. +PrivKey defines a secp256r1 ECDSA private key. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `secret` | [bytes](#bytes) | | secret number serialized using big-endian encoding | - - - - - ### PubKey -PubKey defines a secp256r1 ECDSA public key. +PubKey defines a secp256r1 ECDSA public key. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `key` | [bytes](#bytes) | | Point on secp256r1 curve in a compressed representation as specified in section 4.3.6 of ANSI X9.62: https://webstore.ansi.org/standards/ascx9/ansix9621998 | - - - - @@ -4025,15 +3251,12 @@ PubKey defines a secp256r1 ECDSA public key. ## cosmos/distribution/v1beta1/distribution.proto - - ### CommunityPoolSpendProposal -CommunityPoolSpendProposal details a proposal for use of community funds, -together with how many coins are proposed to be spent, and to which -recipient account. +CommunityPoolSpendProposal details a proposal for use of community funds, together with how many coins are proposed to +be spent, and to which recipient account. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -4042,17 +3265,11 @@ recipient account. | `recipient` | [string](#string) | | | | `amount` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | | - - - - - ### CommunityPoolSpendProposalWithDeposit -CommunityPoolSpendProposalWithDeposit defines a CommunityPoolSpendProposal -with a deposit +CommunityPoolSpendProposalWithDeposit defines a CommunityPoolSpendProposal with a deposit | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -4062,38 +3279,25 @@ with a deposit | `amount` | [string](#string) | | | | `deposit` | [string](#string) | | | - - - - - ### DelegationDelegatorReward -DelegationDelegatorReward represents the properties -of a delegator's delegation reward. +DelegationDelegatorReward represents the properties of a delegator's delegation reward. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validator_address` | [string](#string) | | | | `reward` | [cosmos.base.v1beta1.DecCoin](#cosmos.base.v1beta1.DecCoin) | repeated | | - - - - - ### DelegatorStartingInfo -DelegatorStartingInfo represents the starting info for a delegator reward -period. It tracks the previous validator period, the delegation's amount of -staking token, and the creation height (to check later on if any slashes have -occurred). NOTE: Even though validators are slashed to whole staking tokens, -the delegators within the validator may be left with less than a full token, -thus sdk.Dec is used. +DelegatorStartingInfo represents the starting info for a delegator reward period. It tracks the previous validator +period, the delegation's amount of staking token, and the creation height (to check later on if any slashes have +occurred). NOTE: Even though validators are slashed to whole staking tokens, the delegators within the validator may be +left with less than a full token, thus sdk.Dec is used. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -4101,31 +3305,21 @@ thus sdk.Dec is used. | `stake` | [string](#string) | | | | `height` | [uint64](#uint64) | | | - - - - - ### FeePool -FeePool is the global fee pool for distribution. +FeePool is the global fee pool for distribution. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `community_pool` | [cosmos.base.v1beta1.DecCoin](#cosmos.base.v1beta1.DecCoin) | repeated | | - - - - - ### Params -Params defines the set of params for the distribution module. +Params defines the set of params for the distribution module. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -4134,121 +3328,80 @@ Params defines the set of params for the distribution module. | `bonus_proposer_reward` | [string](#string) | | | | `withdraw_addr_enabled` | [bool](#bool) | | | - - - - - ### ValidatorAccumulatedCommission -ValidatorAccumulatedCommission represents accumulated commission -for a validator kept as a running counter, can be withdrawn at any time. +ValidatorAccumulatedCommission represents accumulated commission for a validator kept as a running counter, can be +withdrawn at any time. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `commission` | [cosmos.base.v1beta1.DecCoin](#cosmos.base.v1beta1.DecCoin) | repeated | | - - - - - ### ValidatorCurrentRewards -ValidatorCurrentRewards represents current rewards and current -period for a validator kept as a running counter and incremented -each block as long as the validator's tokens remain constant. +ValidatorCurrentRewards represents current rewards and current period for a validator kept as a running counter and +incremented each block as long as the validator's tokens remain constant. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `rewards` | [cosmos.base.v1beta1.DecCoin](#cosmos.base.v1beta1.DecCoin) | repeated | | | `period` | [uint64](#uint64) | | | - - - - - ### ValidatorHistoricalRewards -ValidatorHistoricalRewards represents historical rewards for a validator. -Height is implicit within the store key. -Cumulative reward ratio is the sum from the zeroeth period -until this period of rewards / tokens, per the spec. -The reference count indicates the number of objects -which might need to reference this historical entry at any point. -ReferenceCount = - number of outstanding delegations which ended the associated period (and - might need to read that record) - + number of slashes which ended the associated period (and might need to - read that record) - + one per validator for the zeroeth period, set on initialization +ValidatorHistoricalRewards represents historical rewards for a validator. Height is implicit within the store key. +Cumulative reward ratio is the sum from the zeroeth period until this period of rewards / tokens, per the spec. The +reference count indicates the number of objects which might need to reference this historical entry at any point. +ReferenceCount = number of outstanding delegations which ended the associated period (and might need to read that +record) + ++ number of slashes which ended the associated period (and might need to read that record) ++ one per validator for the zeroeth period, set on initialization | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `cumulative_reward_ratio` | [cosmos.base.v1beta1.DecCoin](#cosmos.base.v1beta1.DecCoin) | repeated | | | `reference_count` | [uint32](#uint32) | | | - - - - - ### ValidatorOutstandingRewards -ValidatorOutstandingRewards represents outstanding (un-withdrawn) rewards -for a validator inexpensive to track, allows simple sanity checks. +ValidatorOutstandingRewards represents outstanding (un-withdrawn) rewards for a validator inexpensive to track, allows +simple sanity checks. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `rewards` | [cosmos.base.v1beta1.DecCoin](#cosmos.base.v1beta1.DecCoin) | repeated | | - - - - - ### ValidatorSlashEvent -ValidatorSlashEvent represents a validator slash event. -Height is implicit within the store key. -This is needed to calculate appropriate amount of staking tokens -for delegations which are withdrawn after a slash has occurred. +ValidatorSlashEvent represents a validator slash event. Height is implicit within the store key. This is needed to +calculate appropriate amount of staking tokens for delegations which are withdrawn after a slash has occurred. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validator_period` | [uint64](#uint64) | | | | `fraction` | [string](#string) | | | - - - - - ### ValidatorSlashEvents -ValidatorSlashEvents is a collection of ValidatorSlashEvent messages. +ValidatorSlashEvents is a collection of ValidatorSlashEvent messages. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validator_slash_events` | [ValidatorSlashEvent](#cosmos.distribution.v1beta1.ValidatorSlashEvent) | repeated | | - - - - @@ -4264,13 +3417,11 @@ ValidatorSlashEvents is a collection of ValidatorSlashEvent messages. ## cosmos/distribution/v1beta1/genesis.proto - - ### DelegatorStartingInfoRecord -DelegatorStartingInfoRecord used for import / export via genesis json. +DelegatorStartingInfoRecord used for import / export via genesis json. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -4278,34 +3429,23 @@ DelegatorStartingInfoRecord used for import / export via genesis json. | `validator_address` | [string](#string) | | validator_address is the address of the validator. | | `starting_info` | [DelegatorStartingInfo](#cosmos.distribution.v1beta1.DelegatorStartingInfo) | | starting_info defines the starting info of a delegator. | - - - - - ### DelegatorWithdrawInfo -DelegatorWithdrawInfo is the address for where distributions rewards are -withdrawn to by default this struct is only used at genesis to feed in -default withdraw addresses. +DelegatorWithdrawInfo is the address for where distributions rewards are withdrawn to by default this struct is only +used at genesis to feed in default withdraw addresses. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegator_address` | [string](#string) | | delegator_address is the address of the delegator. | | `withdraw_address` | [string](#string) | | withdraw_address is the address to withdraw the delegation rewards to. | - - - - - ### GenesisState -GenesisState defines the distribution module's genesis state. +GenesisState defines the distribution module's genesis state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -4320,50 +3460,33 @@ GenesisState defines the distribution module's genesis state. | `delegator_starting_infos` | [DelegatorStartingInfoRecord](#cosmos.distribution.v1beta1.DelegatorStartingInfoRecord) | repeated | fee_pool defines the delegator starting infos at genesis. | | `validator_slash_events` | [ValidatorSlashEventRecord](#cosmos.distribution.v1beta1.ValidatorSlashEventRecord) | repeated | fee_pool defines the validator slash events at genesis. | - - - - - ### ValidatorAccumulatedCommissionRecord -ValidatorAccumulatedCommissionRecord is used for import / export via genesis -json. +ValidatorAccumulatedCommissionRecord is used for import / export via genesis json. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validator_address` | [string](#string) | | validator_address is the address of the validator. | | `accumulated` | [ValidatorAccumulatedCommission](#cosmos.distribution.v1beta1.ValidatorAccumulatedCommission) | | accumulated is the accumulated commission of a validator. | - - - - - ### ValidatorCurrentRewardsRecord -ValidatorCurrentRewardsRecord is used for import / export via genesis json. +ValidatorCurrentRewardsRecord is used for import / export via genesis json. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validator_address` | [string](#string) | | validator_address is the address of the validator. | | `rewards` | [ValidatorCurrentRewards](#cosmos.distribution.v1beta1.ValidatorCurrentRewards) | | rewards defines the current rewards of a validator. | - - - - - ### ValidatorHistoricalRewardsRecord -ValidatorHistoricalRewardsRecord is used for import / export via genesis -json. +ValidatorHistoricalRewardsRecord is used for import / export via genesis json. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -4371,32 +3494,22 @@ json. | `period` | [uint64](#uint64) | | period defines the period the historical rewards apply to. | | `rewards` | [ValidatorHistoricalRewards](#cosmos.distribution.v1beta1.ValidatorHistoricalRewards) | | rewards defines the historical rewards of a validator. | - - - - - ### ValidatorOutstandingRewardsRecord -ValidatorOutstandingRewardsRecord is used for import/export via genesis json. +ValidatorOutstandingRewardsRecord is used for import/export via genesis json. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validator_address` | [string](#string) | | validator_address is the address of the validator. | | `outstanding_rewards` | [cosmos.base.v1beta1.DecCoin](#cosmos.base.v1beta1.DecCoin) | repeated | outstanding_rewards represents the oustanding rewards of a validator. | - - - - - ### ValidatorSlashEventRecord -ValidatorSlashEventRecord is used for import / export via genesis json. +ValidatorSlashEventRecord is used for import / export via genesis json. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -4405,10 +3518,6 @@ ValidatorSlashEventRecord is used for import / export via genesis json. | `period` | [uint64](#uint64) | | period is the period of the slash event. | | `validator_slash_event` | [ValidatorSlashEvent](#cosmos.distribution.v1beta1.ValidatorSlashEvent) | | validator_slash_event describes the slash event. | - - - - @@ -4424,260 +3533,165 @@ ValidatorSlashEventRecord is used for import / export via genesis json. ## cosmos/distribution/v1beta1/query.proto - - ### QueryCommunityPoolRequest -QueryCommunityPoolRequest is the request type for the Query/CommunityPool RPC -method. - - - - +QueryCommunityPoolRequest is the request type for the Query/CommunityPool RPC method. ### QueryCommunityPoolResponse -QueryCommunityPoolResponse is the response type for the Query/CommunityPool -RPC method. +QueryCommunityPoolResponse is the response type for the Query/CommunityPool RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `pool` | [cosmos.base.v1beta1.DecCoin](#cosmos.base.v1beta1.DecCoin) | repeated | pool defines community pool's coins. | - - - - - ### QueryDelegationRewardsRequest -QueryDelegationRewardsRequest is the request type for the -Query/DelegationRewards RPC method. +QueryDelegationRewardsRequest is the request type for the Query/DelegationRewards RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegator_address` | [string](#string) | | delegator_address defines the delegator address to query for. | | `validator_address` | [string](#string) | | validator_address defines the validator address to query for. | - - - - - ### QueryDelegationRewardsResponse -QueryDelegationRewardsResponse is the response type for the -Query/DelegationRewards RPC method. +QueryDelegationRewardsResponse is the response type for the Query/DelegationRewards RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `rewards` | [cosmos.base.v1beta1.DecCoin](#cosmos.base.v1beta1.DecCoin) | repeated | rewards defines the rewards accrued by a delegation. | - - - - - ### QueryDelegationTotalRewardsRequest -QueryDelegationTotalRewardsRequest is the request type for the -Query/DelegationTotalRewards RPC method. +QueryDelegationTotalRewardsRequest is the request type for the Query/DelegationTotalRewards RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegator_address` | [string](#string) | | delegator_address defines the delegator address to query for. | - - - - - ### QueryDelegationTotalRewardsResponse -QueryDelegationTotalRewardsResponse is the response type for the -Query/DelegationTotalRewards RPC method. +QueryDelegationTotalRewardsResponse is the response type for the Query/DelegationTotalRewards RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `rewards` | [DelegationDelegatorReward](#cosmos.distribution.v1beta1.DelegationDelegatorReward) | repeated | rewards defines all the rewards accrued by a delegator. | | `total` | [cosmos.base.v1beta1.DecCoin](#cosmos.base.v1beta1.DecCoin) | repeated | total defines the sum of all the rewards. | - - - - - ### QueryDelegatorValidatorsRequest -QueryDelegatorValidatorsRequest is the request type for the -Query/DelegatorValidators RPC method. +QueryDelegatorValidatorsRequest is the request type for the Query/DelegatorValidators RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegator_address` | [string](#string) | | delegator_address defines the delegator address to query for. | - - - - - ### QueryDelegatorValidatorsResponse -QueryDelegatorValidatorsResponse is the response type for the -Query/DelegatorValidators RPC method. +QueryDelegatorValidatorsResponse is the response type for the Query/DelegatorValidators RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validators` | [string](#string) | repeated | validators defines the validators a delegator is delegating for. | - - - - - ### QueryDelegatorWithdrawAddressRequest -QueryDelegatorWithdrawAddressRequest is the request type for the -Query/DelegatorWithdrawAddress RPC method. +QueryDelegatorWithdrawAddressRequest is the request type for the Query/DelegatorWithdrawAddress RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegator_address` | [string](#string) | | delegator_address defines the delegator address to query for. | - - - - - ### QueryDelegatorWithdrawAddressResponse -QueryDelegatorWithdrawAddressResponse is the response type for the -Query/DelegatorWithdrawAddress RPC method. +QueryDelegatorWithdrawAddressResponse is the response type for the Query/DelegatorWithdrawAddress RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `withdraw_address` | [string](#string) | | withdraw_address defines the delegator address to query for. | - - - - - ### QueryParamsRequest -QueryParamsRequest is the request type for the Query/Params RPC method. - - - - +QueryParamsRequest is the request type for the Query/Params RPC method. ### QueryParamsResponse -QueryParamsResponse is the response type for the Query/Params RPC method. +QueryParamsResponse is the response type for the Query/Params RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `params` | [Params](#cosmos.distribution.v1beta1.Params) | | params defines the parameters of the module. | - - - - - ### QueryValidatorCommissionRequest -QueryValidatorCommissionRequest is the request type for the -Query/ValidatorCommission RPC method +QueryValidatorCommissionRequest is the request type for the Query/ValidatorCommission RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validator_address` | [string](#string) | | validator_address defines the validator address to query for. | - - - - - ### QueryValidatorCommissionResponse -QueryValidatorCommissionResponse is the response type for the -Query/ValidatorCommission RPC method +QueryValidatorCommissionResponse is the response type for the Query/ValidatorCommission RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `commission` | [ValidatorAccumulatedCommission](#cosmos.distribution.v1beta1.ValidatorAccumulatedCommission) | | commission defines the commision the validator received. | - - - - - ### QueryValidatorOutstandingRewardsRequest -QueryValidatorOutstandingRewardsRequest is the request type for the -Query/ValidatorOutstandingRewards RPC method. +QueryValidatorOutstandingRewardsRequest is the request type for the Query/ValidatorOutstandingRewards RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validator_address` | [string](#string) | | validator_address defines the validator address to query for. | - - - - - ### QueryValidatorOutstandingRewardsResponse -QueryValidatorOutstandingRewardsResponse is the response type for the -Query/ValidatorOutstandingRewards RPC method. +QueryValidatorOutstandingRewardsResponse is the response type for the Query/ValidatorOutstandingRewards RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `rewards` | [ValidatorOutstandingRewards](#cosmos.distribution.v1beta1.ValidatorOutstandingRewards) | | | - - - - - ### QueryValidatorSlashesRequest -QueryValidatorSlashesRequest is the request type for the -Query/ValidatorSlashes RPC method +QueryValidatorSlashesRequest is the request type for the Query/ValidatorSlashes RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -4686,27 +3700,17 @@ Query/ValidatorSlashes RPC method | `ending_height` | [uint64](#uint64) | | starting_height defines the optional ending height to query the slashes. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryValidatorSlashesResponse -QueryValidatorSlashesResponse is the response type for the -Query/ValidatorSlashes RPC method. +QueryValidatorSlashesResponse is the response type for the Query/ValidatorSlashes RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `slashes` | [ValidatorSlashEvent](#cosmos.distribution.v1beta1.ValidatorSlashEvent) | repeated | slashes defines the slashes the validator received. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - @@ -4717,6 +3721,7 @@ Query/ValidatorSlashes RPC method. ### Query + Query defines the gRPC querier service for distribution module. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -4740,108 +3745,71 @@ Query defines the gRPC querier service for distribution module. ## cosmos/distribution/v1beta1/tx.proto - - ### MsgFundCommunityPool -MsgFundCommunityPool allows an account to directly -fund the community pool. +MsgFundCommunityPool allows an account to directly fund the community pool. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `amount` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | | | `depositor` | [string](#string) | | | - - - - - ### MsgFundCommunityPoolResponse -MsgFundCommunityPoolResponse defines the Msg/FundCommunityPool response type. - - - - +MsgFundCommunityPoolResponse defines the Msg/FundCommunityPool response type. ### MsgSetWithdrawAddress -MsgSetWithdrawAddress sets the withdraw address for -a delegator (or validator self-delegation). +MsgSetWithdrawAddress sets the withdraw address for a delegator (or validator self-delegation). | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegator_address` | [string](#string) | | | | `withdraw_address` | [string](#string) | | | - - - - - ### MsgSetWithdrawAddressResponse -MsgSetWithdrawAddressResponse defines the Msg/SetWithdrawAddress response type. - - - - +MsgSetWithdrawAddressResponse defines the Msg/SetWithdrawAddress response type. ### MsgWithdrawDelegatorReward -MsgWithdrawDelegatorReward represents delegation withdrawal to a delegator -from a single validator. +MsgWithdrawDelegatorReward represents delegation withdrawal to a delegator from a single validator. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegator_address` | [string](#string) | | | | `validator_address` | [string](#string) | | | - - - - - ### MsgWithdrawDelegatorRewardResponse -MsgWithdrawDelegatorRewardResponse defines the Msg/WithdrawDelegatorReward response type. - - - - +MsgWithdrawDelegatorRewardResponse defines the Msg/WithdrawDelegatorReward response type. ### MsgWithdrawValidatorCommission -MsgWithdrawValidatorCommission withdraws the full commission to the validator -address. +MsgWithdrawValidatorCommission withdraws the full commission to the validator address. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validator_address` | [string](#string) | | | - - - - - ### MsgWithdrawValidatorCommissionResponse + MsgWithdrawValidatorCommissionResponse defines the Msg/WithdrawValidatorCommission response type. @@ -4858,6 +3826,7 @@ MsgWithdrawValidatorCommissionResponse defines the Msg/WithdrawValidatorCommissi ### Msg + Msg defines the distribution Msg service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -4876,14 +3845,11 @@ Msg defines the distribution Msg service. ## cosmos/evidence/v1beta1/evidence.proto - - ### Equivocation -Equivocation implements the Evidence interface and defines evidence of double -signing misbehavior. +Equivocation implements the Evidence interface and defines evidence of double signing misbehavior. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -4892,10 +3858,6 @@ signing misbehavior. | `power` | [int64](#int64) | | | | `consensus_address` | [string](#string) | | | - - - - @@ -4911,22 +3873,16 @@ signing misbehavior. ## cosmos/evidence/v1beta1/genesis.proto - - ### GenesisState -GenesisState defines the evidence module's genesis state. +GenesisState defines the evidence module's genesis state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `evidence` | [google.protobuf.Any](#google.protobuf.Any) | repeated | evidence defines all the evidence at genesis. | - - - - @@ -4942,70 +3898,47 @@ GenesisState defines the evidence module's genesis state. ## cosmos/evidence/v1beta1/query.proto - - ### QueryAllEvidenceRequest -QueryEvidenceRequest is the request type for the Query/AllEvidence RPC -method. +QueryEvidenceRequest is the request type for the Query/AllEvidence RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryAllEvidenceResponse -QueryAllEvidenceResponse is the response type for the Query/AllEvidence RPC -method. +QueryAllEvidenceResponse is the response type for the Query/AllEvidence RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `evidence` | [google.protobuf.Any](#google.protobuf.Any) | repeated | evidence returns all evidences. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryEvidenceRequest -QueryEvidenceRequest is the request type for the Query/Evidence RPC method. +QueryEvidenceRequest is the request type for the Query/Evidence RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `evidence_hash` | [bytes](#bytes) | | evidence_hash defines the hash of the requested evidence. | - - - - - ### QueryEvidenceResponse -QueryEvidenceResponse is the response type for the Query/Evidence RPC method. +QueryEvidenceResponse is the response type for the Query/Evidence RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `evidence` | [google.protobuf.Any](#google.protobuf.Any) | | evidence returns the requested evidence. | - - - - @@ -5016,6 +3949,7 @@ QueryEvidenceResponse is the response type for the Query/Evidence RPC method. ### Query + Query defines the gRPC querier service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -5032,39 +3966,28 @@ Query defines the gRPC querier service. ## cosmos/evidence/v1beta1/tx.proto - - ### MsgSubmitEvidence -MsgSubmitEvidence represents a message that supports submitting arbitrary -Evidence of misbehavior such as equivocation or counterfactual signing. +MsgSubmitEvidence represents a message that supports submitting arbitrary Evidence of misbehavior such as equivocation +or counterfactual signing. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `submitter` | [string](#string) | | | | `evidence` | [google.protobuf.Any](#google.protobuf.Any) | | | - - - - - ### MsgSubmitEvidenceResponse -MsgSubmitEvidenceResponse defines the Msg/SubmitEvidence response type. +MsgSubmitEvidenceResponse defines the Msg/SubmitEvidence response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `hash` | [bytes](#bytes) | | hash defines the hash of the evidence. | - - - - @@ -5075,6 +3998,7 @@ MsgSubmitEvidenceResponse defines the Msg/SubmitEvidence response type. ### Msg + Msg defines the evidence Msg service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -5089,47 +4013,37 @@ Msg defines the evidence Msg service.

Top

## cosmos/feegrant/v1beta1/feegrant.proto -Since: cosmos-sdk 0.43 +Since: cosmos-sdk 0.43 ### AllowedMsgAllowance -AllowedMsgAllowance creates allowance only for specified message types. +AllowedMsgAllowance creates allowance only for specified message types. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `allowance` | [google.protobuf.Any](#google.protobuf.Any) | | allowance can be any of basic and filtered fee allowance. | | `allowed_messages` | [string](#string) | repeated | allowed_messages are the messages for which the grantee has the access. | - - - - - ### BasicAllowance -BasicAllowance implements Allowance with a one-time grant of tokens -that optionally expires. The grantee can use up to SpendLimit to cover fees. +BasicAllowance implements Allowance with a one-time grant of tokens that optionally expires. The grantee can use up to +SpendLimit to cover fees. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `spend_limit` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | spend_limit specifies the maximum amount of tokens that can be spent by this allowance and will be updated as tokens are spent. If it is empty, there is no spend limit and any amount of coins can be spent. | | `expiration` | [google.protobuf.Timestamp](#google.protobuf.Timestamp) | | expiration specifies an optional time when this allowance expires | - - - - - ### Grant -Grant is stored in the KVStore to record a grant with full context +Grant is stored in the KVStore to record a grant with full context | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -5137,17 +4051,11 @@ Grant is stored in the KVStore to record a grant with full context | `grantee` | [string](#string) | | grantee is the address of the user being granted an allowance of another user's funds. | | `allowance` | [google.protobuf.Any](#google.protobuf.Any) | | allowance can be any of basic and filtered fee allowance. | - - - - - ### PeriodicAllowance -PeriodicAllowance extends Allowance to allow for both a maximum cap, -as well as a limit per time period. +PeriodicAllowance extends Allowance to allow for both a maximum cap, as well as a limit per time period. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -5157,10 +4065,6 @@ as well as a limit per time period. | `period_can_spend` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | period_can_spend is the number of coins left to be spent before the period_reset time | | `period_reset` | [google.protobuf.Timestamp](#google.protobuf.Timestamp) | | period_reset is the time at which this period resets and a new one begins, it is calculated from the start time of the first transaction after the last period ended | - - - - @@ -5175,23 +4079,19 @@ as well as a limit per time period.

Top

## cosmos/feegrant/v1beta1/genesis.proto -Since: cosmos-sdk 0.43 +Since: cosmos-sdk 0.43 ### GenesisState -GenesisState contains a set of fee allowances, persisted from the store +GenesisState contains a set of fee allowances, persisted from the store | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `allowances` | [Grant](#cosmos.feegrant.v1beta1.Grant) | repeated | | - - - - @@ -5206,71 +4106,52 @@ GenesisState contains a set of fee allowances, persisted from the store

Top

## cosmos/feegrant/v1beta1/query.proto -Since: cosmos-sdk 0.43 +Since: cosmos-sdk 0.43 ### QueryAllowanceRequest -QueryAllowanceRequest is the request type for the Query/Allowance RPC method. +QueryAllowanceRequest is the request type for the Query/Allowance RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `granter` | [string](#string) | | granter is the address of the user granting an allowance of their funds. | | `grantee` | [string](#string) | | grantee is the address of the user being granted an allowance of another user's funds. | - - - - - ### QueryAllowanceResponse -QueryAllowanceResponse is the response type for the Query/Allowance RPC method. +QueryAllowanceResponse is the response type for the Query/Allowance RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `allowance` | [Grant](#cosmos.feegrant.v1beta1.Grant) | | allowance is a allowance granted for grantee by granter. | - - - - - ### QueryAllowancesRequest -QueryAllowancesRequest is the request type for the Query/Allowances RPC method. +QueryAllowancesRequest is the request type for the Query/Allowances RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `grantee` | [string](#string) | | | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an pagination for the request. | + +### QueryAllowancesResponse - - - - - -### QueryAllowancesResponse -QueryAllowancesResponse is the response type for the Query/Allowances RPC method. - +QueryAllowancesResponse is the response type for the Query/Allowances RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `allowances` | [Grant](#cosmos.feegrant.v1beta1.Grant) | repeated | allowances are allowance's granted for grantee by granter. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines an pagination for the response. | - - - - @@ -5281,6 +4162,7 @@ QueryAllowancesResponse is the response type for the Query/Allowances RPC method ### Query + Query defines the gRPC querier service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -5296,15 +4178,14 @@ Query defines the gRPC querier service.

Top

## cosmos/feegrant/v1beta1/tx.proto -Since: cosmos-sdk 0.43 +Since: cosmos-sdk 0.43 ### MsgGrantAllowance -MsgGrantAllowance adds permission for Grantee to spend up to Allowance -of fees from the account of Granter. +MsgGrantAllowance adds permission for Grantee to spend up to Allowance of fees from the account of Granter. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -5312,40 +4193,27 @@ of fees from the account of Granter. | `grantee` | [string](#string) | | grantee is the address of the user being granted an allowance of another user's funds. | | `allowance` | [google.protobuf.Any](#google.protobuf.Any) | | allowance can be any of basic and filtered fee allowance. | - - - - - ### MsgGrantAllowanceResponse -MsgGrantAllowanceResponse defines the Msg/GrantAllowanceResponse response type. - - - - +MsgGrantAllowanceResponse defines the Msg/GrantAllowanceResponse response type. ### MsgRevokeAllowance -MsgRevokeAllowance removes any existing Allowance from Granter to Grantee. +MsgRevokeAllowance removes any existing Allowance from Granter to Grantee. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `granter` | [string](#string) | | granter is the address of the user granting an allowance of their funds. | | `grantee` | [string](#string) | | grantee is the address of the user being granted an allowance of another user's funds. | - - - - - ### MsgRevokeAllowanceResponse + MsgRevokeAllowanceResponse defines the Msg/RevokeAllowanceResponse response type. @@ -5362,6 +4230,7 @@ MsgRevokeAllowanceResponse defines the Msg/RevokeAllowanceResponse response type ### Msg + Msg defines the feegrant msg service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -5378,22 +4247,16 @@ Msg defines the feegrant msg service. ## cosmos/genutil/v1beta1/genesis.proto - - ### GenesisState -GenesisState defines the raw genesis transaction in JSON. +GenesisState defines the raw genesis transaction in JSON. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `gen_txs` | [bytes](#bytes) | repeated | gen_txs defines the genesis transactions. | - - - - @@ -5409,14 +4272,11 @@ GenesisState defines the raw genesis transaction in JSON. ## cosmos/gov/v1beta1/gov.proto - - ### Deposit -Deposit defines an amount deposited by an account address to an active -proposal. +Deposit defines an amount deposited by an account address to an active proposal. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -5424,32 +4284,22 @@ proposal. | `depositor` | [string](#string) | | | | `amount` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | | - - - - - ### DepositParams -DepositParams defines the params for deposits on governance proposals. +DepositParams defines the params for deposits on governance proposals. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `min_deposit` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | Minimum deposit for a proposal to enter voting period. | | `max_deposit_period` | [google.protobuf.Duration](#google.protobuf.Duration) | | Maximum period for Atom holders to deposit on a proposal. Initial value: 2 months. | - - - - - ### Proposal -Proposal defines the core field members of a governance proposal. +Proposal defines the core field members of a governance proposal. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -5463,16 +4313,11 @@ Proposal defines the core field members of a governance proposal. | `voting_start_time` | [google.protobuf.Timestamp](#google.protobuf.Timestamp) | | | | `voting_end_time` | [google.protobuf.Timestamp](#google.protobuf.Timestamp) | | | - - - - - ### TallyParams -TallyParams defines the params for tallying votes on governance proposals. +TallyParams defines the params for tallying votes on governance proposals. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -5480,16 +4325,11 @@ TallyParams defines the params for tallying votes on governance proposals. | `threshold` | [bytes](#bytes) | | Minimum proportion of Yes votes for proposal to pass. Default value: 0.5. | | `veto_threshold` | [bytes](#bytes) | | Minimum value of Veto votes to Total votes ratio for proposal to be vetoed. Default value: 1/3. | - - - - - ### TallyResult -TallyResult defines a standard tally for a governance proposal. +TallyResult defines a standard tally for a governance proposal. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -5498,85 +4338,61 @@ TallyResult defines a standard tally for a governance proposal. | `no` | [string](#string) | | | | `no_with_veto` | [string](#string) | | | - - - - - ### TextProposal -TextProposal defines a standard text proposal whose changes need to be -manually updated in case of approval. +TextProposal defines a standard text proposal whose changes need to be manually updated in case of approval. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `title` | [string](#string) | | | | `description` | [string](#string) | | | - - - - - ### Vote -Vote defines a vote on a governance proposal. -A Vote consists of a proposal ID, the voter, and the vote option. +Vote defines a vote on a governance proposal. A Vote consists of a proposal ID, the voter, and the vote option. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | | | `voter` | [string](#string) | | | -| `option` | [VoteOption](#cosmos.gov.v1beta1.VoteOption) | | **Deprecated.** Deprecated: Prefer to use `options` instead. This field is set in queries if and only if `len(options) == 1` and that option has weight 1. In all other cases, this field will default to VOTE_OPTION_UNSPECIFIED. | +| `option` | [VoteOption](#cosmos.gov.v1beta1.VoteOption) | | ** +Deprecated.** Deprecated: Prefer to use `options` instead. This field is set in queries if and only if `len(options) == 1` and that option has weight 1. In all other cases, this field will default to VOTE_OPTION_UNSPECIFIED. | | `options` | [WeightedVoteOption](#cosmos.gov.v1beta1.WeightedVoteOption) | repeated | Since: cosmos-sdk 0.43 | - - - - - ### VotingParams -VotingParams defines the params for voting on governance proposals. +VotingParams defines the params for voting on governance proposals. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `voting_period` | [google.protobuf.Duration](#google.protobuf.Duration) | | Length of the voting period. | - - - - - ### WeightedVoteOption + WeightedVoteOption defines a unit of vote for vote split. Since: cosmos-sdk 0.43 - | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `option` | [VoteOption](#cosmos.gov.v1beta1.VoteOption) | | | | `weight` | [string](#string) | | | - - - - ### ProposalStatus + ProposalStatus enumerates the valid statuses of a proposal. | Name | Number | Description | @@ -5588,11 +4404,10 @@ ProposalStatus enumerates the valid statuses of a proposal. | PROPOSAL_STATUS_REJECTED | 4 | PROPOSAL_STATUS_REJECTED defines a proposal status of a proposal that has been rejected. | | PROPOSAL_STATUS_FAILED | 5 | PROPOSAL_STATUS_FAILED defines a proposal status of a proposal that has failed. | - - ### VoteOption + VoteOption enumerates the valid vote options for a given governance proposal. | Name | Number | Description | @@ -5603,7 +4418,6 @@ VoteOption enumerates the valid vote options for a given governance proposal. | VOTE_OPTION_NO | 3 | VOTE_OPTION_NO defines a no vote option. | | VOTE_OPTION_NO_WITH_VETO | 4 | VOTE_OPTION_NO_WITH_VETO defines a no with veto vote option. | - @@ -5617,13 +4431,11 @@ VoteOption enumerates the valid vote options for a given governance proposal. ## cosmos/gov/v1beta1/genesis.proto - - ### GenesisState -GenesisState defines the gov module's genesis state. +GenesisState defines the gov module's genesis state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -5635,10 +4447,6 @@ GenesisState defines the gov module's genesis state. | `voting_params` | [VotingParams](#cosmos.gov.v1beta1.VotingParams) | | params defines all the paramaters of related to voting. | | `tally_params` | [TallyParams](#cosmos.gov.v1beta1.TallyParams) | | params defines all the paramaters of related to tally. | - - - - @@ -5654,91 +4462,64 @@ GenesisState defines the gov module's genesis state. ## cosmos/gov/v1beta1/query.proto - - ### QueryDepositRequest -QueryDepositRequest is the request type for the Query/Deposit RPC method. +QueryDepositRequest is the request type for the Query/Deposit RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal_id defines the unique id of the proposal. | | `depositor` | [string](#string) | | depositor defines the deposit addresses from the proposals. | - - - - - ### QueryDepositResponse -QueryDepositResponse is the response type for the Query/Deposit RPC method. +QueryDepositResponse is the response type for the Query/Deposit RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `deposit` | [Deposit](#cosmos.gov.v1beta1.Deposit) | | deposit defines the requested deposit. | - - - - - ### QueryDepositsRequest -QueryDepositsRequest is the request type for the Query/Deposits RPC method. +QueryDepositsRequest is the request type for the Query/Deposits RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal_id defines the unique id of the proposal. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryDepositsResponse -QueryDepositsResponse is the response type for the Query/Deposits RPC method. +QueryDepositsResponse is the response type for the Query/Deposits RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `deposits` | [Deposit](#cosmos.gov.v1beta1.Deposit) | repeated | | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryParamsRequest -QueryParamsRequest is the request type for the Query/Params RPC method. +QueryParamsRequest is the request type for the Query/Params RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `params_type` | [string](#string) | | params_type defines which parameters to query for, can be one of "voting", "tallying" or "deposit". | - - - - - ### QueryParamsResponse -QueryParamsResponse is the response type for the Query/Params RPC method. +QueryParamsResponse is the response type for the Query/Params RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -5746,46 +4527,31 @@ QueryParamsResponse is the response type for the Query/Params RPC method. | `deposit_params` | [DepositParams](#cosmos.gov.v1beta1.DepositParams) | | deposit_params defines the parameters related to deposit. | | `tally_params` | [TallyParams](#cosmos.gov.v1beta1.TallyParams) | | tally_params defines the parameters related to tally. | - - - - - ### QueryProposalRequest -QueryProposalRequest is the request type for the Query/Proposal RPC method. +QueryProposalRequest is the request type for the Query/Proposal RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal_id defines the unique id of the proposal. | - - - - - ### QueryProposalResponse -QueryProposalResponse is the response type for the Query/Proposal RPC method. +QueryProposalResponse is the response type for the Query/Proposal RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal` | [Proposal](#cosmos.gov.v1beta1.Proposal) | | | - - - - - ### QueryProposalsRequest -QueryProposalsRequest is the request type for the Query/Proposals RPC method. +QueryProposalsRequest is the request type for the Query/Proposals RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -5794,120 +4560,80 @@ QueryProposalsRequest is the request type for the Query/Proposals RPC method. | `depositor` | [string](#string) | | depositor defines the deposit addresses from the proposals. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryProposalsResponse -QueryProposalsResponse is the response type for the Query/Proposals RPC -method. +QueryProposalsResponse is the response type for the Query/Proposals RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposals` | [Proposal](#cosmos.gov.v1beta1.Proposal) | repeated | | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryTallyResultRequest -QueryTallyResultRequest is the request type for the Query/Tally RPC method. +QueryTallyResultRequest is the request type for the Query/Tally RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal_id defines the unique id of the proposal. | - - - - - ### QueryTallyResultResponse -QueryTallyResultResponse is the response type for the Query/Tally RPC method. +QueryTallyResultResponse is the response type for the Query/Tally RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `tally` | [TallyResult](#cosmos.gov.v1beta1.TallyResult) | | tally defines the requested tally. | - - - - - ### QueryVoteRequest -QueryVoteRequest is the request type for the Query/Vote RPC method. +QueryVoteRequest is the request type for the Query/Vote RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal_id defines the unique id of the proposal. | | `voter` | [string](#string) | | voter defines the oter address for the proposals. | - - - - - ### QueryVoteResponse -QueryVoteResponse is the response type for the Query/Vote RPC method. +QueryVoteResponse is the response type for the Query/Vote RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `vote` | [Vote](#cosmos.gov.v1beta1.Vote) | | vote defined the queried vote. | - - - - - ### QueryVotesRequest -QueryVotesRequest is the request type for the Query/Votes RPC method. +QueryVotesRequest is the request type for the Query/Votes RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal_id defines the unique id of the proposal. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryVotesResponse -QueryVotesResponse is the response type for the Query/Votes RPC method. +QueryVotesResponse is the response type for the Query/Votes RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `votes` | [Vote](#cosmos.gov.v1beta1.Vote) | repeated | votes defined the queried votes. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - @@ -5918,6 +4644,7 @@ QueryVotesResponse is the response type for the Query/Votes RPC method. ### Query + Query defines the gRPC querier service for gov module | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -5940,13 +4667,11 @@ Query defines the gRPC querier service for gov module ## cosmos/gov/v1beta1/tx.proto - - ### MsgDeposit -MsgDeposit defines a message to submit a deposit to an existing proposal. +MsgDeposit defines a message to submit a deposit to an existing proposal. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -5954,27 +4679,17 @@ MsgDeposit defines a message to submit a deposit to an existing proposal. | `depositor` | [string](#string) | | | | `amount` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | | - - - - - ### MsgDepositResponse -MsgDepositResponse defines the Msg/Deposit response type. - - - - +MsgDepositResponse defines the Msg/Deposit response type. ### MsgSubmitProposal -MsgSubmitProposal defines an sdk.Msg type that supports submitting arbitrary -proposal Content. +MsgSubmitProposal defines an sdk.Msg type that supports submitting arbitrary proposal Content. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -5982,31 +4697,21 @@ proposal Content. | `initial_deposit` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | | | `proposer` | [string](#string) | | | - - - - - ### MsgSubmitProposalResponse -MsgSubmitProposalResponse defines the Msg/SubmitProposal response type. +MsgSubmitProposalResponse defines the Msg/SubmitProposal response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | | - - - - - ### MsgVote -MsgVote defines a message to cast a vote. +MsgVote defines a message to cast a vote. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6014,43 +4719,30 @@ MsgVote defines a message to cast a vote. | `voter` | [string](#string) | | | | `option` | [VoteOption](#cosmos.gov.v1beta1.VoteOption) | | | + +### MsgVoteResponse - - - - - -### MsgVoteResponse MsgVoteResponse defines the Msg/Vote response type. - - - - - ### MsgVoteWeighted + MsgVoteWeighted defines a message to cast a vote. Since: cosmos-sdk 0.43 - | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | | | `voter` | [string](#string) | | | | `options` | [WeightedVoteOption](#cosmos.gov.v1beta1.WeightedVoteOption) | repeated | | - - - - - ### MsgVoteWeightedResponse + MsgVoteWeightedResponse defines the Msg/VoteWeighted response type. Since: cosmos-sdk 0.43 @@ -6069,6 +4761,7 @@ Since: cosmos-sdk 0.43 ### Msg + Msg defines the bank Msg service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -6077,8 +4770,9 @@ Msg defines the bank Msg service. | `Vote` | [MsgVote](#cosmos.gov.v1beta1.MsgVote) | [MsgVoteResponse](#cosmos.gov.v1beta1.MsgVoteResponse) | Vote defines a method to add a vote on a specific proposal. | | | `VoteWeighted` | [MsgVoteWeighted](#cosmos.gov.v1beta1.MsgVoteWeighted) | [MsgVoteWeightedResponse](#cosmos.gov.v1beta1.MsgVoteWeightedResponse) | VoteWeighted defines a method to add a weighted vote on a specific proposal. -Since: cosmos-sdk 0.43 | | -| `Deposit` | [MsgDeposit](#cosmos.gov.v1beta1.MsgDeposit) | [MsgDepositResponse](#cosmos.gov.v1beta1.MsgDepositResponse) | Deposit defines a method to add deposit on a specific proposal. | | +Since: cosmos-sdk 0.43 | | | `Deposit` | [MsgDeposit](#cosmos.gov.v1beta1.MsgDeposit) +| [MsgDepositResponse](#cosmos.gov.v1beta1.MsgDepositResponse) | Deposit defines a method to add deposit on a specific +proposal. | | @@ -6089,14 +4783,11 @@ Since: cosmos-sdk 0.43 | | ## cosmos/gov/v1beta2/gov.proto - - ### Deposit -Deposit defines an amount deposited by an account address to an active -proposal. +Deposit defines an amount deposited by an account address to an active proposal. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6104,32 +4795,22 @@ proposal. | `depositor` | [string](#string) | | | | `amount` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | | - - - - - ### DepositParams -DepositParams defines the params for deposits on governance proposals. +DepositParams defines the params for deposits on governance proposals. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `min_deposit` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | Minimum deposit for a proposal to enter voting period. | | `max_deposit_period` | [google.protobuf.Duration](#google.protobuf.Duration) | | Maximum period for Atom holders to deposit on a proposal. Initial value: 2 months. | - - - - - ### Proposal -Proposal defines the core field members of a governance proposal. +Proposal defines the core field members of a governance proposal. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6143,16 +4824,11 @@ Proposal defines the core field members of a governance proposal. | `voting_start_time` | [google.protobuf.Timestamp](#google.protobuf.Timestamp) | | | | `voting_end_time` | [google.protobuf.Timestamp](#google.protobuf.Timestamp) | | | - - - - - ### TallyParams -TallyParams defines the params for tallying votes on governance proposals. +TallyParams defines the params for tallying votes on governance proposals. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6160,16 +4836,11 @@ TallyParams defines the params for tallying votes on governance proposals. | `threshold` | [bytes](#bytes) | | Minimum proportion of Yes votes for proposal to pass. Default value: 0.5. | | `veto_threshold` | [bytes](#bytes) | | Minimum value of Veto votes to Total votes ratio for proposal to be vetoed. Default value: 1/3. | - - - - - ### TallyResult -TallyResult defines a standard tally for a governance proposal. +TallyResult defines a standard tally for a governance proposal. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6178,66 +4849,48 @@ TallyResult defines a standard tally for a governance proposal. | `no` | [string](#string) | | | | `no_with_veto` | [string](#string) | | | - - - - - ### Vote -Vote defines a vote on a governance proposal. -A Vote consists of a proposal ID, the voter, and the vote option. +Vote defines a vote on a governance proposal. A Vote consists of a proposal ID, the voter, and the vote option. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | | | `voter` | [string](#string) | | | -| `option` | [VoteOption](#cosmos.gov.v1beta2.VoteOption) | | **Deprecated.** Deprecated: Prefer to use `options` instead. This field is set in queries if and only if `len(options) == 1` and that option has weight 1. In all other cases, this field will default to VOTE_OPTION_UNSPECIFIED. | +| `option` | [VoteOption](#cosmos.gov.v1beta2.VoteOption) | | ** +Deprecated.** Deprecated: Prefer to use `options` instead. This field is set in queries if and only if `len(options) == 1` and that option has weight 1. In all other cases, this field will default to VOTE_OPTION_UNSPECIFIED. | | `options` | [WeightedVoteOption](#cosmos.gov.v1beta2.WeightedVoteOption) | repeated | | - - - - - ### VotingParams -VotingParams defines the params for voting on governance proposals. +VotingParams defines the params for voting on governance proposals. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `voting_period` | [google.protobuf.Duration](#google.protobuf.Duration) | | Length of the voting period. | - - - - - ### WeightedVoteOption -WeightedVoteOption defines a unit of vote for vote split. +WeightedVoteOption defines a unit of vote for vote split. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `option` | [VoteOption](#cosmos.gov.v1beta2.VoteOption) | | | | `weight` | [string](#string) | | | - - - - ### ProposalStatus + ProposalStatus enumerates the valid statuses of a proposal. | Name | Number | Description | @@ -6249,11 +4902,10 @@ ProposalStatus enumerates the valid statuses of a proposal. | PROPOSAL_STATUS_REJECTED | 4 | PROPOSAL_STATUS_REJECTED defines a proposal status of a proposal that has been rejected. | | PROPOSAL_STATUS_FAILED | 5 | PROPOSAL_STATUS_FAILED defines a proposal status of a proposal that has failed. | - - ### VoteOption + VoteOption enumerates the valid vote options for a given governance proposal. | Name | Number | Description | @@ -6264,7 +4916,6 @@ VoteOption enumerates the valid vote options for a given governance proposal. | VOTE_OPTION_NO | 3 | VOTE_OPTION_NO defines a no vote option. | | VOTE_OPTION_NO_WITH_VETO | 4 | VOTE_OPTION_NO_WITH_VETO defines a no with veto vote option. | - @@ -6278,13 +4929,11 @@ VoteOption enumerates the valid vote options for a given governance proposal. ## cosmos/gov/v1beta2/genesis.proto - - ### GenesisState -GenesisState defines the gov module's genesis state. +GenesisState defines the gov module's genesis state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6296,10 +4945,6 @@ GenesisState defines the gov module's genesis state. | `voting_params` | [VotingParams](#cosmos.gov.v1beta2.VotingParams) | | params defines all the paramaters of related to voting. | | `tally_params` | [TallyParams](#cosmos.gov.v1beta2.TallyParams) | | params defines all the paramaters of related to tally. | - - - - @@ -6315,91 +4960,64 @@ GenesisState defines the gov module's genesis state. ## cosmos/gov/v1beta2/query.proto - - ### QueryDepositRequest -QueryDepositRequest is the request type for the Query/Deposit RPC method. +QueryDepositRequest is the request type for the Query/Deposit RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal_id defines the unique id of the proposal. | | `depositor` | [string](#string) | | depositor defines the deposit addresses from the proposals. | - - - - - ### QueryDepositResponse -QueryDepositResponse is the response type for the Query/Deposit RPC method. +QueryDepositResponse is the response type for the Query/Deposit RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `deposit` | [Deposit](#cosmos.gov.v1beta2.Deposit) | | deposit defines the requested deposit. | - - - - - ### QueryDepositsRequest -QueryDepositsRequest is the request type for the Query/Deposits RPC method. +QueryDepositsRequest is the request type for the Query/Deposits RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal_id defines the unique id of the proposal. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryDepositsResponse -QueryDepositsResponse is the response type for the Query/Deposits RPC method. +QueryDepositsResponse is the response type for the Query/Deposits RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `deposits` | [Deposit](#cosmos.gov.v1beta2.Deposit) | repeated | | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryParamsRequest -QueryParamsRequest is the request type for the Query/Params RPC method. +QueryParamsRequest is the request type for the Query/Params RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `params_type` | [string](#string) | | params_type defines which parameters to query for, can be one of "voting", "tallying" or "deposit". | - - - - - ### QueryParamsResponse -QueryParamsResponse is the response type for the Query/Params RPC method. +QueryParamsResponse is the response type for the Query/Params RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6407,46 +5025,31 @@ QueryParamsResponse is the response type for the Query/Params RPC method. | `deposit_params` | [DepositParams](#cosmos.gov.v1beta2.DepositParams) | | deposit_params defines the parameters related to deposit. | | `tally_params` | [TallyParams](#cosmos.gov.v1beta2.TallyParams) | | tally_params defines the parameters related to tally. | - - - - - ### QueryProposalRequest -QueryProposalRequest is the request type for the Query/Proposal RPC method. +QueryProposalRequest is the request type for the Query/Proposal RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal_id defines the unique id of the proposal. | - - - - - ### QueryProposalResponse -QueryProposalResponse is the response type for the Query/Proposal RPC method. +QueryProposalResponse is the response type for the Query/Proposal RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal` | [Proposal](#cosmos.gov.v1beta2.Proposal) | | | - - - - - ### QueryProposalsRequest -QueryProposalsRequest is the request type for the Query/Proposals RPC method. +QueryProposalsRequest is the request type for the Query/Proposals RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6455,120 +5058,80 @@ QueryProposalsRequest is the request type for the Query/Proposals RPC method. | `depositor` | [string](#string) | | depositor defines the deposit addresses from the proposals. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryProposalsResponse -QueryProposalsResponse is the response type for the Query/Proposals RPC -method. +QueryProposalsResponse is the response type for the Query/Proposals RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposals` | [Proposal](#cosmos.gov.v1beta2.Proposal) | repeated | | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryTallyResultRequest -QueryTallyResultRequest is the request type for the Query/Tally RPC method. +QueryTallyResultRequest is the request type for the Query/Tally RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal_id defines the unique id of the proposal. | - - - - - ### QueryTallyResultResponse -QueryTallyResultResponse is the response type for the Query/Tally RPC method. +QueryTallyResultResponse is the response type for the Query/Tally RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `tally` | [TallyResult](#cosmos.gov.v1beta2.TallyResult) | | tally defines the requested tally. | - - - - - ### QueryVoteRequest -QueryVoteRequest is the request type for the Query/Vote RPC method. +QueryVoteRequest is the request type for the Query/Vote RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal_id defines the unique id of the proposal. | | `voter` | [string](#string) | | voter defines the oter address for the proposals. | - - - - - ### QueryVoteResponse -QueryVoteResponse is the response type for the Query/Vote RPC method. +QueryVoteResponse is the response type for the Query/Vote RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `vote` | [Vote](#cosmos.gov.v1beta2.Vote) | | vote defined the queried vote. | - - - - - ### QueryVotesRequest -QueryVotesRequest is the request type for the Query/Votes RPC method. +QueryVotesRequest is the request type for the Query/Votes RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal_id defines the unique id of the proposal. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryVotesResponse -QueryVotesResponse is the response type for the Query/Votes RPC method. +QueryVotesResponse is the response type for the Query/Votes RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `votes` | [Vote](#cosmos.gov.v1beta2.Vote) | repeated | votes defined the queried votes. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - @@ -6579,6 +5142,7 @@ QueryVotesResponse is the response type for the Query/Votes RPC method. ### Query + Query defines the gRPC querier service for gov module | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -6601,13 +5165,11 @@ Query defines the gRPC querier service for gov module ## cosmos/gov/v1beta2/tx.proto - - ### MsgDeposit -MsgDeposit defines a message to submit a deposit to an existing proposal. +MsgDeposit defines a message to submit a deposit to an existing proposal. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6615,27 +5177,17 @@ MsgDeposit defines a message to submit a deposit to an existing proposal. | `depositor` | [string](#string) | | | | `amount` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | | - - - - - ### MsgDepositResponse -MsgDepositResponse defines the Msg/Deposit response type. - - - - +MsgDepositResponse defines the Msg/Deposit response type. ### MsgSubmitProposal -MsgSubmitProposal defines an sdk.Msg type that supports submitting arbitrary -proposal Content. +MsgSubmitProposal defines an sdk.Msg type that supports submitting arbitrary proposal Content. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6643,31 +5195,21 @@ proposal Content. | `initial_deposit` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | | | `proposer` | [string](#string) | | | - - - - - ### MsgSubmitProposalResponse -MsgSubmitProposalResponse defines the Msg/SubmitProposal response type. +MsgSubmitProposalResponse defines the Msg/SubmitProposal response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | | - - - - - ### MsgVote -MsgVote defines a message to cast a vote. +MsgVote defines a message to cast a vote. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6675,43 +5217,30 @@ MsgVote defines a message to cast a vote. | `voter` | [string](#string) | | | | `option` | [VoteOption](#cosmos.gov.v1beta2.VoteOption) | | | - - - - - ### MsgVoteResponse -MsgVoteResponse defines the Msg/Vote response type. - - - - +MsgVoteResponse defines the Msg/Vote response type. ### MsgVoteWeighted + MsgVoteWeighted defines a message to cast a vote. Since: cosmos-sdk 0.43 - | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | | | `voter` | [string](#string) | | | | `options` | [WeightedVoteOption](#cosmos.gov.v1beta2.WeightedVoteOption) | repeated | | - - - - - ### MsgVoteWeightedResponse + MsgVoteWeightedResponse defines the Msg/VoteWeighted response type. Since: cosmos-sdk 0.43 @@ -6730,6 +5259,7 @@ Since: cosmos-sdk 0.43 ### Msg + Msg defines the gov Msg service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -6738,8 +5268,9 @@ Msg defines the gov Msg service. | `Vote` | [MsgVote](#cosmos.gov.v1beta2.MsgVote) | [MsgVoteResponse](#cosmos.gov.v1beta2.MsgVoteResponse) | Vote defines a method to add a vote on a specific proposal. | | | `VoteWeighted` | [MsgVoteWeighted](#cosmos.gov.v1beta2.MsgVoteWeighted) | [MsgVoteWeightedResponse](#cosmos.gov.v1beta2.MsgVoteWeightedResponse) | VoteWeighted defines a method to add a weighted vote on a specific proposal. -Since: cosmos-sdk 0.43 | | -| `Deposit` | [MsgDeposit](#cosmos.gov.v1beta2.MsgDeposit) | [MsgDepositResponse](#cosmos.gov.v1beta2.MsgDepositResponse) | Deposit defines a method to add deposit on a specific proposal. | | +Since: cosmos-sdk 0.43 | | | `Deposit` | [MsgDeposit](#cosmos.gov.v1beta2.MsgDeposit) +| [MsgDepositResponse](#cosmos.gov.v1beta2.MsgDepositResponse) | Deposit defines a method to add deposit on a specific +proposal. | | @@ -6750,13 +5281,11 @@ Since: cosmos-sdk 0.43 | | ## cosmos/group/v1beta1/types.proto - - ### GroupAccountInfo -GroupAccountInfo represents the high-level on-chain information for a group account. +GroupAccountInfo represents the high-level on-chain information for a group account. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6768,16 +5297,11 @@ GroupAccountInfo represents the high-level on-chain information for a group acco | `decision_policy` | [google.protobuf.Any](#google.protobuf.Any) | | decision_policy specifies the group account's decision policy. | | `derivation_key` | [bytes](#bytes) | | derivation_key is the "derivation" key of the group account, which is needed to derive the group root module key and execute proposals. | - - - - - ### GroupInfo -GroupInfo represents the high-level on-chain information for a group. +GroupInfo represents the high-level on-chain information for a group. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6787,33 +5311,22 @@ GroupInfo represents the high-level on-chain information for a group. | `version` | [uint64](#uint64) | | version is used to track changes to a group's membership structure that would break existing proposals. Whenever any members weight is changed, or any member is added or removed this version is incremented and will cause proposals based on older versions of this group to fail | | `total_weight` | [string](#string) | | total_weight is the sum of the group members' weights. | - - - - - ### GroupMember -GroupMember represents the relationship between a group and a member. +GroupMember represents the relationship between a group and a member. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `group_id` | [uint64](#uint64) | | group_id is the unique ID of the group. | | `member` | [Member](#cosmos.group.v1beta1.Member) | | member is the member data. | - - - - - ### Member -Member represents a group member with an account address, -non-zero weight and metadata. +Member represents a group member with an account address, non-zero weight and metadata. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6821,34 +5334,23 @@ non-zero weight and metadata. | `weight` | [string](#string) | | weight is the member's voting weight that should be greater than 0. | | `metadata` | [bytes](#bytes) | | metadata is any arbitrary metadata to attached to the member. | - - - - - ### Members -Members defines a repeated slice of Member objects. +Members defines a repeated slice of Member objects. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `members` | [Member](#cosmos.group.v1beta1.Member) | repeated | members is the list of members. | - - - - - ### Proposal -Proposal defines a group proposal. Any member of a group can submit a proposal -for a group account to decide upon. -A proposal consists of a set of `sdk.Msg`s that will be executed if the proposal -passes as well as some optional metadata associated with the proposal. +Proposal defines a group proposal. Any member of a group can submit a proposal for a group account to decide upon. A +proposal consists of a set of `sdk.Msg`s that will be executed if the proposal passes as well as some optional metadata +associated with the proposal. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6866,16 +5368,11 @@ passes as well as some optional metadata associated with the proposal. | `executor_result` | [Proposal.ExecutorResult](#cosmos.group.v1beta1.Proposal.ExecutorResult) | | executor_result is the final result based on the votes and election rule. Initial value is NotRun. | | `msgs` | [google.protobuf.Any](#google.protobuf.Any) | repeated | msgs is a list of Msgs that will be executed if the proposal passes. | - - - - - ### Tally -Tally represents the sum of weighted votes. +Tally represents the sum of weighted votes. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6884,32 +5381,22 @@ Tally represents the sum of weighted votes. | `abstain_count` | [string](#string) | | abstain_count is the weighted sum of abstainers | | `veto_count` | [string](#string) | | veto_count is the weighted sum of vetoes. | - - - - - ### ThresholdDecisionPolicy -ThresholdDecisionPolicy implements the DecisionPolicy interface +ThresholdDecisionPolicy implements the DecisionPolicy interface | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `threshold` | [string](#string) | | threshold is the minimum weighted sum of yes votes that must be met or exceeded for a proposal to succeed. | | `timeout` | [google.protobuf.Duration](#google.protobuf.Duration) | | timeout is the duration from submission of a proposal to the end of voting period Within this times votes and exec messages can be submitted. | - - - - - ### Vote -Vote represents a vote for a proposal. +Vote represents a vote for a proposal. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -6919,16 +5406,13 @@ Vote represents a vote for a proposal. | `metadata` | [bytes](#bytes) | | metadata is any arbitrary metadata to attached to the vote. | | `submitted_at` | [google.protobuf.Timestamp](#google.protobuf.Timestamp) | | submitted_at is the timestamp when the vote was submitted. | - - - - ### Choice + Choice defines available types of choices for voting. | Name | Number | Description | @@ -6939,11 +5423,10 @@ Choice defines available types of choices for voting. | CHOICE_ABSTAIN | 3 | CHOICE_ABSTAIN defines an abstaining voting choice. | | CHOICE_VETO | 4 | CHOICE_VETO defines a voting choice with veto. | - - ### Proposal.ExecutorResult + ExecutorResult defines types of proposal executor results. | Name | Number | Description | @@ -6953,11 +5436,10 @@ ExecutorResult defines types of proposal executor results. | EXECUTOR_RESULT_SUCCESS | 2 | The executor was successful and proposed action updated state. | | EXECUTOR_RESULT_FAILURE | 3 | The executor returned an error and proposed action didn't update state. | - - ### Proposal.Result + Result defines types of proposal results. | Name | Number | Description | @@ -6967,11 +5449,10 @@ Result defines types of proposal results. | RESULT_ACCEPTED | 2 | Final result of the tally | | RESULT_REJECTED | 3 | Final result of the tally | - - ### Proposal.Status + Status defines proposal statuses. | Name | Number | Description | @@ -6981,7 +5462,6 @@ Status defines proposal statuses. | STATUS_CLOSED | 2 | Final status of a proposal when the final tally was executed. | | STATUS_ABORTED | 3 | Final status of a proposal when the group was modified before the final tally. | - @@ -6995,352 +5475,241 @@ Status defines proposal statuses. ## cosmos/group/v1beta1/query.proto - - ### QueryGroupAccountInfoRequest -QueryGroupAccountInfoRequest is the Query/GroupAccountInfo request type. +QueryGroupAccountInfoRequest is the Query/GroupAccountInfo request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address` | [string](#string) | | address is the account address of the group account. | - - - - - ### QueryGroupAccountInfoResponse -QueryGroupAccountInfoResponse is the Query/GroupAccountInfo response type. +QueryGroupAccountInfoResponse is the Query/GroupAccountInfo response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `info` | [GroupAccountInfo](#cosmos.group.v1beta1.GroupAccountInfo) | | info is the GroupAccountInfo for the group account. | - - - - - ### QueryGroupAccountsByAdminRequest -QueryGroupAccountsByAdminRequest is the Query/GroupAccountsByAdmin request type. +QueryGroupAccountsByAdminRequest is the Query/GroupAccountsByAdmin request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `admin` | [string](#string) | | admin is the admin address of the group account. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryGroupAccountsByAdminResponse -QueryGroupAccountsByAdminResponse is the Query/GroupAccountsByAdmin response type. +QueryGroupAccountsByAdminResponse is the Query/GroupAccountsByAdmin response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `group_accounts` | [GroupAccountInfo](#cosmos.group.v1beta1.GroupAccountInfo) | repeated | group_accounts are the group accounts info with provided admin. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryGroupAccountsByGroupRequest -QueryGroupAccountsByGroupRequest is the Query/GroupAccountsByGroup request type. +QueryGroupAccountsByGroupRequest is the Query/GroupAccountsByGroup request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `group_id` | [uint64](#uint64) | | group_id is the unique ID of the group account's group. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryGroupAccountsByGroupResponse -QueryGroupAccountsByGroupResponse is the Query/GroupAccountsByGroup response type. +QueryGroupAccountsByGroupResponse is the Query/GroupAccountsByGroup response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `group_accounts` | [GroupAccountInfo](#cosmos.group.v1beta1.GroupAccountInfo) | repeated | group_accounts are the group accounts info associated with the provided group. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryGroupInfoRequest -QueryGroupInfoRequest is the Query/GroupInfo request type. +QueryGroupInfoRequest is the Query/GroupInfo request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `group_id` | [uint64](#uint64) | | group_id is the unique ID of the group. | - - - - - ### QueryGroupInfoResponse -QueryGroupInfoResponse is the Query/GroupInfo response type. +QueryGroupInfoResponse is the Query/GroupInfo response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `info` | [GroupInfo](#cosmos.group.v1beta1.GroupInfo) | | info is the GroupInfo for the group. | - - - - - ### QueryGroupMembersRequest -QueryGroupMembersRequest is the Query/GroupMembersRequest request type. +QueryGroupMembersRequest is the Query/GroupMembersRequest request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `group_id` | [uint64](#uint64) | | group_id is the unique ID of the group. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryGroupMembersResponse -QueryGroupMembersResponse is the Query/GroupMembersResponse response type. +QueryGroupMembersResponse is the Query/GroupMembersResponse response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `members` | [GroupMember](#cosmos.group.v1beta1.GroupMember) | repeated | members are the members of the group with given group_id. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryGroupsByAdminRequest -QueryGroupsByAdminRequest is the Query/GroupsByAdminRequest request type. +QueryGroupsByAdminRequest is the Query/GroupsByAdminRequest request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `admin` | [string](#string) | | admin is the account address of a group's admin. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryGroupsByAdminResponse -QueryGroupsByAdminResponse is the Query/GroupsByAdminResponse response type. +QueryGroupsByAdminResponse is the Query/GroupsByAdminResponse response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `groups` | [GroupInfo](#cosmos.group.v1beta1.GroupInfo) | repeated | groups are the groups info with the provided admin. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryProposalRequest -QueryProposalRequest is the Query/Proposal request type. +QueryProposalRequest is the Query/Proposal request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal_id is the unique ID of a proposal. | - - - - - ### QueryProposalResponse -QueryProposalResponse is the Query/Proposal response type. +QueryProposalResponse is the Query/Proposal response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal` | [Proposal](#cosmos.group.v1beta1.Proposal) | | proposal is the proposal info. | - - - - - ### QueryProposalsByGroupAccountRequest -QueryProposalsByGroupAccountRequest is the Query/ProposalByGroupAccount request type. +QueryProposalsByGroupAccountRequest is the Query/ProposalByGroupAccount request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address` | [string](#string) | | address is the group account address related to proposals. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryProposalsByGroupAccountResponse -QueryProposalsByGroupAccountResponse is the Query/ProposalByGroupAccount response type. +QueryProposalsByGroupAccountResponse is the Query/ProposalByGroupAccount response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposals` | [Proposal](#cosmos.group.v1beta1.Proposal) | repeated | proposals are the proposals with given group account. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryVoteByProposalVoterRequest -QueryVoteByProposalVoterResponse is the Query/VoteByProposalVoter request type. +QueryVoteByProposalVoterResponse is the Query/VoteByProposalVoter request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal_id is the unique ID of a proposal. | | `voter` | [string](#string) | | voter is a proposal voter account address. | - - - - - ### QueryVoteByProposalVoterResponse -QueryVoteByProposalVoterResponse is the Query/VoteByProposalVoter response type. +QueryVoteByProposalVoterResponse is the Query/VoteByProposalVoter response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `vote` | [Vote](#cosmos.group.v1beta1.Vote) | | vote is the vote with given proposal_id and voter. | - - - - - ### QueryVotesByProposalRequest -QueryVotesByProposalResponse is the Query/VotesByProposal request type. +QueryVotesByProposalResponse is the Query/VotesByProposal request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal_id is the unique ID of a proposal. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryVotesByProposalResponse -QueryVotesByProposalResponse is the Query/VotesByProposal response type. +QueryVotesByProposalResponse is the Query/VotesByProposal response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `votes` | [Vote](#cosmos.group.v1beta1.Vote) | repeated | votes are the list of votes for given proposal_id. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryVotesByVoterRequest -QueryVotesByVoterResponse is the Query/VotesByVoter request type. +QueryVotesByVoterResponse is the Query/VotesByVoter request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `voter` | [string](#string) | | voter is a proposal voter account address. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryVotesByVoterResponse -QueryVotesByVoterResponse is the Query/VotesByVoter response type. +QueryVotesByVoterResponse is the Query/VotesByVoter response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `votes` | [Vote](#cosmos.group.v1beta1.Vote) | repeated | votes are the list of votes by given voter. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - @@ -7351,6 +5720,7 @@ QueryVotesByVoterResponse is the Query/VotesByVoter response type. ### Query + Query is the cosmos.group.v1beta1 Query service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -7376,13 +5746,11 @@ Query is the cosmos.group.v1beta1 Query service. ## cosmos/group/v1beta1/tx.proto - - ### MsgCreateGroupAccountRequest -MsgCreateGroupAccountRequest is the Msg/CreateGroupAccount request type. +MsgCreateGroupAccountRequest is the Msg/CreateGroupAccount request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -7391,31 +5759,21 @@ MsgCreateGroupAccountRequest is the Msg/CreateGroupAccount request type. | `metadata` | [bytes](#bytes) | | metadata is any arbitrary metadata to attached to the group account. | | `decision_policy` | [google.protobuf.Any](#google.protobuf.Any) | | decision_policy specifies the group account's decision policy. | - - - - - ### MsgCreateGroupAccountResponse -MsgCreateGroupAccountResponse is the Msg/CreateGroupAccount response type. +MsgCreateGroupAccountResponse is the Msg/CreateGroupAccount response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address` | [string](#string) | | address is the account address of the newly created group account. | - - - - - ### MsgCreateGroupRequest -MsgCreateGroupRequest is the Msg/CreateGroup request type. +MsgCreateGroupRequest is the Msg/CreateGroup request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -7423,31 +5781,21 @@ MsgCreateGroupRequest is the Msg/CreateGroup request type. | `members` | [Member](#cosmos.group.v1beta1.Member) | repeated | members defines the group members. | | `metadata` | [bytes](#bytes) | | metadata is any arbitrary metadata to attached to the group. | - - - - - ### MsgCreateGroupResponse -MsgCreateGroupResponse is the Msg/CreateGroup response type. +MsgCreateGroupResponse is the Msg/CreateGroup response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `group_id` | [uint64](#uint64) | | group_id is the unique ID of the newly created group. | - - - - - ### MsgCreateProposalRequest -MsgCreateProposalRequest is the Msg/CreateProposal request type. +MsgCreateProposalRequest is the Msg/CreateProposal request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -7457,57 +5805,38 @@ MsgCreateProposalRequest is the Msg/CreateProposal request type. | `msgs` | [google.protobuf.Any](#google.protobuf.Any) | repeated | msgs is a list of Msgs that will be executed if the proposal passes. | | `exec` | [Exec](#cosmos.group.v1beta1.Exec) | | exec defines the mode of execution of the proposal, whether it should be executed immediately on creation or not. If so, proposers signatures are considered as Yes votes. | - - - - - ### MsgCreateProposalResponse -MsgCreateProposalResponse is the Msg/CreateProposal response type. +MsgCreateProposalResponse is the Msg/CreateProposal response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal is the unique ID of the proposal. | - - - - - ### MsgExecRequest -MsgExecRequest is the Msg/Exec request type. +MsgExecRequest is the Msg/Exec request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal is the unique ID of the proposal. | | `signer` | [string](#string) | | signer is the account address used to execute the proposal. | - - - - - ### MsgExecResponse -MsgExecResponse is the Msg/Exec request type. - - - - +MsgExecResponse is the Msg/Exec request type. ### MsgUpdateGroupAccountAdminRequest -MsgUpdateGroupAccountAdminRequest is the Msg/UpdateGroupAccountAdmin request type. +MsgUpdateGroupAccountAdminRequest is the Msg/UpdateGroupAccountAdmin request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -7515,26 +5844,17 @@ MsgUpdateGroupAccountAdminRequest is the Msg/UpdateGroupAccountAdmin request typ | `address` | [string](#string) | | address is the group account address. | | `new_admin` | [string](#string) | | new_admin is the new group account admin. | - - - - - ### MsgUpdateGroupAccountAdminResponse -MsgUpdateGroupAccountAdminResponse is the Msg/UpdateGroupAccountAdmin response type. - - - - +MsgUpdateGroupAccountAdminResponse is the Msg/UpdateGroupAccountAdmin response type. ### MsgUpdateGroupAccountDecisionPolicyRequest -MsgUpdateGroupAccountDecisionPolicyRequest is the Msg/UpdateGroupAccountDecisionPolicy request type. +MsgUpdateGroupAccountDecisionPolicyRequest is the Msg/UpdateGroupAccountDecisionPolicy request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -7542,26 +5862,17 @@ MsgUpdateGroupAccountDecisionPolicyRequest is the Msg/UpdateGroupAccountDecision | `address` | [string](#string) | | address is the group account address. | | `decision_policy` | [google.protobuf.Any](#google.protobuf.Any) | | decision_policy is the updated group account decision policy. | - - - - - ### MsgUpdateGroupAccountDecisionPolicyResponse -MsgUpdateGroupAccountDecisionPolicyResponse is the Msg/UpdateGroupAccountDecisionPolicy response type. - - - - +MsgUpdateGroupAccountDecisionPolicyResponse is the Msg/UpdateGroupAccountDecisionPolicy response type. ### MsgUpdateGroupAccountMetadataRequest -MsgUpdateGroupAccountMetadataRequest is the Msg/UpdateGroupAccountMetadata request type. +MsgUpdateGroupAccountMetadataRequest is the Msg/UpdateGroupAccountMetadata request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -7569,26 +5880,17 @@ MsgUpdateGroupAccountMetadataRequest is the Msg/UpdateGroupAccountMetadata reque | `address` | [string](#string) | | address is the group account address. | | `metadata` | [bytes](#bytes) | | metadata is the updated group account metadata. | - - - - - ### MsgUpdateGroupAccountMetadataResponse -MsgUpdateGroupAccountMetadataResponse is the Msg/UpdateGroupAccountMetadata response type. - - - - +MsgUpdateGroupAccountMetadataResponse is the Msg/UpdateGroupAccountMetadata response type. ### MsgUpdateGroupAdminRequest -MsgUpdateGroupAdminRequest is the Msg/UpdateGroupAdmin request type. +MsgUpdateGroupAdminRequest is the Msg/UpdateGroupAdmin request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -7596,26 +5898,17 @@ MsgUpdateGroupAdminRequest is the Msg/UpdateGroupAdmin request type. | `group_id` | [uint64](#uint64) | | group_id is the unique ID of the group. | | `new_admin` | [string](#string) | | new_admin is the group new admin account address. | - - - - - ### MsgUpdateGroupAdminResponse -MsgUpdateGroupAdminResponse is the Msg/UpdateGroupAdmin response type. - - - - +MsgUpdateGroupAdminResponse is the Msg/UpdateGroupAdmin response type. ### MsgUpdateGroupMembersRequest -MsgUpdateGroupMembersRequest is the Msg/UpdateGroupMembers request type. +MsgUpdateGroupMembersRequest is the Msg/UpdateGroupMembers request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -7623,26 +5916,17 @@ MsgUpdateGroupMembersRequest is the Msg/UpdateGroupMembers request type. | `group_id` | [uint64](#uint64) | | group_id is the unique ID of the group. | | `member_updates` | [Member](#cosmos.group.v1beta1.Member) | repeated | member_updates is the list of members to update, set weight to 0 to remove a member. | - - - - - ### MsgUpdateGroupMembersResponse -MsgUpdateGroupMembersResponse is the Msg/UpdateGroupMembers response type. - - - - +MsgUpdateGroupMembersResponse is the Msg/UpdateGroupMembers response type. ### MsgUpdateGroupMetadataRequest -MsgUpdateGroupMetadataRequest is the Msg/UpdateGroupMetadata request type. +MsgUpdateGroupMetadataRequest is the Msg/UpdateGroupMetadata request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -7650,43 +5934,30 @@ MsgUpdateGroupMetadataRequest is the Msg/UpdateGroupMetadata request type. | `group_id` | [uint64](#uint64) | | group_id is the unique ID of the group. | | `metadata` | [bytes](#bytes) | | metadata is the updated group's metadata. | - - - - - ### MsgUpdateGroupMetadataResponse -MsgUpdateGroupMetadataResponse is the Msg/UpdateGroupMetadata response type. - - - - +MsgUpdateGroupMetadataResponse is the Msg/UpdateGroupMetadata response type. ### MsgVoteRequest -MsgVoteRequest is the Msg/Vote request type. +MsgVoteRequest is the Msg/Vote request type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `proposal_id` | [uint64](#uint64) | | proposal is the unique ID of the proposal. | | `voter` | [string](#string) | | voter is the voter account address. | | `choice` | [Choice](#cosmos.group.v1beta1.Choice) | | choice is the voter's choice on the proposal. | -| `metadata` | [bytes](#bytes) | | metadata is any arbitrary metadata to attached to the vote. | -| `exec` | [Exec](#cosmos.group.v1beta1.Exec) | | exec defines whether the proposal should be executed immediately after voting or not. | - - - - - +| `metadata` | [bytes](#bytes) | | metadata is any arbitrary metadata to attached to the vote. | +| `exec` | [Exec](#cosmos.group.v1beta1.Exec) | | exec defines whether the proposal should be executed immediately after voting or not. | ### MsgVoteResponse + MsgVoteResponse is the Msg/Vote response type. @@ -7699,6 +5970,7 @@ MsgVoteResponse is the Msg/Vote response type. ### Exec + Exec defines modes of execution of a proposal on creation or on new vote. | Name | Number | Description | @@ -7706,7 +5978,6 @@ Exec defines modes of execution of a proposal on creation or on new vote. | EXEC_UNSPECIFIED | 0 | An empty value means that there should be a separate MsgExec request for the proposal to execute. | | EXEC_TRY | 1 | Try to execute the proposal immediately. If the proposal is not allowed per the DecisionPolicy, the proposal will still be open and could be executed at a later point. | - @@ -7715,6 +5986,7 @@ Exec defines modes of execution of a proposal on creation or on new vote. ### Msg + Msg is the cosmos.group.v1beta1 Msg service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -7740,29 +6012,22 @@ Msg is the cosmos.group.v1beta1 Msg service. ## cosmos/mint/v1beta1/mint.proto - - ### Minter -Minter represents the minting state. +Minter represents the minting state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `inflation` | [string](#string) | | current annual inflation rate | | `annual_provisions` | [string](#string) | | current annual expected provisions | - - - - - ### Params -Params holds parameters for the mint module. +Params holds parameters for the mint module. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -7773,10 +6038,6 @@ Params holds parameters for the mint module. | `goal_bonded` | [string](#string) | | goal of percent bonded atoms | | `blocks_per_year` | [uint64](#uint64) | | expected blocks per year | - - - - @@ -7792,23 +6053,17 @@ Params holds parameters for the mint module. ## cosmos/mint/v1beta1/genesis.proto - - ### GenesisState -GenesisState defines the mint module's genesis state. +GenesisState defines the mint module's genesis state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `minter` | [Minter](#cosmos.mint.v1beta1.Minter) | | minter is a space for holding current inflation information. | | `params` | [Params](#cosmos.mint.v1beta1.Params) | | params defines all the paramaters of the module. | - - - - @@ -7824,85 +6079,54 @@ GenesisState defines the mint module's genesis state. ## cosmos/mint/v1beta1/query.proto - - ### QueryAnnualProvisionsRequest -QueryAnnualProvisionsRequest is the request type for the -Query/AnnualProvisions RPC method. - - - - +QueryAnnualProvisionsRequest is the request type for the Query/AnnualProvisions RPC method. ### QueryAnnualProvisionsResponse -QueryAnnualProvisionsResponse is the response type for the -Query/AnnualProvisions RPC method. +QueryAnnualProvisionsResponse is the response type for the Query/AnnualProvisions RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `annual_provisions` | [bytes](#bytes) | | annual_provisions is the current minting annual provisions value. | - - - - - ### QueryInflationRequest -QueryInflationRequest is the request type for the Query/Inflation RPC method. - - - - +QueryInflationRequest is the request type for the Query/Inflation RPC method. ### QueryInflationResponse -QueryInflationResponse is the response type for the Query/Inflation RPC -method. +QueryInflationResponse is the response type for the Query/Inflation RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `inflation` | [bytes](#bytes) | | inflation is the current minting inflation value. | - - - - - ### QueryParamsRequest -QueryParamsRequest is the request type for the Query/Params RPC method. - - - - +QueryParamsRequest is the request type for the Query/Params RPC method. ### QueryParamsResponse -QueryParamsResponse is the response type for the Query/Params RPC method. +QueryParamsResponse is the response type for the Query/Params RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `params` | [Params](#cosmos.mint.v1beta1.Params) | | params defines the parameters of the module. | - - - - @@ -7913,6 +6137,7 @@ QueryParamsResponse is the response type for the Query/Params RPC method. ### Query + Query provides defines the gRPC querier service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -7930,13 +6155,11 @@ Query provides defines the gRPC querier service. ## cosmos/nft/v1beta1/event.proto - - ### EventBurn -EventBurn is emitted on Burn +EventBurn is emitted on Burn | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -7944,16 +6167,11 @@ EventBurn is emitted on Burn | `id` | [string](#string) | | | | `owner` | [string](#string) | | | - - - - - ### EventMint -EventMint is emitted on Mint +EventMint is emitted on Mint | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -7961,16 +6179,11 @@ EventMint is emitted on Mint | `id` | [string](#string) | | | | `owner` | [string](#string) | | | - - - - - ### EventSend -EventSend is emitted on Msg/Send +EventSend is emitted on Msg/Send | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -7979,10 +6192,6 @@ EventSend is emitted on Msg/Send | `sender` | [string](#string) | | | | `receiver` | [string](#string) | | | - - - - @@ -7998,13 +6207,11 @@ EventSend is emitted on Msg/Send ## cosmos/nft/v1beta1/nft.proto - - ### Class -Class defines the class of the nft type. +Class defines the class of the nft type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -8016,16 +6223,11 @@ Class defines the class of the nft type. | `uri_hash` | [string](#string) | | uri_hash is a hash of the document pointed by uri. Optional | | `data` | [google.protobuf.Any](#google.protobuf.Any) | | data is the app specific metadata of the NFT class. Optional | - - - - - ### NFT -NFT defines the NFT. +NFT defines the NFT. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -8035,10 +6237,6 @@ NFT defines the NFT. | `uri_hash` | [string](#string) | | uri_hash is a hash of the document pointed by uri | | `data` | [google.protobuf.Any](#google.protobuf.Any) | | data is an app specific data of the NFT. Optional | - - - - @@ -8054,39 +6252,28 @@ NFT defines the NFT. ## cosmos/nft/v1beta1/genesis.proto - - ### Entry -Entry Defines all nft owned by a person +Entry Defines all nft owned by a person | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `owner` | [string](#string) | | owner is the owner address of the following nft | | `nfts` | [NFT](#cosmos.nft.v1beta1.NFT) | repeated | nfts is a group of nfts of the same owner | - - - - - ### GenesisState -GenesisState defines the nft module's genesis state. +GenesisState defines the nft module's genesis state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `classes` | [Class](#cosmos.nft.v1beta1.Class) | repeated | class defines the class of the nft type. | | `entries` | [Entry](#cosmos.nft.v1beta1.Entry) | repeated | | - - - - @@ -8102,136 +6289,94 @@ GenesisState defines the nft module's genesis state. ## cosmos/nft/v1beta1/query.proto - - ### QueryBalanceRequest -QueryBalanceRequest is the request type for the Query/Balance RPC method +QueryBalanceRequest is the request type for the Query/Balance RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `class_id` | [string](#string) | | | | `owner` | [string](#string) | | | - - - - - ### QueryBalanceResponse -QueryBalanceResponse is the response type for the Query/Balance RPC method +QueryBalanceResponse is the response type for the Query/Balance RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `amount` | [uint64](#uint64) | | | - - - - - ### QueryClassRequest -QueryClassRequest is the request type for the Query/Class RPC method +QueryClassRequest is the request type for the Query/Class RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `class_id` | [string](#string) | | | - - - - - ### QueryClassResponse -QueryClassResponse is the response type for the Query/Class RPC method +QueryClassResponse is the response type for the Query/Class RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `class` | [Class](#cosmos.nft.v1beta1.Class) | | | - - - - - ### QueryClassesRequest -QueryClassesRequest is the request type for the Query/Classes RPC method +QueryClassesRequest is the request type for the Query/Classes RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryClassesResponse -QueryClassesResponse is the response type for the Query/Classes RPC method +QueryClassesResponse is the response type for the Query/Classes RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `classes` | [Class](#cosmos.nft.v1beta1.Class) | repeated | | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | | - - - - - ### QueryNFTRequest -QueryNFTRequest is the request type for the Query/NFT RPC method +QueryNFTRequest is the request type for the Query/NFT RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `class_id` | [string](#string) | | | | `id` | [string](#string) | | | - - - - - ### QueryNFTResponse -QueryNFTResponse is the response type for the Query/NFT RPC method +QueryNFTResponse is the response type for the Query/NFT RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `nft` | [NFT](#cosmos.nft.v1beta1.NFT) | | | - - - - - ### QueryNFTsOfClassRequest -QueryNFTsOfClassRequest is the request type for the Query/NFTsOfClass RPC method +QueryNFTsOfClassRequest is the request type for the Query/NFTsOfClass RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -8239,87 +6384,58 @@ QueryNFTsOfClassRequest is the request type for the Query/NFTsOfClass RPC method | `owner` | [string](#string) | | | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | | - - - - - ### QueryNFTsOfClassResponse -QueryNFTsOfClassResponse is the response type for the Query/NFTsOfClass and Query/NFTsOfClassByOwner RPC methods +QueryNFTsOfClassResponse is the response type for the Query/NFTsOfClass and Query/NFTsOfClassByOwner RPC methods | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `nfts` | [NFT](#cosmos.nft.v1beta1.NFT) | repeated | | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | | - - - - - ### QueryOwnerRequest -QueryOwnerRequest is the request type for the Query/Owner RPC method +QueryOwnerRequest is the request type for the Query/Owner RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `class_id` | [string](#string) | | | | `id` | [string](#string) | | | - - - - - ### QueryOwnerResponse -QueryOwnerResponse is the response type for the Query/Owner RPC method +QueryOwnerResponse is the response type for the Query/Owner RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `owner` | [string](#string) | | | - - - - - ### QuerySupplyRequest -QuerySupplyRequest is the request type for the Query/Supply RPC method +QuerySupplyRequest is the request type for the Query/Supply RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `class_id` | [string](#string) | | | - - - - - ### QuerySupplyResponse -QuerySupplyResponse is the response type for the Query/Supply RPC method +QuerySupplyResponse is the response type for the Query/Supply RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `amount` | [uint64](#uint64) | | | - - - - @@ -8330,6 +6446,7 @@ QuerySupplyResponse is the response type for the Query/Supply RPC method ### Query + Query defines the gRPC querier service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -8351,13 +6468,11 @@ Query defines the gRPC querier service. ## cosmos/nft/v1beta1/tx.proto - - ### MsgSend -MsgSend represents a message to send a nft from one account to another account. +MsgSend represents a message to send a nft from one account to another account. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -8366,14 +6481,10 @@ MsgSend represents a message to send a nft from one account to another account. | `sender` | [string](#string) | | sender is the address of the owner of nft | | `receiver` | [string](#string) | | receiver is the receiver address of nft | - - - - - ### MsgSendResponse + MsgSendResponse defines the Msg/Send response type. @@ -8390,6 +6501,7 @@ MsgSendResponse defines the Msg/Send response type. ### Msg + Msg defines the nft Msg service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -8405,14 +6517,11 @@ Msg defines the nft Msg service. ## cosmos/params/v1beta1/params.proto - - ### ParamChange -ParamChange defines an individual parameter change, for use in -ParameterChangeProposal. +ParamChange defines an individual parameter change, for use in ParameterChangeProposal. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -8420,16 +6529,11 @@ ParameterChangeProposal. | `key` | [string](#string) | | | | `value` | [string](#string) | | | - - - - - ### ParameterChangeProposal -ParameterChangeProposal defines a proposal to change one or more parameters. +ParameterChangeProposal defines a proposal to change one or more parameters. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -8437,10 +6541,6 @@ ParameterChangeProposal defines a proposal to change one or more parameters. | `description` | [string](#string) | | | | `changes` | [ParamChange](#cosmos.params.v1beta1.ParamChange) | repeated | | - - - - @@ -8456,82 +6556,54 @@ ParameterChangeProposal defines a proposal to change one or more parameters. ## cosmos/params/v1beta1/query.proto - - ### QueryParamsRequest -QueryParamsRequest is request type for the Query/Params RPC method. +QueryParamsRequest is request type for the Query/Params RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `subspace` | [string](#string) | | subspace defines the module to query the parameter for. | | `key` | [string](#string) | | key defines the key of the parameter in the subspace. | - - - - - ### QueryParamsResponse -QueryParamsResponse is response type for the Query/Params RPC method. +QueryParamsResponse is response type for the Query/Params RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `param` | [ParamChange](#cosmos.params.v1beta1.ParamChange) | | param defines the queried parameter. | - - - - - ### QuerySubspacesRequest -QuerySubspacesRequest defines a request type for querying for all registered -subspaces and all keys for a subspace. - - - - +QuerySubspacesRequest defines a request type for querying for all registered subspaces and all keys for a subspace. ### QuerySubspacesResponse -QuerySubspacesResponse defines the response types for querying for all -registered subspaces and all keys for a subspace. +QuerySubspacesResponse defines the response types for querying for all registered subspaces and all keys for a subspace. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `subspaces` | [Subspace](#cosmos.params.v1beta1.Subspace) | repeated | | - - - - - ### Subspace -Subspace defines a parameter subspace name and all the keys that exist for -the subspace. +Subspace defines a parameter subspace name and all the keys that exist for the subspace. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `subspace` | [string](#string) | | | | `keys` | [string](#string) | repeated | | - - - - @@ -8542,6 +6614,7 @@ the subspace. ### Query + Query defines the gRPC querier service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -8558,13 +6631,11 @@ Query defines the gRPC querier service. ## cosmos/slashing/v1beta1/slashing.proto - - ### Params -Params represents the parameters used for by the slashing module. +Params represents the parameters used for by the slashing module. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -8574,17 +6645,11 @@ Params represents the parameters used for by the slashing module. | `slash_fraction_double_sign` | [bytes](#bytes) | | | | `slash_fraction_downtime` | [bytes](#bytes) | | | - - - - - ### ValidatorSigningInfo -ValidatorSigningInfo defines a validator's signing info for monitoring their -liveness activity. +ValidatorSigningInfo defines a validator's signing info for monitoring their liveness activity. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -8595,10 +6660,6 @@ liveness activity. | `tombstoned` | [bool](#bool) | | Whether or not a validator has been tombstoned (killed out of validator set). It is set once the validator commits an equivocation or for any other configured misbehiavor. | | `missed_blocks_counter` | [int64](#int64) | | A counter kept to avoid unnecessary array reads. Note that `Sum(MissedBlocksBitArray)` always equals `MissedBlocksCounter`. | - - - - @@ -8614,13 +6675,11 @@ liveness activity. ## cosmos/slashing/v1beta1/genesis.proto - - ### GenesisState -GenesisState defines the slashing module's genesis state. +GenesisState defines the slashing module's genesis state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -8628,59 +6687,39 @@ GenesisState defines the slashing module's genesis state. | `signing_infos` | [SigningInfo](#cosmos.slashing.v1beta1.SigningInfo) | repeated | signing_infos represents a map between validator addresses and their signing infos. | | `missed_blocks` | [ValidatorMissedBlocks](#cosmos.slashing.v1beta1.ValidatorMissedBlocks) | repeated | missed_blocks represents a map between validator addresses and their missed blocks. | - - - - - ### MissedBlock -MissedBlock contains height and missed status as boolean. +MissedBlock contains height and missed status as boolean. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `index` | [int64](#int64) | | index is the height at which the block was missed. | | `missed` | [bool](#bool) | | missed is the missed status. | - - - - - ### SigningInfo -SigningInfo stores validator signing info of corresponding address. +SigningInfo stores validator signing info of corresponding address. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address` | [string](#string) | | address is the validator address. | | `validator_signing_info` | [ValidatorSigningInfo](#cosmos.slashing.v1beta1.ValidatorSigningInfo) | | validator_signing_info represents the signing info of this validator. | - - - - - ### ValidatorMissedBlocks -ValidatorMissedBlocks contains array of missed blocks of corresponding -address. +ValidatorMissedBlocks contains array of missed blocks of corresponding address. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address` | [string](#string) | | address is the validator address. | | `missed_blocks` | [MissedBlock](#cosmos.slashing.v1beta1.MissedBlock) | repeated | missed_blocks is an array of missed blocks by the validator. | - - - - @@ -8696,97 +6735,63 @@ address. ## cosmos/slashing/v1beta1/query.proto - - ### QueryParamsRequest -QueryParamsRequest is the request type for the Query/Params RPC method - - - - +QueryParamsRequest is the request type for the Query/Params RPC method ### QueryParamsResponse -QueryParamsResponse is the response type for the Query/Params RPC method +QueryParamsResponse is the response type for the Query/Params RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `params` | [Params](#cosmos.slashing.v1beta1.Params) | | | - - - - - ### QuerySigningInfoRequest -QuerySigningInfoRequest is the request type for the Query/SigningInfo RPC -method +QuerySigningInfoRequest is the request type for the Query/SigningInfo RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `cons_address` | [string](#string) | | cons_address is the address to query signing info of | - - - - - ### QuerySigningInfoResponse -QuerySigningInfoResponse is the response type for the Query/SigningInfo RPC -method +QuerySigningInfoResponse is the response type for the Query/SigningInfo RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `val_signing_info` | [ValidatorSigningInfo](#cosmos.slashing.v1beta1.ValidatorSigningInfo) | | val_signing_info is the signing info of requested val cons address | - - - - - ### QuerySigningInfosRequest -QuerySigningInfosRequest is the request type for the Query/SigningInfos RPC -method +QuerySigningInfosRequest is the request type for the Query/SigningInfos RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | | - - - - - ### QuerySigningInfosResponse -QuerySigningInfosResponse is the response type for the Query/SigningInfos RPC -method +QuerySigningInfosResponse is the response type for the Query/SigningInfos RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `info` | [ValidatorSigningInfo](#cosmos.slashing.v1beta1.ValidatorSigningInfo) | repeated | info is the signing info of all validators | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | | - - - - @@ -8797,6 +6802,7 @@ method ### Query + Query provides defines the gRPC querier service | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -8814,26 +6820,20 @@ Query provides defines the gRPC querier service ## cosmos/slashing/v1beta1/tx.proto - - ### MsgUnjail -MsgUnjail defines the Msg/Unjail request type +MsgUnjail defines the Msg/Unjail request type | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validator_addr` | [string](#string) | | | - - - - - ### MsgUnjailResponse + MsgUnjailResponse defines the Msg/Unjail response type @@ -8850,6 +6850,7 @@ MsgUnjailResponse defines the Msg/Unjail response type ### Msg + Msg defines the slashing Msg service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -8865,16 +6866,14 @@ Msg defines the slashing Msg service. ## cosmos/staking/v1beta1/authz.proto - - ### StakeAuthorization + StakeAuthorization defines authorization for delegate/undelegate/redelegate. Since: cosmos-sdk 0.43 - | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `max_tokens` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | | max_tokens specifies the maximum amount of tokens can be delegate to a validator. If it is empty, there is no spend limit and any amount of coins can be delegated. | @@ -8882,31 +6881,23 @@ Since: cosmos-sdk 0.43 | `deny_list` | [StakeAuthorization.Validators](#cosmos.staking.v1beta1.StakeAuthorization.Validators) | | deny_list specifies list of validator addresses to whom grantee can not delegate tokens. | | `authorization_type` | [AuthorizationType](#cosmos.staking.v1beta1.AuthorizationType) | | authorization_type defines one of AuthorizationType. | - - - - - ### StakeAuthorization.Validators -Validators defines list of validator addresses. +Validators defines list of validator addresses. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address` | [string](#string) | repeated | | - - - - ### AuthorizationType + AuthorizationType defines the type of staking module authorization type Since: cosmos-sdk 0.43 @@ -8918,7 +6909,6 @@ Since: cosmos-sdk 0.43 | AUTHORIZATION_TYPE_UNDELEGATE | 2 | AUTHORIZATION_TYPE_UNDELEGATE defines an authorization type for Msg/Undelegate | | AUTHORIZATION_TYPE_REDELEGATE | 3 | AUTHORIZATION_TYPE_REDELEGATE defines an authorization type for Msg/BeginRedelegate | - @@ -8932,30 +6922,22 @@ Since: cosmos-sdk 0.43 ## cosmos/staking/v1beta1/staking.proto - - ### Commission -Commission defines commission parameters for a given validator. +Commission defines commission parameters for a given validator. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `commission_rates` | [CommissionRates](#cosmos.staking.v1beta1.CommissionRates) | | commission_rates defines the initial commission rates to be used for creating a validator. | | `update_time` | [google.protobuf.Timestamp](#google.protobuf.Timestamp) | | update_time is the last time the commission rate was changed. | - - - - - ### CommissionRates -CommissionRates defines the initial commission rates to be used for creating -a validator. +CommissionRates defines the initial commission rates to be used for creating a validator. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -8963,52 +6945,35 @@ a validator. | `max_rate` | [string](#string) | | max_rate defines the maximum commission rate which validator can ever charge, as a fraction. | | `max_change_rate` | [string](#string) | | max_change_rate defines the maximum daily increase of the validator commission, as a fraction. | - - - - - ### DVPair -DVPair is struct that just has a delegator-validator pair with no other data. -It is intended to be used as a marshalable pointer. For example, a DVPair can -be used to construct the key to getting an UnbondingDelegation from state. +DVPair is struct that just has a delegator-validator pair with no other data. It is intended to be used as a marshalable +pointer. For example, a DVPair can be used to construct the key to getting an UnbondingDelegation from state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegator_address` | [string](#string) | | | | `validator_address` | [string](#string) | | | - - - - - ### DVPairs -DVPairs defines an array of DVPair objects. +DVPairs defines an array of DVPair objects. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `pairs` | [DVPair](#cosmos.staking.v1beta1.DVPair) | repeated | | - - - - - ### DVVTriplet -DVVTriplet is struct that just has a delegator-validator-validator triplet -with no other data. It is intended to be used as a marshalable pointer. For -example, a DVVTriplet can be used to construct the key to getting a -Redelegation from state. +DVVTriplet is struct that just has a delegator-validator-validator triplet with no other data. It is intended to be used +as a marshalable pointer. For example, a DVVTriplet can be used to construct the key to getting a Redelegation from +state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -9016,33 +6981,22 @@ Redelegation from state. | `validator_src_address` | [string](#string) | | | | `validator_dst_address` | [string](#string) | | | - - - - - ### DVVTriplets -DVVTriplets defines an array of DVVTriplet objects. +DVVTriplets defines an array of DVVTriplet objects. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `triplets` | [DVVTriplet](#cosmos.staking.v1beta1.DVVTriplet) | repeated | | - - - - - ### Delegation -Delegation represents the bond with tokens held by an account. It is -owned by one delegator, and is associated with the voting power of one -validator. +Delegation represents the bond with tokens held by an account. It is owned by one delegator, and is associated with the +voting power of one validator. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -9050,33 +7004,23 @@ validator. | `validator_address` | [string](#string) | | validator_address is the bech32-encoded address of the validator. | | `shares` | [string](#string) | | shares define the delegation shares received. | - - - - - ### DelegationResponse -DelegationResponse is equivalent to Delegation except that it contains a -balance in addition to shares which is more suitable for client responses. +DelegationResponse is equivalent to Delegation except that it contains a balance in addition to shares which is more +suitable for client responses. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegation` | [Delegation](#cosmos.staking.v1beta1.Delegation) | | | | `balance` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | | | - - - - - ### Description -Description defines a validator description. +Description defines a validator description. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -9086,35 +7030,24 @@ Description defines a validator description. | `security_contact` | [string](#string) | | security_contact defines an optional email for security contact. | | `details` | [string](#string) | | details define other optional details. | - - - - - ### HistoricalInfo -HistoricalInfo contains header and validator information for a given block. -It is stored as part of staking module's state, which persists the `n` most -recent HistoricalInfo -(`n` is set by the staking module's `historical_entries` parameter). +HistoricalInfo contains header and validator information for a given block. It is stored as part of staking module's +state, which persists the `n` most recent HistoricalInfo +(`n` is set by the staking module's `historical_entries` parameter). | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `header` | [tendermint.types.Header](#tendermint.types.Header) | | | | `valset` | [Validator](#cosmos.staking.v1beta1.Validator) | repeated | | - - - - - ### Params -Params defines the parameters for the staking module. +Params defines the parameters for the staking module. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -9125,34 +7058,23 @@ Params defines the parameters for the staking module. | `bond_denom` | [string](#string) | | bond_denom defines the bondable coin denomination. | | `min_commission_rate` | [string](#string) | | min_commission_rate is the chain-wide minimum commission rate that a validator can charge their delegators | - - - - - ### Pool -Pool is used for tracking bonded and not-bonded token supply of the bond -denomination. +Pool is used for tracking bonded and not-bonded token supply of the bond denomination. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `not_bonded_tokens` | [string](#string) | | | | `bonded_tokens` | [string](#string) | | | - - - - - ### Redelegation -Redelegation contains the list of a particular delegator's redelegating bonds -from a particular source validator to a particular destination validator. +Redelegation contains the list of a particular delegator's redelegating bonds from a particular source validator to a +particular destination validator. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -9163,16 +7085,11 @@ from a particular source validator to a particular destination validator. redelegation entries | - - - - - ### RedelegationEntry -RedelegationEntry defines a redelegation object with relevant metadata. +RedelegationEntry defines a redelegation object with relevant metadata. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -9181,53 +7098,35 @@ RedelegationEntry defines a redelegation object with relevant metadata. | `initial_balance` | [string](#string) | | initial_balance defines the initial balance when redelegation started. | | `shares_dst` | [string](#string) | | shares_dst is the amount of destination-validator shares created by redelegation. | - - - - - ### RedelegationEntryResponse -RedelegationEntryResponse is equivalent to a RedelegationEntry except that it -contains a balance in addition to shares which is more suitable for client -responses. +RedelegationEntryResponse is equivalent to a RedelegationEntry except that it contains a balance in addition to shares +which is more suitable for client responses. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `redelegation_entry` | [RedelegationEntry](#cosmos.staking.v1beta1.RedelegationEntry) | | | | `balance` | [string](#string) | | | - - - - - ### RedelegationResponse -RedelegationResponse is equivalent to a Redelegation except that its entries -contain a balance in addition to shares which is more suitable for client -responses. +RedelegationResponse is equivalent to a Redelegation except that its entries contain a balance in addition to shares +which is more suitable for client responses. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `redelegation` | [Redelegation](#cosmos.staking.v1beta1.Redelegation) | | | | `entries` | [RedelegationEntryResponse](#cosmos.staking.v1beta1.RedelegationEntryResponse) | repeated | | - - - - - ### UnbondingDelegation -UnbondingDelegation stores all of a single delegator's unbonding bonds -for a single validator in an time-ordered list. +UnbondingDelegation stores all of a single delegator's unbonding bonds for a single validator in an time-ordered list. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -9237,16 +7136,11 @@ for a single validator in an time-ordered list. unbonding delegation entries | - - - - - ### UnbondingDelegationEntry -UnbondingDelegationEntry defines an unbonding object with relevant metadata. +UnbondingDelegationEntry defines an unbonding object with relevant metadata. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -9255,38 +7149,25 @@ UnbondingDelegationEntry defines an unbonding object with relevant metadata. | `initial_balance` | [string](#string) | | initial_balance defines the tokens initially scheduled to receive at completion. | | `balance` | [string](#string) | | balance defines the tokens to receive at completion. | - - - - - ### ValAddresses -ValAddresses defines a repeated set of validator addresses. +ValAddresses defines a repeated set of validator addresses. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `addresses` | [string](#string) | repeated | | - - - - - ### Validator -Validator defines a validator, together with the total amount of the -Validator's bond shares and their exchange rate to coins. Slashing results in -a decrease in the exchange rate, allowing correct calculation of future -undelegations without iterating over delegators. When coins are delegated to -this validator, the validator is credited with a delegation whose number of -bond shares is based on the amount of coins delegated divided by the current -exchange rate. Voting power can be calculated as total bonded shares -multiplied by exchange rate. +Validator defines a validator, together with the total amount of the Validator's bond shares and their exchange rate to +coins. Slashing results in a decrease in the exchange rate, allowing correct calculation of future undelegations without +iterating over delegators. When coins are delegated to this validator, the validator is credited with a delegation whose +number of bond shares is based on the amount of coins delegated divided by the current exchange rate. Voting power can +be calculated as total bonded shares multiplied by exchange rate. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -9302,16 +7183,13 @@ multiplied by exchange rate. | `commission` | [Commission](#cosmos.staking.v1beta1.Commission) | | commission defines the commission parameters. | | `min_self_delegation` | [string](#string) | | min_self_delegation is the validator's self declared minimum self delegation. | - - - - ### BondStatus + BondStatus is the status of a validator. | Name | Number | Description | @@ -9321,7 +7199,6 @@ BondStatus is the status of a validator. | BOND_STATUS_UNBONDING | 2 | UNBONDING defines a validator that is unbonding. | | BOND_STATUS_BONDED | 3 | BONDED defines a validator that is bonded. | - @@ -9335,13 +7212,11 @@ BondStatus is the status of a validator. ## cosmos/staking/v1beta1/genesis.proto - - ### GenesisState -GenesisState defines the staking module's genesis state. +GenesisState defines the staking module's genesis state. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -9354,26 +7229,17 @@ GenesisState defines the staking module's genesis state. | `redelegations` | [Redelegation](#cosmos.staking.v1beta1.Redelegation) | repeated | redelegations defines the redelegations active at genesis. | | `exported` | [bool](#bool) | | | - - - - - ### LastValidatorPower -LastValidatorPower required for validator set update logic. +LastValidatorPower required for validator set update logic. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `address` | [string](#string) | | address is the address of the validator. | | `power` | [int64](#int64) | | power defines the power of the validator. | - - - - @@ -9387,264 +7253,173 @@ LastValidatorPower required for validator set update logic.

Top

-## cosmos/staking/v1beta1/query.proto - - +## cosmos/staking/v1beta1/query.proto ### QueryDelegationRequest -QueryDelegationRequest is request type for the Query/Delegation RPC method. +QueryDelegationRequest is request type for the Query/Delegation RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegator_addr` | [string](#string) | | delegator_addr defines the delegator address to query for. | | `validator_addr` | [string](#string) | | validator_addr defines the validator address to query for. | - - - - - ### QueryDelegationResponse -QueryDelegationResponse is response type for the Query/Delegation RPC method. +QueryDelegationResponse is response type for the Query/Delegation RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegation_response` | [DelegationResponse](#cosmos.staking.v1beta1.DelegationResponse) | | delegation_responses defines the delegation info of a delegation. | - - - - - ### QueryDelegatorDelegationsRequest -QueryDelegatorDelegationsRequest is request type for the -Query/DelegatorDelegations RPC method. +QueryDelegatorDelegationsRequest is request type for the Query/DelegatorDelegations RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegator_addr` | [string](#string) | | delegator_addr defines the delegator address to query for. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryDelegatorDelegationsResponse -QueryDelegatorDelegationsResponse is response type for the -Query/DelegatorDelegations RPC method. +QueryDelegatorDelegationsResponse is response type for the Query/DelegatorDelegations RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegation_responses` | [DelegationResponse](#cosmos.staking.v1beta1.DelegationResponse) | repeated | delegation_responses defines all the delegations' info of a delegator. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryDelegatorUnbondingDelegationsRequest -QueryDelegatorUnbondingDelegationsRequest is request type for the -Query/DelegatorUnbondingDelegations RPC method. +QueryDelegatorUnbondingDelegationsRequest is request type for the Query/DelegatorUnbondingDelegations RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegator_addr` | [string](#string) | | delegator_addr defines the delegator address to query for. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryDelegatorUnbondingDelegationsResponse -QueryUnbondingDelegatorDelegationsResponse is response type for the -Query/UnbondingDelegatorDelegations RPC method. +QueryUnbondingDelegatorDelegationsResponse is response type for the Query/UnbondingDelegatorDelegations RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `unbonding_responses` | [UnbondingDelegation](#cosmos.staking.v1beta1.UnbondingDelegation) | repeated | | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryDelegatorValidatorRequest -QueryDelegatorValidatorRequest is request type for the -Query/DelegatorValidator RPC method. +QueryDelegatorValidatorRequest is request type for the Query/DelegatorValidator RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegator_addr` | [string](#string) | | delegator_addr defines the delegator address to query for. | | `validator_addr` | [string](#string) | | validator_addr defines the validator address to query for. | - - - - - ### QueryDelegatorValidatorResponse -QueryDelegatorValidatorResponse response type for the -Query/DelegatorValidator RPC method. +QueryDelegatorValidatorResponse response type for the Query/DelegatorValidator RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validator` | [Validator](#cosmos.staking.v1beta1.Validator) | | validator defines the the validator info. | - - - - - ### QueryDelegatorValidatorsRequest -QueryDelegatorValidatorsRequest is request type for the -Query/DelegatorValidators RPC method. +QueryDelegatorValidatorsRequest is request type for the Query/DelegatorValidators RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegator_addr` | [string](#string) | | delegator_addr defines the delegator address to query for. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryDelegatorValidatorsResponse -QueryDelegatorValidatorsResponse is response type for the -Query/DelegatorValidators RPC method. +QueryDelegatorValidatorsResponse is response type for the Query/DelegatorValidators RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validators` | [Validator](#cosmos.staking.v1beta1.Validator) | repeated | validators defines the the validators' info of a delegator. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryHistoricalInfoRequest -QueryHistoricalInfoRequest is request type for the Query/HistoricalInfo RPC -method. +QueryHistoricalInfoRequest is request type for the Query/HistoricalInfo RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `height` | [int64](#int64) | | height defines at which height to query the historical info. | - - - - - ### QueryHistoricalInfoResponse -QueryHistoricalInfoResponse is response type for the Query/HistoricalInfo RPC -method. +QueryHistoricalInfoResponse is response type for the Query/HistoricalInfo RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `hist` | [HistoricalInfo](#cosmos.staking.v1beta1.HistoricalInfo) | | hist defines the historical info at the given height. | - - - - - ### QueryParamsRequest -QueryParamsRequest is request type for the Query/Params RPC method. - - - - +QueryParamsRequest is request type for the Query/Params RPC method. ### QueryParamsResponse -QueryParamsResponse is response type for the Query/Params RPC method. +QueryParamsResponse is response type for the Query/Params RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `params` | [Params](#cosmos.staking.v1beta1.Params) | | params holds all the parameters of this module. | - - - - - ### QueryPoolRequest -QueryPoolRequest is request type for the Query/Pool RPC method. - - - - +QueryPoolRequest is request type for the Query/Pool RPC method. ### QueryPoolResponse -QueryPoolResponse is response type for the Query/Pool RPC method. +QueryPoolResponse is response type for the Query/Pool RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `pool` | [Pool](#cosmos.staking.v1beta1.Pool) | | pool defines the pool info. | - - - - - ### QueryRedelegationsRequest -QueryRedelegationsRequest is request type for the Query/Redelegations RPC -method. +QueryRedelegationsRequest is request type for the Query/Redelegations RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -9653,190 +7428,124 @@ method. | `dst_validator_addr` | [string](#string) | | dst_validator_addr defines the validator address to redelegate to. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryRedelegationsResponse -QueryRedelegationsResponse is response type for the Query/Redelegations RPC -method. +QueryRedelegationsResponse is response type for the Query/Redelegations RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `redelegation_responses` | [RedelegationResponse](#cosmos.staking.v1beta1.RedelegationResponse) | repeated | | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryUnbondingDelegationRequest -QueryUnbondingDelegationRequest is request type for the -Query/UnbondingDelegation RPC method. +QueryUnbondingDelegationRequest is request type for the Query/UnbondingDelegation RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegator_addr` | [string](#string) | | delegator_addr defines the delegator address to query for. | | `validator_addr` | [string](#string) | | validator_addr defines the validator address to query for. | - - - - - ### QueryUnbondingDelegationResponse -QueryDelegationResponse is response type for the Query/UnbondingDelegation -RPC method. +QueryDelegationResponse is response type for the Query/UnbondingDelegation RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `unbond` | [UnbondingDelegation](#cosmos.staking.v1beta1.UnbondingDelegation) | | unbond defines the unbonding information of a delegation. | - - - - - ### QueryValidatorDelegationsRequest -QueryValidatorDelegationsRequest is request type for the -Query/ValidatorDelegations RPC method +QueryValidatorDelegationsRequest is request type for the Query/ValidatorDelegations RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validator_addr` | [string](#string) | | validator_addr defines the validator address to query for. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryValidatorDelegationsResponse -QueryValidatorDelegationsResponse is response type for the -Query/ValidatorDelegations RPC method +QueryValidatorDelegationsResponse is response type for the Query/ValidatorDelegations RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `delegation_responses` | [DelegationResponse](#cosmos.staking.v1beta1.DelegationResponse) | repeated | | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryValidatorRequest -QueryValidatorRequest is response type for the Query/Validator RPC method +QueryValidatorRequest is response type for the Query/Validator RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validator_addr` | [string](#string) | | validator_addr defines the validator address to query for. | - - - - - ### QueryValidatorResponse -QueryValidatorResponse is response type for the Query/Validator RPC method +QueryValidatorResponse is response type for the Query/Validator RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validator` | [Validator](#cosmos.staking.v1beta1.Validator) | | validator defines the the validator info. | - - - - - ### QueryValidatorUnbondingDelegationsRequest -QueryValidatorUnbondingDelegationsRequest is required type for the -Query/ValidatorUnbondingDelegations RPC method +QueryValidatorUnbondingDelegationsRequest is required type for the Query/ValidatorUnbondingDelegations RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validator_addr` | [string](#string) | | validator_addr defines the validator address to query for. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryValidatorUnbondingDelegationsResponse -QueryValidatorUnbondingDelegationsResponse is response type for the -Query/ValidatorUnbondingDelegations RPC method. +QueryValidatorUnbondingDelegationsResponse is response type for the Query/ValidatorUnbondingDelegations RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `unbonding_responses` | [UnbondingDelegation](#cosmos.staking.v1beta1.UnbondingDelegation) | repeated | | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - - ### QueryValidatorsRequest -QueryValidatorsRequest is request type for Query/Validators RPC method. +QueryValidatorsRequest is request type for Query/Validators RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `status` | [string](#string) | | status enables to query for validators matching a given status. | | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an optional pagination for the request. | - - - - - ### QueryValidatorsResponse -QueryValidatorsResponse is response type for the Query/Validators RPC method +QueryValidatorsResponse is response type for the Query/Validators RPC method | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `validators` | [Validator](#cosmos.staking.v1beta1.Validator) | repeated | validators contains all the queried validators. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines the pagination in the response. | - - - - @@ -9847,6 +7556,7 @@ QueryValidatorsResponse is response type for the Query/Validators RPC method ### Query + Query defines the gRPC querier service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -9875,14 +7585,12 @@ Query defines the gRPC querier service. ## cosmos/staking/v1beta1/tx.proto - - ### MsgBeginRedelegate -MsgBeginRedelegate defines a SDK message for performing a redelegation -of coins from a delegator and source validator to a destination validator. +MsgBeginRedelegate defines a SDK message for performing a redelegation of coins from a delegator and source validator to +a destination validator. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -9891,31 +7599,21 @@ of coins from a delegator and source validator to a destination validator. | `validator_dst_address` | [string](#string) | | | | `amount` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | | | - - - - - ### MsgBeginRedelegateResponse -MsgBeginRedelegateResponse defines the Msg/BeginRedelegate response type. +MsgBeginRedelegateResponse defines the Msg/BeginRedelegate response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `completion_time` | [google.protobuf.Timestamp](#google.protobuf.Timestamp) | | | - - - - - ### MsgCreateValidator -MsgCreateValidator defines a SDK message for creating a new validator. +MsgCreateValidator defines a SDK message for creating a new validator. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -9927,27 +7625,17 @@ MsgCreateValidator defines a SDK message for creating a new validator. | `pubkey` | [google.protobuf.Any](#google.protobuf.Any) | | | | `value` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | | | - - - - - ### MsgCreateValidatorResponse -MsgCreateValidatorResponse defines the Msg/CreateValidator response type. - - - - +MsgCreateValidatorResponse defines the Msg/CreateValidator response type. ### MsgDelegate -MsgDelegate defines a SDK message for performing a delegation of coins -from a delegator to a validator. +MsgDelegate defines a SDK message for performing a delegation of coins from a delegator to a validator. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -9955,26 +7643,17 @@ from a delegator to a validator. | `validator_address` | [string](#string) | | | | `amount` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | | | - - - - - ### MsgDelegateResponse -MsgDelegateResponse defines the Msg/Delegate response type. - - - - +MsgDelegateResponse defines the Msg/Delegate response type. ### MsgEditValidator -MsgEditValidator defines a SDK message for editing an existing validator. +MsgEditValidator defines a SDK message for editing an existing validator. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -9983,27 +7662,17 @@ MsgEditValidator defines a SDK message for editing an existing validator. | `commission_rate` | [string](#string) | | We pass a reference to the new commission rate and min self delegation as it's not mandatory to update. If not updated, the deserialized rate will be zero with no way to distinguish if an update was intended. REF: #2373 | | `min_self_delegation` | [string](#string) | | | - - - - - ### MsgEditValidatorResponse -MsgEditValidatorResponse defines the Msg/EditValidator response type. - - - - +MsgEditValidatorResponse defines the Msg/EditValidator response type. ### MsgUndelegate -MsgUndelegate defines a SDK message for performing an undelegation from a -delegate and a validator. +MsgUndelegate defines a SDK message for performing an undelegation from a delegate and a validator. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -10011,25 +7680,16 @@ delegate and a validator. | `validator_address` | [string](#string) | | | | `amount` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | | | - - - - - ### MsgUndelegateResponse -MsgUndelegateResponse defines the Msg/Undelegate response type. +MsgUndelegateResponse defines the Msg/Undelegate response type. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `completion_time` | [google.protobuf.Timestamp](#google.protobuf.Timestamp) | | | - - - - @@ -10040,6 +7700,7 @@ MsgUndelegateResponse defines the Msg/Undelegate response type. ### Msg + Msg defines the staking Msg service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -10059,16 +7720,12 @@ Msg defines the staking Msg service. ## cosmos/tx/signing/v1beta1/signing.proto - - ### SignatureDescriptor -SignatureDescriptor is a convenience type which represents the full data for -a signature including the public key of the signer, signing modes and the -signature itself. It is primarily used for coordinating signatures between -clients. +SignatureDescriptor is a convenience type which represents the full data for a signature including the public key of the +signer, signing modes and the signature itself. It is primarily used for coordinating signatures between clients. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -10076,79 +7733,56 @@ clients. | `data` | [SignatureDescriptor.Data](#cosmos.tx.signing.v1beta1.SignatureDescriptor.Data) | | | | `sequence` | [uint64](#uint64) | | sequence is the sequence of the account, which describes the number of committed transactions signed by a given address. It is used to prevent replay attacks. | - - - - - ### SignatureDescriptor.Data -Data represents signature data +Data represents signature data | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `single` | [SignatureDescriptor.Data.Single](#cosmos.tx.signing.v1beta1.SignatureDescriptor.Data.Single) | | single represents a single signer | | `multi` | [SignatureDescriptor.Data.Multi](#cosmos.tx.signing.v1beta1.SignatureDescriptor.Data.Multi) | | multi represents a multisig signer | - - - - - ### SignatureDescriptor.Data.Multi -Multi is the signature data for a multisig public key +Multi is the signature data for a multisig public key | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `bitarray` | [cosmos.crypto.multisig.v1beta1.CompactBitArray](#cosmos.crypto.multisig.v1beta1.CompactBitArray) | | bitarray specifies which keys within the multisig are signing | | `signatures` | [SignatureDescriptor.Data](#cosmos.tx.signing.v1beta1.SignatureDescriptor.Data) | repeated | signatures is the signatures of the multi-signature | - - - - - ### SignatureDescriptor.Data.Single -Single is the signature data for a single signer +Single is the signature data for a single signer | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `mode` | [SignMode](#cosmos.tx.signing.v1beta1.SignMode) | | mode is the signing mode of the single signer | | `signature` | [bytes](#bytes) | | signature is the raw signature bytes | - - - - - ### SignatureDescriptors -SignatureDescriptors wraps multiple SignatureDescriptor's. +SignatureDescriptors wraps multiple SignatureDescriptor's. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `signatures` | [SignatureDescriptor](#cosmos.tx.signing.v1beta1.SignatureDescriptor) | repeated | signatures are the signature descriptors | - - - - ### SignMode + SignMode represents a signing mode with its own security guarantees. | Name | Number | Description | @@ -10159,7 +7793,6 @@ SignMode represents a signing mode with its own security guarantees. | SIGN_MODE_DIRECT_AUX | 3 | SIGN_MODE_DIRECT_AUX specifies a signing mode which uses SignDocDirectAux. As opposed to SIGN_MODE_DIRECT, this sign mode does not require signers signing over other signers' `signer_info`. It also allows for adding Tips in transactions. | | SIGN_MODE_LEGACY_AMINO_JSON | 127 | SIGN_MODE_LEGACY_AMINO_JSON is a backwards compatibility mode which uses Amino JSON and will be removed in the future. | - @@ -10173,14 +7806,11 @@ SignMode represents a signing mode with its own security guarantees. ## cosmos/tx/v1beta1/tx.proto - - ### AuthInfo -AuthInfo describes the fee and signer modes that are used to sign a -transaction. +AuthInfo describes the fee and signer modes that are used to sign a transaction. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -10190,21 +7820,15 @@ transaction. Since: cosmos-sdk 0.45 | - - - - - ### AuxSignerData -AuxSignerData is the intermediary format that an auxiliary signer (e.g. a -tipper) builds and sends to the fee payer (who will build and broadcast the -actual tx). AuxSignerData is not a valid tx in itself, and will be rejected -by the node if sent directly as-is. -Since: cosmos-sdk 0.45 +AuxSignerData is the intermediary format that an auxiliary signer (e.g. a tipper) builds and sends to the fee payer (who +will build and broadcast the actual tx). AuxSignerData is not a valid tx in itself, and will be rejected by the node if +sent directly as-is. +Since: cosmos-sdk 0.45 | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -10213,85 +7837,59 @@ Since: cosmos-sdk 0.45 | `mode` | [cosmos.tx.signing.v1beta1.SignMode](#cosmos.tx.signing.v1beta1.SignMode) | | mode is the signing mode of the single signer | | `sig` | [bytes](#bytes) | | sig is the signature of the sign doc. | - - - - - ### Fee -Fee includes the amount of coins paid in fees and the maximum -gas to be used by the transaction. The ratio yields an effective "gasprice", -which must be above some miminum to be accepted into the mempool. +Fee includes the amount of coins paid in fees and the maximum gas to be used by the transaction. The ratio yields an +effective "gasprice", which must be above some miminum to be accepted into the mempool. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `amount` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | amount is the amount of coins to be paid as a fee | | `gas_limit` | [uint64](#uint64) | | gas_limit is the maximum gas that can be used in transaction processing before an out of gas error occurs | -| `payer` | [string](#string) | | if unset, the first signer is responsible for paying the fees. If set, the specified account must pay the fees. the payer must be a tx signer (and thus have signed this field in AuthInfo). setting this field does *not* change the ordering of required signers for the transaction. | +| `payer` | [string](#string) | | if unset, the first signer is responsible for paying the fees. If set, the specified account must pay the fees. the payer must be a tx signer (and thus have signed this field in AuthInfo). setting this field does * +not* change the ordering of required signers for the transaction. | | `granter` | [string](#string) | | if set, the fee payer (either the first signer or the value of the payer field) requests that a fee grant be used to pay fees instead of the fee payer's own balance. If an appropriate fee grant does not exist or the chain does not support fee grants, this will fail | - - - - - ### ModeInfo -ModeInfo describes the signing mode of a single or nested multisig signer. +ModeInfo describes the signing mode of a single or nested multisig signer. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `single` | [ModeInfo.Single](#cosmos.tx.v1beta1.ModeInfo.Single) | | single represents a single signer | | `multi` | [ModeInfo.Multi](#cosmos.tx.v1beta1.ModeInfo.Multi) | | multi represents a nested multisig signer | - - - - - ### ModeInfo.Multi -Multi is the mode info for a multisig public key +Multi is the mode info for a multisig public key | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `bitarray` | [cosmos.crypto.multisig.v1beta1.CompactBitArray](#cosmos.crypto.multisig.v1beta1.CompactBitArray) | | bitarray specifies which keys within the multisig are signing | | `mode_infos` | [ModeInfo](#cosmos.tx.v1beta1.ModeInfo) | repeated | mode_infos is the corresponding modes of the signers of the multisig which could include nested multisig public keys | - - - - - ### ModeInfo.Single -Single is the mode info for a single signer. It is structured as a message -to allow for additional fields such as locale for SIGN_MODE_TEXTUAL in the -future +Single is the mode info for a single signer. It is structured as a message to allow for additional fields such as locale +for SIGN_MODE_TEXTUAL in the future | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `mode` | [cosmos.tx.signing.v1beta1.SignMode](#cosmos.tx.signing.v1beta1.SignMode) | | mode is the signing mode of the single signer | - - - - - ### SignDoc -SignDoc is the type used for generating sign bytes for SIGN_MODE_DIRECT. +SignDoc is the type used for generating sign bytes for SIGN_MODE_DIRECT. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -10300,17 +7898,11 @@ SignDoc is the type used for generating sign bytes for SIGN_MODE_DIRECT. | `chain_id` | [string](#string) | | chain_id is the unique identifier of the chain this transaction targets. It prevents signed transactions from being used on another chain by an attacker | | `account_number` | [uint64](#uint64) | | account_number is the account number of the account in state | - - - - - ### SignDocDirectAux -SignDocDirectAux is the type used for generating sign bytes for -SIGN_MODE_DIRECT_AUX. +SignDocDirectAux is the type used for generating sign bytes for SIGN_MODE_DIRECT_AUX. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -10321,17 +7913,11 @@ SIGN_MODE_DIRECT_AUX. | `sequence` | [uint64](#uint64) | | sequence is the sequence number of the signing account. | | `tip` | [Tip](#cosmos.tx.v1beta1.Tip) | | Tip is the optional tip used for meta-transactions. It should be left empty if the signer is not the tipper for this transaction. | - - - - - ### SignerInfo -SignerInfo describes the public key and signing mode of a single top-level -signer. +SignerInfo describes the public key and signing mode of a single top-level signer. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -10339,32 +7925,22 @@ signer. | `mode_info` | [ModeInfo](#cosmos.tx.v1beta1.ModeInfo) | | mode_info describes the signing mode of the signer and is a nested structure to support nested multisig pubkey's | | `sequence` | [uint64](#uint64) | | sequence is the sequence of the account, which describes the number of committed transactions signed by a given address. It is used to prevent replay attacks. | - - - - - ### Tip -Tip is the tip used for meta-transactions. +Tip is the tip used for meta-transactions. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `amount` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | amount is the amount of the tip | | `tipper` | [string](#string) | | tipper is the address of the account paying for the tip | - - - - - ### Tx -Tx is the standard type used for broadcasting transactions. +Tx is the standard type used for broadcasting transactions. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -10372,16 +7948,11 @@ Tx is the standard type used for broadcasting transactions. | `auth_info` | [AuthInfo](#cosmos.tx.v1beta1.AuthInfo) | | auth_info is the authorization related content of the transaction, specifically signers, signer modes and fee | | `signatures` | [bytes](#bytes) | repeated | signatures is a list of signatures that matches the length and order of AuthInfo's signer_infos to allow connecting signature meta information like public key and signing mode by position. | - - - - - ### TxBody -TxBody is the body of a transaction that all signers sign over. +TxBody is the body of a transaction that all signers sign over. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -10391,20 +7962,13 @@ TxBody is the body of a transaction that all signers sign over. | `extension_options` | [google.protobuf.Any](#google.protobuf.Any) | repeated | extension_options are arbitrary options that can be added by chains when the default options are not sufficient. If any of these are present and can't be handled, the transaction will be rejected | | `non_critical_extension_options` | [google.protobuf.Any](#google.protobuf.Any) | repeated | extension_options are arbitrary options that can be added by chains when the default options are not sufficient. If any of these are present and can't be handled, they will be ignored | - - - - - ### TxRaw -TxRaw is a variant of Tx that pins the signer's exact binary representation -of body and auth_info. This is used for signing, broadcasting and -verification. The binary `serialize(tx: TxRaw)` is stored in Tendermint and -the hash `sha256(serialize(tx: TxRaw))` becomes the "txhash", commonly used -as the transaction ID. +TxRaw is a variant of Tx that pins the signer's exact binary representation of body and auth_info. This is used for +signing, broadcasting and verification. The binary `serialize(tx: TxRaw)` is stored in Tendermint and the +hash `sha256(serialize(tx: TxRaw))` becomes the "txhash", commonly used as the transaction ID. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -10412,10 +7976,6 @@ as the transaction ID. | `auth_info_bytes` | [bytes](#bytes) | | auth_info_bytes is a protobuf serialization of an AuthInfo that matches the representation in SignDoc. | | `signatures` | [bytes](#bytes) | repeated | signatures is a list of signatures that matches the length and order of AuthInfo's signer_infos to allow connecting signature meta information like public key and signing mode by position. | - - - - @@ -10431,79 +7991,53 @@ as the transaction ID. ## cosmos/tx/v1beta1/service.proto - - ### BroadcastTxRequest -BroadcastTxRequest is the request type for the Service.BroadcastTxRequest -RPC method. +BroadcastTxRequest is the request type for the Service.BroadcastTxRequest RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `tx_bytes` | [bytes](#bytes) | | tx_bytes is the raw transaction. | | `mode` | [BroadcastMode](#cosmos.tx.v1beta1.BroadcastMode) | | | - - - - - ### BroadcastTxResponse -BroadcastTxResponse is the response type for the -Service.BroadcastTx method. +BroadcastTxResponse is the response type for the Service.BroadcastTx method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `tx_response` | [cosmos.base.abci.v1beta1.TxResponse](#cosmos.base.abci.v1beta1.TxResponse) | | tx_response is the queried TxResponses. | - - - - - ### GetTxRequest -GetTxRequest is the request type for the Service.GetTx -RPC method. +GetTxRequest is the request type for the Service.GetTx RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `hash` | [string](#string) | | hash is the tx hash to query, encoded as a hex string. | - - - - - ### GetTxResponse -GetTxResponse is the response type for the Service.GetTx method. +GetTxResponse is the response type for the Service.GetTx method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `tx` | [Tx](#cosmos.tx.v1beta1.Tx) | | tx is the queried transaction. | | `tx_response` | [cosmos.base.abci.v1beta1.TxResponse](#cosmos.base.abci.v1beta1.TxResponse) | | tx_response is the queried TxResponses. | - - - - - ### GetTxsEventRequest -GetTxsEventRequest is the request type for the Service.TxsByEvents -RPC method. +GetTxsEventRequest is the request type for the Service.TxsByEvents RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -10511,17 +8045,11 @@ RPC method. | `pagination` | [cosmos.base.query.v1beta1.PageRequest](#cosmos.base.query.v1beta1.PageRequest) | | pagination defines an pagination for the request. | | `order_by` | [OrderBy](#cosmos.tx.v1beta1.OrderBy) | | | - - - - - ### GetTxsEventResponse -GetTxsEventResponse is the response type for the Service.TxsByEvents -RPC method. +GetTxsEventResponse is the response type for the Service.TxsByEvents RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -10529,52 +8057,38 @@ RPC method. | `tx_responses` | [cosmos.base.abci.v1beta1.TxResponse](#cosmos.base.abci.v1beta1.TxResponse) | repeated | tx_responses is the list of queried TxResponses. | | `pagination` | [cosmos.base.query.v1beta1.PageResponse](#cosmos.base.query.v1beta1.PageResponse) | | pagination defines an pagination for the response. | - - - - - ### SimulateRequest -SimulateRequest is the request type for the Service.Simulate -RPC method. +SimulateRequest is the request type for the Service.Simulate RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | -| `tx` | [Tx](#cosmos.tx.v1beta1.Tx) | | **Deprecated.** tx is the transaction to simulate. Deprecated. Send raw tx bytes instead. | +| `tx` | [Tx](#cosmos.tx.v1beta1.Tx) | | ** +Deprecated.** tx is the transaction to simulate. Deprecated. Send raw tx bytes instead. | | `tx_bytes` | [bytes](#bytes) | | tx_bytes is the raw transaction. Since: cosmos-sdk 0.43 | - - - - - ### SimulateResponse -SimulateResponse is the response type for the -Service.SimulateRPC method. +SimulateResponse is the response type for the Service.SimulateRPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `gas_info` | [cosmos.base.abci.v1beta1.GasInfo](#cosmos.base.abci.v1beta1.GasInfo) | | gas_info is the information about gas used in the simulation. | | `result` | [cosmos.base.abci.v1beta1.Result](#cosmos.base.abci.v1beta1.Result) | | result is the result of the simulation. | - - - - ### BroadcastMode + BroadcastMode specifies the broadcast mode for the TxService.Broadcast RPC method. | Name | Number | Description | @@ -10584,11 +8098,10 @@ BroadcastMode specifies the broadcast mode for the TxService.Broadcast RPC metho | BROADCAST_MODE_SYNC | 2 | BROADCAST_MODE_SYNC defines a tx broadcasting mode where the client waits for a CheckTx execution response only. | | BROADCAST_MODE_ASYNC | 3 | BROADCAST_MODE_ASYNC defines a tx broadcasting mode where the client returns immediately. | - - ### OrderBy + OrderBy defines the sorting order | Name | Number | Description | @@ -10597,7 +8110,6 @@ OrderBy defines the sorting order | ORDER_BY_ASC | 1 | ORDER_BY_ASC defines ascending order | | ORDER_BY_DESC | 2 | ORDER_BY_DESC defines descending order | - @@ -10606,6 +8118,7 @@ OrderBy defines the sorting order ### Service + Service defines a gRPC service for interacting with transactions. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -10624,68 +8137,51 @@ Service defines a gRPC service for interacting with transactions. ## cosmos/upgrade/v1beta1/upgrade.proto - - ### CancelSoftwareUpgradeProposal -CancelSoftwareUpgradeProposal is a gov Content type for cancelling a software -upgrade. +CancelSoftwareUpgradeProposal is a gov Content type for cancelling a software upgrade. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `title` | [string](#string) | | | | `description` | [string](#string) | | | - - - - - ### ModuleVersion + ModuleVersion specifies a module and its consensus version. Since: cosmos-sdk 0.43 - | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `name` | [string](#string) | | name of the app module | | `version` | [uint64](#uint64) | | consensus version of the app module | - - - - - ### Plan -Plan specifies information about a planned upgrade and when it should occur. +Plan specifies information about a planned upgrade and when it should occur. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `name` | [string](#string) | | Sets the name for the upgrade. This name will be used by the upgraded version of the software to apply any special "on-upgrade" commands during the first BeginBlock method after the upgrade is applied. It is also used to detect whether a software version can handle a given upgrade. If no upgrade handler with this name has been set in the software, it will be assumed that the software is out-of-date when the upgrade Time or Height is reached and the software will exit. | -| `time` | [google.protobuf.Timestamp](#google.protobuf.Timestamp) | | **Deprecated.** Deprecated: Time based upgrades have been deprecated. Time based upgrade logic has been removed from the SDK. If this field is not empty, an error will be thrown. | +| `time` | [google.protobuf.Timestamp](#google.protobuf.Timestamp) | | ** +Deprecated.** Deprecated: Time based upgrades have been deprecated. Time based upgrade logic has been removed from the SDK. If this field is not empty, an error will be thrown. | | `height` | [int64](#int64) | | The height at which the upgrade must be performed. Only used if Time is not set. | | `info` | [string](#string) | | Any application specific upgrade info to be included on-chain such as a git commit that validators could automatically upgrade to | -| `upgraded_client_state` | [google.protobuf.Any](#google.protobuf.Any) | | **Deprecated.** Deprecated: UpgradedClientState field has been deprecated. IBC upgrade logic has been moved to the IBC module in the sub module 02-client. If this field is not empty, an error will be thrown. | - - - - - +| `upgraded_client_state` | [google.protobuf.Any](#google.protobuf.Any) | | ** +Deprecated.** Deprecated: UpgradedClientState field has been deprecated. IBC upgrade logic has been moved to the IBC module in the sub module 02-client. If this field is not empty, an error will be thrown. | ### SoftwareUpgradeProposal -SoftwareUpgradeProposal is a gov Content type for initiating a software -upgrade. +SoftwareUpgradeProposal is a gov Content type for initiating a software upgrade. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -10693,10 +8189,6 @@ upgrade. | `description` | [string](#string) | | | | `plan` | [Plan](#cosmos.upgrade.v1beta1.Plan) | | | - - - - @@ -10712,134 +8204,86 @@ upgrade. ## cosmos/upgrade/v1beta1/query.proto - - ### QueryAppliedPlanRequest -QueryCurrentPlanRequest is the request type for the Query/AppliedPlan RPC -method. +QueryCurrentPlanRequest is the request type for the Query/AppliedPlan RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `name` | [string](#string) | | name is the name of the applied plan to query for. | - - - - - ### QueryAppliedPlanResponse -QueryAppliedPlanResponse is the response type for the Query/AppliedPlan RPC -method. +QueryAppliedPlanResponse is the response type for the Query/AppliedPlan RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `height` | [int64](#int64) | | height is the block height at which the plan was applied. | - - - - - ### QueryCurrentPlanRequest -QueryCurrentPlanRequest is the request type for the Query/CurrentPlan RPC -method. - - - - +QueryCurrentPlanRequest is the request type for the Query/CurrentPlan RPC method. ### QueryCurrentPlanResponse -QueryCurrentPlanResponse is the response type for the Query/CurrentPlan RPC -method. +QueryCurrentPlanResponse is the response type for the Query/CurrentPlan RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `plan` | [Plan](#cosmos.upgrade.v1beta1.Plan) | | plan is the current upgrade plan. | - - - - - ### QueryModuleVersionsRequest -QueryModuleVersionsRequest is the request type for the Query/ModuleVersions -RPC method. -Since: cosmos-sdk 0.43 +QueryModuleVersionsRequest is the request type for the Query/ModuleVersions RPC method. +Since: cosmos-sdk 0.43 | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `module_name` | [string](#string) | | module_name is a field to query a specific module consensus version from state. Leaving this empty will fetch the full list of module versions from state | - - - - - ### QueryModuleVersionsResponse -QueryModuleVersionsResponse is the response type for the Query/ModuleVersions -RPC method. -Since: cosmos-sdk 0.43 +QueryModuleVersionsResponse is the response type for the Query/ModuleVersions RPC method. +Since: cosmos-sdk 0.43 | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `module_versions` | [ModuleVersion](#cosmos.upgrade.v1beta1.ModuleVersion) | repeated | module_versions is a list of module names with their consensus versions. | - - - - - ### QueryUpgradedConsensusStateRequest -QueryUpgradedConsensusStateRequest is the request type for the Query/UpgradedConsensusState -RPC method. +QueryUpgradedConsensusStateRequest is the request type for the Query/UpgradedConsensusState RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `last_height` | [int64](#int64) | | last height of the current chain must be sent in request as this is the height under which next consensus state is stored | - - - - - ### QueryUpgradedConsensusStateResponse -QueryUpgradedConsensusStateResponse is the response type for the Query/UpgradedConsensusState -RPC method. +QueryUpgradedConsensusStateResponse is the response type for the Query/UpgradedConsensusState RPC method. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `upgraded_consensus_state` | [bytes](#bytes) | | Since: cosmos-sdk 0.43 | - - - - @@ -10850,6 +8294,7 @@ RPC method. ### Query + Query defines the gRPC upgrade querier service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -10870,14 +8315,12 @@ Since: cosmos-sdk 0.43 | GET|/cosmos/upgrade/v1beta1/module_versions| ## cosmos/vesting/v1beta1/vesting.proto - - ### BaseVestingAccount -BaseVestingAccount implements the VestingAccount interface. It contains all -the necessary fields needed for any vesting account implementation. +BaseVestingAccount implements the VestingAccount interface. It contains all the necessary fields needed for any vesting +account implementation. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -10887,67 +8330,46 @@ the necessary fields needed for any vesting account implementation. | `delegated_vesting` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | | | `end_time` | [int64](#int64) | | | - - - - - ### ContinuousVestingAccount -ContinuousVestingAccount implements the VestingAccount interface. It -continuously vests by unlocking coins linearly with respect to time. +ContinuousVestingAccount implements the VestingAccount interface. It continuously vests by unlocking coins linearly with +respect to time. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `base_vesting_account` | [BaseVestingAccount](#cosmos.vesting.v1beta1.BaseVestingAccount) | | | | `start_time` | [int64](#int64) | | | - - - - - ### DelayedVestingAccount -DelayedVestingAccount implements the VestingAccount interface. It vests all -coins after a specific time, but non prior. In other words, it keeps them -locked until a specified time. +DelayedVestingAccount implements the VestingAccount interface. It vests all coins after a specific time, but non prior. +In other words, it keeps them locked until a specified time. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `base_vesting_account` | [BaseVestingAccount](#cosmos.vesting.v1beta1.BaseVestingAccount) | | | - - - - - ### Period -Period defines a length of time and amount of coins that will vest. +Period defines a length of time and amount of coins that will vest. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `length` | [int64](#int64) | | | | `amount` | [cosmos.base.v1beta1.Coin](#cosmos.base.v1beta1.Coin) | repeated | | - - - - - ### PeriodicVestingAccount -PeriodicVestingAccount implements the VestingAccount interface. It -periodically vests by unlocking coins during each specified period. +PeriodicVestingAccount implements the VestingAccount interface. It periodically vests by unlocking coins during each +specified period. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -10955,29 +8377,19 @@ periodically vests by unlocking coins during each specified period. | `start_time` | [int64](#int64) | | | | `vesting_periods` | [Period](#cosmos.vesting.v1beta1.Period) | repeated | | - - - - - ### PermanentLockedAccount -PermanentLockedAccount implements the VestingAccount interface. It does -not ever release coins, locking them indefinitely. Coins in this account can -still be used for delegating and for governance votes even while locked. -Since: cosmos-sdk 0.43 +PermanentLockedAccount implements the VestingAccount interface. It does not ever release coins, locking them +indefinitely. Coins in this account can still be used for delegating and for governance votes even while locked. +Since: cosmos-sdk 0.43 | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | | `base_vesting_account` | [BaseVestingAccount](#cosmos.vesting.v1beta1.BaseVestingAccount) | | | - - - - @@ -10993,14 +8405,11 @@ Since: cosmos-sdk 0.43 ## cosmos/vesting/v1beta1/tx.proto - - ### MsgCreatePeriodicVestingAccount -MsgCreateVestingAccount defines a message that enables creating a vesting -account. +MsgCreateVestingAccount defines a message that enables creating a vesting account. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -11009,28 +8418,17 @@ account. | `start_time` | [int64](#int64) | | | | `vesting_periods` | [Period](#cosmos.vesting.v1beta1.Period) | repeated | | - - - - - ### MsgCreatePeriodicVestingAccountResponse -MsgCreateVestingAccountResponse defines the Msg/CreatePeriodicVestingAccount -response type. - - - - +MsgCreateVestingAccountResponse defines the Msg/CreatePeriodicVestingAccount response type. ### MsgCreateVestingAccount -MsgCreateVestingAccount defines a message that enables creating a vesting -account. +MsgCreateVestingAccount defines a message that enables creating a vesting account. | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | @@ -11040,14 +8438,10 @@ account. | `end_time` | [int64](#int64) | | | | `delayed` | [bool](#bool) | | | - - - - - ### MsgCreateVestingAccountResponse + MsgCreateVestingAccountResponse defines the Msg/CreateVestingAccount response type. @@ -11064,6 +8458,7 @@ MsgCreateVestingAccountResponse defines the Msg/CreateVestingAccount response ty ### Msg + Msg defines the bank Msg service. | Method Name | Request Type | Response Type | Description | HTTP Verb | Endpoint | @@ -11073,25 +8468,29 @@ Msg defines the bank Msg service. - - ## Scalar Value Types -| .proto Type | Notes | C++ | Java | Python | Go | C# | PHP | Ruby | -| ----------- | ----- | --- | ---- | ------ | -- | -- | --- | ---- | -| double | | double | double | float | float64 | double | float | Float | -| float | | float | float | float | float32 | float | float | Float | -| int32 | Uses variable-length encoding. Inefficient for encoding negative numbers – if your field is likely to have negative values, use sint32 instead. | int32 | int | int | int32 | int | integer | Bignum or Fixnum (as required) | -| int64 | Uses variable-length encoding. Inefficient for encoding negative numbers – if your field is likely to have negative values, use sint64 instead. | int64 | long | int/long | int64 | long | integer/string | Bignum | -| uint32 | Uses variable-length encoding. | uint32 | int | int/long | uint32 | uint | integer | Bignum or Fixnum (as required) | -| uint64 | Uses variable-length encoding. | uint64 | long | int/long | uint64 | ulong | integer/string | Bignum or Fixnum (as required) | -| sint32 | Uses variable-length encoding. Signed int value. These more efficiently encode negative numbers than regular int32s. | int32 | int | int | int32 | int | integer | Bignum or Fixnum (as required) | -| sint64 | Uses variable-length encoding. Signed int value. These more efficiently encode negative numbers than regular int64s. | int64 | long | int/long | int64 | long | integer/string | Bignum | -| fixed32 | Always four bytes. More efficient than uint32 if values are often greater than 2^28. | uint32 | int | int | uint32 | uint | integer | Bignum or Fixnum (as required) | -| fixed64 | Always eight bytes. More efficient than uint64 if values are often greater than 2^56. | uint64 | long | int/long | uint64 | ulong | integer/string | Bignum | -| sfixed32 | Always four bytes. | int32 | int | int | int32 | int | integer | Bignum or Fixnum (as required) | -| sfixed64 | Always eight bytes. | int64 | long | int/long | int64 | long | integer/string | Bignum | -| bool | | bool | boolean | boolean | bool | bool | boolean | TrueClass/FalseClass | -| string | A string must always contain UTF-8 encoded or 7-bit ASCII text. | string | String | str/unicode | string | string | string | String (UTF-8) | -| bytes | May contain any arbitrary sequence of bytes. | string | ByteString | str | []byte | ByteString | string | String (ASCII-8BIT) | +| .proto Type | Notes | C++ | Java | Python | Go | C# | PHP | Ruby | | ----------- | ----- | --- | ---- | ------ | -- | +-- | --- | ---- | | double | | double | double | float | float64 | double | float | Float | +| float | | float | float | float | float32 | float | float | Float | | int32 | +Uses variable-length encoding. Inefficient for encoding negative numbers – if your field is likely to have negative +values, use sint32 instead. | int32 | int | int | int32 | int | integer | Bignum or Fixnum (as required) | +| int64 | Uses variable-length encoding. Inefficient for encoding negative numbers – if your field is +likely to have negative values, use sint64 instead. | int64 | long | int/long | int64 | long | integer/string | Bignum | +| uint32 | Uses variable-length encoding. | uint32 | int | int/long | uint32 | uint | integer | +Bignum or Fixnum (as required) | | uint64 | Uses variable-length encoding. | uint64 | long | +int/long | uint64 | ulong | integer/string | Bignum or Fixnum (as required) | | sint32 | Uses +variable-length encoding. Signed int value. These more efficiently encode negative numbers than regular int32s. | int32 +| int | int | int32 | int | integer | Bignum or Fixnum (as required) | | sint64 | Uses +variable-length encoding. Signed int value. These more efficiently encode negative numbers than regular int64s. | int64 +| long | int/long | int64 | long | integer/string | Bignum | | fixed32 | Always four bytes. More +efficient than uint32 if values are often greater than 2^28. | uint32 | int | int | uint32 | uint | integer | Bignum or +Fixnum (as required) | | fixed64 | Always eight bytes. More efficient than uint64 if values are +often greater than 2^56. | uint64 | long | int/long | uint64 | ulong | integer/string | Bignum | | +sfixed32 | Always four bytes. | int32 | int | int | int32 | int | integer | Bignum or Fixnum (as required) | +| sfixed64 | Always eight bytes. | int64 | long | int/long | int64 | long | integer/string | +Bignum | | bool | | bool | boolean | boolean | bool | bool | boolean | TrueClass/FalseClass | +| string | A string must always contain UTF-8 encoded or 7-bit ASCII text. | string | String | +str/unicode | string | string | string | String (UTF-8) | | bytes | May contain any arbitrary +sequence of bytes. | string | ByteString | str | []byte | ByteString | string | String (ASCII-8BIT) | diff --git a/proto/cosmos/base/store/v1beta1/snapshot.proto b/proto/cosmos/base/store/v1beta1/snapshot.proto index 834855093b24..de5dee678f7b 100644 --- a/proto/cosmos/base/store/v1beta1/snapshot.proto +++ b/proto/cosmos/base/store/v1beta1/snapshot.proto @@ -10,7 +10,9 @@ message SnapshotItem { // item is the specific type of snapshot item. oneof item { SnapshotStoreItem store = 1; - SnapshotIAVLItem iavl = 2 [(gogoproto.customname) = "IAVL"]; + SnapshotIAVLItem iavl = 2 [(gogoproto.customname) = "IAVL"]; + SnapshotKVItem kv = 3 [(gogoproto.customname) = "KV"]; + SnapshotSchema schema = 4; } } @@ -21,8 +23,20 @@ message SnapshotStoreItem { // SnapshotIAVLItem is an exported IAVL node. message SnapshotIAVLItem { - bytes key = 1; - bytes value = 2; + bytes key = 1; + bytes value = 2; int64 version = 3; - int32 height = 4; + int32 height = 4; +} + +// SnapshotKVItem is an exported Key/Value Pair +message SnapshotKVItem { + bytes key = 1; + bytes value = 2; +} + +// SnapshotSchema is an exported schema of store +message SnapshotSchema{ + bytes key = 1; + bytes type = 2; } \ No newline at end of file diff --git a/store/types/snapshot.pb.go b/store/types/snapshot.pb.go index 6f001da1bf28..9a64862de162 100644 --- a/store/types/snapshot.pb.go +++ b/store/types/snapshot.pb.go @@ -30,6 +30,8 @@ type SnapshotItem struct { // Types that are valid to be assigned to Item: // *SnapshotItem_Store // *SnapshotItem_IAVL + // *SnapshotItem_KV + // *SnapshotItem_Schema Item isSnapshotItem_Item `protobuf_oneof:"item"` } @@ -78,9 +80,17 @@ type SnapshotItem_Store struct { type SnapshotItem_IAVL struct { IAVL *SnapshotIAVLItem `protobuf:"bytes,2,opt,name=iavl,proto3,oneof" json:"iavl,omitempty"` } +type SnapshotItem_KV struct { + KV *SnapshotKVItem `protobuf:"bytes,3,opt,name=kv,proto3,oneof" json:"kv,omitempty"` +} +type SnapshotItem_Schema struct { + Schema *SnapshotSchema `protobuf:"bytes,4,opt,name=schema,proto3,oneof" json:"schema,omitempty"` +} -func (*SnapshotItem_Store) isSnapshotItem_Item() {} -func (*SnapshotItem_IAVL) isSnapshotItem_Item() {} +func (*SnapshotItem_Store) isSnapshotItem_Item() {} +func (*SnapshotItem_IAVL) isSnapshotItem_Item() {} +func (*SnapshotItem_KV) isSnapshotItem_Item() {} +func (*SnapshotItem_Schema) isSnapshotItem_Item() {} func (m *SnapshotItem) GetItem() isSnapshotItem_Item { if m != nil { @@ -103,11 +113,27 @@ func (m *SnapshotItem) GetIAVL() *SnapshotIAVLItem { return nil } +func (m *SnapshotItem) GetKV() *SnapshotKVItem { + if x, ok := m.GetItem().(*SnapshotItem_KV); ok { + return x.KV + } + return nil +} + +func (m *SnapshotItem) GetSchema() *SnapshotSchema { + if x, ok := m.GetItem().(*SnapshotItem_Schema); ok { + return x.Schema + } + return nil +} + // XXX_OneofWrappers is for the internal use of the proto package. func (*SnapshotItem) XXX_OneofWrappers() []interface{} { return []interface{}{ (*SnapshotItem_Store)(nil), (*SnapshotItem_IAVL)(nil), + (*SnapshotItem_KV)(nil), + (*SnapshotItem_Schema)(nil), } } @@ -225,10 +251,118 @@ func (m *SnapshotIAVLItem) GetHeight() int32 { return 0 } +// SnapshotKVItem is an exported Key/Value Pair +type SnapshotKVItem struct { + Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + Value []byte `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` +} + +func (m *SnapshotKVItem) Reset() { *m = SnapshotKVItem{} } +func (m *SnapshotKVItem) String() string { return proto.CompactTextString(m) } +func (*SnapshotKVItem) ProtoMessage() {} +func (*SnapshotKVItem) Descriptor() ([]byte, []int) { + return fileDescriptor_9c55879db4cc4502, []int{3} +} +func (m *SnapshotKVItem) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SnapshotKVItem) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SnapshotKVItem.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SnapshotKVItem) XXX_Merge(src proto.Message) { + xxx_messageInfo_SnapshotKVItem.Merge(m, src) +} +func (m *SnapshotKVItem) XXX_Size() int { + return m.Size() +} +func (m *SnapshotKVItem) XXX_DiscardUnknown() { + xxx_messageInfo_SnapshotKVItem.DiscardUnknown(m) +} + +var xxx_messageInfo_SnapshotKVItem proto.InternalMessageInfo + +func (m *SnapshotKVItem) GetKey() []byte { + if m != nil { + return m.Key + } + return nil +} + +func (m *SnapshotKVItem) GetValue() []byte { + if m != nil { + return m.Value + } + return nil +} + +// SnapshotSchema is an exported schema of store +type SnapshotSchema struct { + Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + Type []byte `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` +} + +func (m *SnapshotSchema) Reset() { *m = SnapshotSchema{} } +func (m *SnapshotSchema) String() string { return proto.CompactTextString(m) } +func (*SnapshotSchema) ProtoMessage() {} +func (*SnapshotSchema) Descriptor() ([]byte, []int) { + return fileDescriptor_9c55879db4cc4502, []int{4} +} +func (m *SnapshotSchema) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SnapshotSchema) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_SnapshotSchema.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *SnapshotSchema) XXX_Merge(src proto.Message) { + xxx_messageInfo_SnapshotSchema.Merge(m, src) +} +func (m *SnapshotSchema) XXX_Size() int { + return m.Size() +} +func (m *SnapshotSchema) XXX_DiscardUnknown() { + xxx_messageInfo_SnapshotSchema.DiscardUnknown(m) +} + +var xxx_messageInfo_SnapshotSchema proto.InternalMessageInfo + +func (m *SnapshotSchema) GetKey() []byte { + if m != nil { + return m.Key + } + return nil +} + +func (m *SnapshotSchema) GetType() []byte { + if m != nil { + return m.Type + } + return nil +} + func init() { proto.RegisterType((*SnapshotItem)(nil), "cosmos.base.store.v1beta1.SnapshotItem") proto.RegisterType((*SnapshotStoreItem)(nil), "cosmos.base.store.v1beta1.SnapshotStoreItem") proto.RegisterType((*SnapshotIAVLItem)(nil), "cosmos.base.store.v1beta1.SnapshotIAVLItem") + proto.RegisterType((*SnapshotKVItem)(nil), "cosmos.base.store.v1beta1.SnapshotKVItem") + proto.RegisterType((*SnapshotSchema)(nil), "cosmos.base.store.v1beta1.SnapshotSchema") } func init() { @@ -236,28 +370,32 @@ func init() { } var fileDescriptor_9c55879db4cc4502 = []byte{ - // 324 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x91, 0xc1, 0x4a, 0xc3, 0x30, - 0x18, 0xc7, 0x1b, 0xd7, 0x4d, 0xfd, 0xdc, 0x61, 0x86, 0x21, 0xd5, 0x43, 0x1d, 0xbb, 0x58, 0x50, - 0x13, 0xa6, 0x4f, 0x60, 0xf1, 0xb0, 0xa1, 0xa7, 0x0c, 0x3c, 0x78, 0x4b, 0x67, 0x68, 0xcb, 0xd6, - 0x65, 0x2c, 0x59, 0x61, 0x6f, 0xe1, 0x6b, 0xf8, 0x26, 0x1e, 0x77, 0xf4, 0x24, 0xd2, 0xbd, 0x88, - 0x24, 0xe9, 0x2e, 0x8a, 0xe0, 0xa9, 0xdf, 0xbf, 0xfc, 0xfe, 0xbf, 0x7c, 0xf0, 0x41, 0x34, 0x91, - 0xaa, 0x90, 0x8a, 0x26, 0x5c, 0x09, 0xaa, 0xb4, 0x5c, 0x0a, 0x5a, 0x0e, 0x12, 0xa1, 0xf9, 0x80, - 0xaa, 0x39, 0x5f, 0xa8, 0x4c, 0x6a, 0xb2, 0x58, 0x4a, 0x2d, 0xf1, 0xa9, 0x23, 0x89, 0x21, 0x89, - 0x25, 0x49, 0x4d, 0x9e, 0x75, 0x53, 0x99, 0x4a, 0x4b, 0x51, 0x33, 0xb9, 0x42, 0xff, 0x0d, 0x41, - 0x7b, 0x5c, 0x3b, 0x46, 0x5a, 0x14, 0xf8, 0x1e, 0x9a, 0xb6, 0x17, 0xa0, 0x1e, 0x8a, 0x8e, 0x6e, - 0xae, 0xc8, 0x9f, 0x46, 0xb2, 0xeb, 0x8d, 0xcd, 0x5f, 0x53, 0x1e, 0x7a, 0xcc, 0x95, 0xf1, 0x03, - 0xf8, 0x39, 0x2f, 0x67, 0xc1, 0x9e, 0x95, 0x5c, 0xfe, 0x43, 0x32, 0xba, 0x7b, 0x7a, 0x34, 0x8e, - 0xf8, 0xa0, 0xfa, 0x3c, 0xf7, 0x4d, 0x1a, 0x7a, 0xcc, 0x4a, 0xe2, 0x16, 0xf8, 0xb9, 0x16, 0x45, - 0xff, 0x02, 0x8e, 0x7f, 0x3d, 0x89, 0x31, 0xf8, 0x73, 0x5e, 0xb8, 0x75, 0x0f, 0x99, 0x9d, 0xfb, - 0x33, 0xe8, 0xfc, 0xd4, 0xe2, 0x0e, 0x34, 0xa6, 0x62, 0x6d, 0xb1, 0x36, 0x33, 0x23, 0xee, 0x42, - 0xb3, 0xe4, 0xb3, 0x95, 0xb0, 0x4b, 0xb6, 0x99, 0x0b, 0x38, 0x80, 0xfd, 0x52, 0x2c, 0x55, 0x2e, - 0xe7, 0x41, 0xa3, 0x87, 0xa2, 0x06, 0xdb, 0x45, 0x7c, 0x02, 0xad, 0x4c, 0xe4, 0x69, 0xa6, 0x03, - 0xbf, 0x87, 0xa2, 0x26, 0xab, 0x53, 0x1c, 0xbf, 0x57, 0x21, 0xda, 0x54, 0x21, 0xfa, 0xaa, 0x42, - 0xf4, 0xba, 0x0d, 0xbd, 0xcd, 0x36, 0xf4, 0x3e, 0xb6, 0xa1, 0xf7, 0x1c, 0xa5, 0xb9, 0xce, 0x56, - 0x09, 0x99, 0xc8, 0x82, 0xd6, 0x27, 0x74, 0x9f, 0x6b, 0xf5, 0x32, 0xad, 0x0f, 0xa9, 0xd7, 0x0b, - 0xa1, 0x92, 0x96, 0xbd, 0xc6, 0xed, 0x77, 0x00, 0x00, 0x00, 0xff, 0xff, 0x75, 0x87, 0x24, 0x7b, - 0xea, 0x01, 0x00, 0x00, + // 391 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x92, 0xcf, 0x6a, 0xab, 0x40, + 0x14, 0x87, 0xd5, 0x18, 0xef, 0xbd, 0xe7, 0x86, 0x4b, 0xee, 0x10, 0x2e, 0xde, 0x2e, 0x4c, 0x71, + 0xd3, 0x94, 0xb6, 0x4a, 0x5a, 0x28, 0xdd, 0xd6, 0x74, 0x61, 0x48, 0x57, 0x13, 0xc8, 0xa2, 0x3b, + 0x4d, 0x07, 0x15, 0x63, 0x26, 0x64, 0x26, 0x42, 0xde, 0xa2, 0xef, 0xd2, 0x97, 0xe8, 0x32, 0xcb, + 0xae, 0x42, 0x31, 0x2f, 0x52, 0x66, 0x34, 0x94, 0xfe, 0x03, 0x57, 0x9e, 0x23, 0xbf, 0xef, 0x1b, + 0xce, 0xe1, 0x40, 0x6f, 0x4a, 0x59, 0x46, 0x99, 0x1b, 0x06, 0x8c, 0xb8, 0x8c, 0xd3, 0x25, 0x71, + 0xf3, 0x7e, 0x48, 0x78, 0xd0, 0x77, 0xd9, 0x3c, 0x58, 0xb0, 0x98, 0x72, 0x67, 0xb1, 0xa4, 0x9c, + 0xa2, 0xff, 0x65, 0xd2, 0x11, 0x49, 0x47, 0x26, 0x9d, 0x2a, 0x79, 0xd0, 0x89, 0x68, 0x44, 0x65, + 0xca, 0x15, 0x55, 0x09, 0xd8, 0x8f, 0x1a, 0xb4, 0xc6, 0x95, 0x63, 0xc8, 0x49, 0x86, 0x6e, 0xa0, + 0x29, 0x39, 0x53, 0x3d, 0x54, 0x7b, 0xbf, 0xcf, 0x4f, 0x9d, 0x6f, 0x8d, 0xce, 0x9e, 0x1b, 0x8b, + 0xbf, 0x02, 0xf6, 0x15, 0x5c, 0xc2, 0x68, 0x04, 0x7a, 0x12, 0xe4, 0x33, 0x53, 0x93, 0x92, 0x93, + 0x1a, 0x92, 0xe1, 0xf5, 0xe4, 0x56, 0x38, 0xbc, 0x9f, 0xc5, 0xb6, 0xab, 0x8b, 0xce, 0x57, 0xb0, + 0x94, 0xa0, 0x01, 0x68, 0x69, 0x6e, 0x36, 0xa4, 0xea, 0xb8, 0x86, 0x6a, 0x34, 0x91, 0x22, 0xa3, + 0xd8, 0x76, 0xb5, 0xd1, 0xc4, 0x57, 0xb0, 0x96, 0xe6, 0x68, 0x00, 0x06, 0x9b, 0xc6, 0x24, 0x0b, + 0x4c, 0xbd, 0xb6, 0x68, 0x2c, 0x01, 0x5f, 0xc1, 0x15, 0xea, 0x19, 0xa0, 0x27, 0x9c, 0x64, 0xf6, + 0x11, 0xfc, 0xfd, 0x34, 0x3c, 0x42, 0xa0, 0xcf, 0x83, 0xac, 0x5c, 0xdc, 0x2f, 0x2c, 0x6b, 0x7b, + 0x06, 0xed, 0x8f, 0x03, 0xa2, 0x36, 0x34, 0x52, 0xb2, 0x96, 0xb1, 0x16, 0x16, 0x25, 0xea, 0x40, + 0x33, 0x0f, 0x66, 0x2b, 0x22, 0xd7, 0xd5, 0xc2, 0x65, 0x83, 0x4c, 0xf8, 0x91, 0x93, 0x25, 0x4b, + 0xe8, 0x5c, 0xce, 0xde, 0xc0, 0xfb, 0x16, 0xfd, 0x03, 0x23, 0x26, 0x49, 0x14, 0x73, 0x39, 0x4b, + 0x13, 0x57, 0x9d, 0x7d, 0x05, 0x7f, 0xde, 0xef, 0xa0, 0xee, 0x5b, 0xf6, 0xe5, 0x1b, 0x59, 0x0e, + 0xfd, 0x05, 0x89, 0x40, 0xe7, 0xeb, 0xc5, 0x1e, 0x94, 0xb5, 0xe7, 0x3d, 0x15, 0x96, 0xba, 0x29, + 0x2c, 0xf5, 0xa5, 0xb0, 0xd4, 0x87, 0x9d, 0xa5, 0x6c, 0x76, 0x96, 0xf2, 0xbc, 0xb3, 0x94, 0xbb, + 0x5e, 0x94, 0xf0, 0x78, 0x15, 0x3a, 0x53, 0x9a, 0xb9, 0xd5, 0xf9, 0x96, 0x9f, 0x33, 0x76, 0x9f, + 0x56, 0x47, 0x2c, 0x14, 0x2c, 0x34, 0xe4, 0x25, 0x5e, 0xbc, 0x06, 0x00, 0x00, 0xff, 0xff, 0x38, + 0x51, 0x1f, 0x3e, 0xe6, 0x02, 0x00, 0x00, } func (m *SnapshotItem) Marshal() (dAtA []byte, err error) { @@ -334,6 +472,48 @@ func (m *SnapshotItem_IAVL) MarshalToSizedBuffer(dAtA []byte) (int, error) { } return len(dAtA) - i, nil } +func (m *SnapshotItem_KV) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SnapshotItem_KV) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.KV != nil { + { + size, err := m.KV.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSnapshot(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + return len(dAtA) - i, nil +} +func (m *SnapshotItem_Schema) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SnapshotItem_Schema) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.Schema != nil { + { + size, err := m.Schema.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSnapshot(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 + } + return len(dAtA) - i, nil +} func (m *SnapshotStoreItem) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -411,6 +591,80 @@ func (m *SnapshotIAVLItem) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *SnapshotKVItem) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SnapshotKVItem) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SnapshotKVItem) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Value) > 0 { + i -= len(m.Value) + copy(dAtA[i:], m.Value) + i = encodeVarintSnapshot(dAtA, i, uint64(len(m.Value))) + i-- + dAtA[i] = 0x12 + } + if len(m.Key) > 0 { + i -= len(m.Key) + copy(dAtA[i:], m.Key) + i = encodeVarintSnapshot(dAtA, i, uint64(len(m.Key))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *SnapshotSchema) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *SnapshotSchema) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *SnapshotSchema) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.Type) > 0 { + i -= len(m.Type) + copy(dAtA[i:], m.Type) + i = encodeVarintSnapshot(dAtA, i, uint64(len(m.Type))) + i-- + dAtA[i] = 0x12 + } + if len(m.Key) > 0 { + i -= len(m.Key) + copy(dAtA[i:], m.Key) + i = encodeVarintSnapshot(dAtA, i, uint64(len(m.Key))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func encodeVarintSnapshot(dAtA []byte, offset int, v uint64) int { offset -= sovSnapshot(v) base := offset @@ -458,6 +712,30 @@ func (m *SnapshotItem_IAVL) Size() (n int) { } return n } +func (m *SnapshotItem_KV) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.KV != nil { + l = m.KV.Size() + n += 1 + l + sovSnapshot(uint64(l)) + } + return n +} +func (m *SnapshotItem_Schema) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Schema != nil { + l = m.Schema.Size() + n += 1 + l + sovSnapshot(uint64(l)) + } + return n +} func (m *SnapshotStoreItem) Size() (n int) { if m == nil { return 0 @@ -494,6 +772,40 @@ func (m *SnapshotIAVLItem) Size() (n int) { return n } +func (m *SnapshotKVItem) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Key) + if l > 0 { + n += 1 + l + sovSnapshot(uint64(l)) + } + l = len(m.Value) + if l > 0 { + n += 1 + l + sovSnapshot(uint64(l)) + } + return n +} + +func (m *SnapshotSchema) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Key) + if l > 0 { + n += 1 + l + sovSnapshot(uint64(l)) + } + l = len(m.Type) + if l > 0 { + n += 1 + l + sovSnapshot(uint64(l)) + } + return n +} + func sovSnapshot(x uint64) (n int) { return (math_bits.Len64(x|1) + 6) / 7 } @@ -599,6 +911,76 @@ func (m *SnapshotItem) Unmarshal(dAtA []byte) error { } m.Item = &SnapshotItem_IAVL{v} iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field KV", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSnapshot + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSnapshot + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSnapshot + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &SnapshotKVItem{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Item = &SnapshotItem_KV{v} + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Schema", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSnapshot + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSnapshot + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSnapshot + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &SnapshotSchema{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Item = &SnapshotItem_Schema{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipSnapshot(dAtA[iNdEx:]) @@ -858,6 +1240,242 @@ func (m *SnapshotIAVLItem) Unmarshal(dAtA []byte) error { } return nil } +func (m *SnapshotKVItem) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSnapshot + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SnapshotKVItem: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SnapshotKVItem: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSnapshot + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthSnapshot + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthSnapshot + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) + if m.Key == nil { + m.Key = []byte{} + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSnapshot + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthSnapshot + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthSnapshot + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Value = append(m.Value[:0], dAtA[iNdEx:postIndex]...) + if m.Value == nil { + m.Value = []byte{} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSnapshot(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSnapshot + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SnapshotSchema) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSnapshot + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SnapshotSchema: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SnapshotSchema: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSnapshot + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthSnapshot + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthSnapshot + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) + if m.Key == nil { + m.Key = []byte{} + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + var byteLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSnapshot + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + byteLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if byteLen < 0 { + return ErrInvalidLengthSnapshot + } + postIndex := iNdEx + byteLen + if postIndex < 0 { + return ErrInvalidLengthSnapshot + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Type = append(m.Type[:0], dAtA[iNdEx:postIndex]...) + if m.Type == nil { + m.Type = []byte{} + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSnapshot(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSnapshot + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func skipSnapshot(dAtA []byte) (n int, err error) { l := len(dAtA) iNdEx := 0 diff --git a/store/v2/root/store.go b/store/v2/root/store.go index 1d94a23eaf20..3eedd009e27c 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -1,8 +1,14 @@ package root import ( + "bufio" + "compress/zlib" "errors" "fmt" + "github.com/cosmos/cosmos-sdk/snapshots" + snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" + storetypes "github.com/cosmos/cosmos-sdk/store/types" + protoio "github.com/gogo/protobuf/io" "io" "math" "strings" @@ -46,6 +52,11 @@ var ( merkleNodePrefix = []byte{3} // Prefix for Merkle tree nodes merkleValuePrefix = []byte{4} // Prefix for Merkle value mappings + // Do not change chunk size without new snapshot format (must be uniform across nodes) + snapshotChunkSize = uint64(10e6) + snapshotBufferSize = int(snapshotChunkSize) + snapshotMaxItemSize = int(64e6) // SDK has no key/value size limit, so we set an arbitrary limit + ErrVersionDoesNotExist = errors.New("version does not exist") ErrMaximumHeight = errors.New("maximum block height reached") ) @@ -529,7 +540,7 @@ func (s *Store) Commit() types.CommitID { if target > math.MaxInt64 { panic(ErrMaximumHeight) } - // Fast forward to initialversion if needed + // Fast forward to initial version if needed if s.InitialVersion != 0 && target < s.InitialVersion { target = s.InitialVersion } @@ -559,7 +570,7 @@ func (s *Store) Commit() types.CommitID { func (s *Store) getMerkleRoots() (ret map[string][]byte, err error) { ret = map[string][]byte{} - for key, _ := range s.schema { + for key := range s.schema { sub, has := s.substoreCache[key] if !has { sub, err = s.getSubstore(key) @@ -900,9 +911,181 @@ func (tlm *traceListenMixin) wrapTraceListen(store types.KVStore, skey types.Sto func (s *Store) GetPruning() types.PruningOptions { return s.Pruning } func (s *Store) SetPruning(po types.PruningOptions) { s.Pruning = po } +// Restore implements snapshottypes.Snapshotter. func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser, ready chan<- struct{}) error { + if height == 0 { + return sdkerrors.Wrap(sdkerrors.ErrLogic, "cannot restore snapshot at height 0") + } + if height > uint64(math.MaxInt64) { + return sdkerrors.Wrapf(snapshottypes.ErrInvalidMetadata, + "snapshot height %v cannot exceed %v", height, int64(math.MaxInt64)) + } + + // Signal readiness. Must be done before the readers below are set up, since the zlib + // reader reads from the stream on initialization, potentially causing deadlocks. + if ready != nil { + close(ready) + } + + // Set up a restore stream pipeline + // chan io.ReadCloser -> chunkReader -> zlib -> delimited Protobuf -> ExportNode + chunkReader := snapshots.NewChunkReader(chunks) + defer chunkReader.Close() + zReader, err := zlib.NewReader(chunkReader) + if err != nil { + return sdkerrors.Wrap(err, "zlib failure") + } + defer zReader.Close() + protoReader := protoio.NewDelimitedReader(zReader, snapshotMaxItemSize) + defer protoReader.Close() + + var sstore *substore + // initialisation empty store-schema for snapshot + ret := prefixRegistry{StoreSchema: StoreSchema{}} + + for { + item := &storetypes.SnapshotItem{} + err := protoReader.ReadMsg(item) + if err == io.EOF { + break + } else if err != nil { + return sdkerrors.Wrap(err, "invalid protobuf message") + } + + switch item := item.Item.(type) { + case *storetypes.SnapshotItem_Schema: + sKey := item.Schema.GetKey() + sValue := item.Schema.GetType() + if len(sValue) != 1 || !validSubStoreType(types.StoreType(sValue[0])) { + return sdkerrors.Wrap(err, fmt.Sprintf("invalid mapping for store key: %v => %v", item.Schema.Key, item.Schema.Type)) + } + ret.StoreSchema[string(sKey)] = types.StoreType(sValue[0]) + ret.reserved = append(ret.reserved, string(sKey)) // assume iter yields keys sorted + + case *storetypes.SnapshotItem_Store: + // set the new snapshot store schema to root-store + rs.schema = ret.StoreSchema + sstore, err = rs.getSubstore(item.Store.Name) + if err != nil { + return sdkerrors.Wrap(err, fmt.Sprintf("error while getting the substore for key %s", item.Store.Name)) + } + + case *storetypes.SnapshotItem_KV: + // update the key/value SMT.Store + sstore.Set(item.KV.Key, item.KV.Value) + } + } + + // commit the all key/values to store + _, err = rs.commit(height) + if err != nil { + return sdkerrors.Wrap(err, fmt.Sprintf("error while commit the store at height %d", height)) + } + return nil } + +// Snapshot implements snapshottypes.Snapshotter. func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) { - return nil, nil + if height == 0 { + return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "cannot snapshot height 0") + } + if height > uint64(rs.LastCommitID().Version) { + return nil, sdkerrors.Wrapf(sdkerrors.ErrLogic, "cannot snapshot future height %v", height) + } + versions, err := rs.stateDB.Versions() + if !versions.Exists(height) { + return nil, sdkerrors.Wrapf(sdkerrors.ErrNotFound, "cannot find snapshot at height %v", height) + } + + // get the saved snapshot at height + vs, err := rs.getView(int64(height)) + if err != nil { + return nil, sdkerrors.Wrap(err, fmt.Sprintf("error while get the version at height %d", height)) + } + + // Spawn goroutine to generate snapshot chunks and pass their io.ReadClosers through a channel + ch := make(chan io.ReadCloser) + go func() { + // Set up a stream pipeline to serialize snapshot nodes: + // ExportNode -> delimited Protobuf -> zlib -> buffer -> chunkWriter -> chan io.ReadCloser + chunkWriter := snapshots.NewChunkWriter(ch, snapshotChunkSize) + defer chunkWriter.Close() + bufWriter := bufio.NewWriterSize(chunkWriter, snapshotBufferSize) + defer func() { + if err := bufWriter.Flush(); err != nil { + chunkWriter.CloseWithError(err) + } + }() + zWriter, err := zlib.NewWriterLevel(bufWriter, 7) + if err != nil { + chunkWriter.CloseWithError(sdkerrors.Wrap(err, "zlib failure")) + return + } + defer func() { + if err := zWriter.Close(); err != nil { + chunkWriter.CloseWithError(err) + } + }() + protoWriter := protoio.NewDelimitedWriter(zWriter) + defer func() { + if err := protoWriter.Close(); err != nil { + chunkWriter.CloseWithError(err) + } + }() + + // sending the snapshot store schema + for skey, typ := range vs.schema { + err = protoWriter.WriteMsg(&storetypes.SnapshotItem{ + Item: &storetypes.SnapshotItem_Schema{ + Schema: &storetypes.SnapshotSchema{ + Key: []byte(skey), + Type: []byte{byte(typ)}, + }, + }, + }) + if err != nil { + chunkWriter.CloseWithError(err) + return + } + } + + for key := range vs.schema { + getSubstore, err := vs.getSubstore(key) + if err := protoWriter.Close(); err != nil { + chunkWriter.CloseWithError(err) + } + + err = protoWriter.WriteMsg(&storetypes.SnapshotItem{ + Item: &storetypes.SnapshotItem_Store{ + Store: &storetypes.SnapshotStoreItem{ + Name: key, + }, + }, + }) + if err != nil { + chunkWriter.CloseWithError(err) + return + } + + iter := getSubstore.Iterator(nil, nil) + for ; iter.Valid(); iter.Next() { + err = protoWriter.WriteMsg(&storetypes.SnapshotItem{ + Item: &storetypes.SnapshotItem_KV{ + KV: &storetypes.SnapshotKVItem{ + Key: iter.Key(), + Value: iter.Value(), + }, + }, + }) + if err != nil { + chunkWriter.CloseWithError(err) + return + } + } + iter.Close() + } + }() + + return ch, nil } From 9eda661098a3b4e33624520af96bbee1fbb7069f Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Thu, 9 Dec 2021 16:10:39 +0530 Subject: [PATCH 39/47] address the pr comments --- docs/core/proto-docs.md | 3 +- .../cosmos/base/store/v1beta1/snapshot.proto | 3 +- store/types/snapshot.pb.go | 141 ++++-------- store/v2/root/snapshot.go | 216 ++++++++++++++++++ store/v2/root/store.go | 186 +-------------- 5 files changed, 264 insertions(+), 285 deletions(-) create mode 100644 store/v2/root/snapshot.go diff --git a/docs/core/proto-docs.md b/docs/core/proto-docs.md index 42008ed98768..50e427a19e6a 100644 --- a/docs/core/proto-docs.md +++ b/docs/core/proto-docs.md @@ -2603,8 +2603,7 @@ SnapshotSchema is an exported schema of store | Field | Type | Label | Description | | ----- | ---- | ----- | ----------- | -| `key` | [bytes](#bytes) | | | -| `type` | [bytes](#bytes) | | | +| `keys` | [bytes](#bytes) | repeated | | diff --git a/proto/cosmos/base/store/v1beta1/snapshot.proto b/proto/cosmos/base/store/v1beta1/snapshot.proto index de5dee678f7b..f1a71607d21d 100644 --- a/proto/cosmos/base/store/v1beta1/snapshot.proto +++ b/proto/cosmos/base/store/v1beta1/snapshot.proto @@ -37,6 +37,5 @@ message SnapshotKVItem { // SnapshotSchema is an exported schema of store message SnapshotSchema{ - bytes key = 1; - bytes type = 2; + repeated bytes keys = 1; } \ No newline at end of file diff --git a/store/types/snapshot.pb.go b/store/types/snapshot.pb.go index 9a64862de162..57fcc4de8e7a 100644 --- a/store/types/snapshot.pb.go +++ b/store/types/snapshot.pb.go @@ -306,8 +306,7 @@ func (m *SnapshotKVItem) GetValue() []byte { // SnapshotSchema is an exported schema of store type SnapshotSchema struct { - Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` - Type []byte `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` + Keys [][]byte `protobuf:"bytes,1,rep,name=keys,proto3" json:"keys,omitempty"` } func (m *SnapshotSchema) Reset() { *m = SnapshotSchema{} } @@ -343,16 +342,9 @@ func (m *SnapshotSchema) XXX_DiscardUnknown() { var xxx_messageInfo_SnapshotSchema proto.InternalMessageInfo -func (m *SnapshotSchema) GetKey() []byte { +func (m *SnapshotSchema) GetKeys() [][]byte { if m != nil { - return m.Key - } - return nil -} - -func (m *SnapshotSchema) GetType() []byte { - if m != nil { - return m.Type + return m.Keys } return nil } @@ -370,32 +362,32 @@ func init() { } var fileDescriptor_9c55879db4cc4502 = []byte{ - // 391 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x92, 0xcf, 0x6a, 0xab, 0x40, - 0x14, 0x87, 0xd5, 0x18, 0xef, 0xbd, 0xe7, 0x86, 0x4b, 0xee, 0x10, 0x2e, 0xde, 0x2e, 0x4c, 0x71, - 0xd3, 0x94, 0xb6, 0x4a, 0x5a, 0x28, 0xdd, 0xd6, 0x74, 0x61, 0x48, 0x57, 0x13, 0xc8, 0xa2, 0x3b, - 0x4d, 0x07, 0x15, 0x63, 0x26, 0x64, 0x26, 0x42, 0xde, 0xa2, 0xef, 0xd2, 0x97, 0xe8, 0x32, 0xcb, - 0xae, 0x42, 0x31, 0x2f, 0x52, 0x66, 0x34, 0x94, 0xfe, 0x03, 0x57, 0x9e, 0x23, 0xbf, 0xef, 0x1b, - 0xce, 0xe1, 0x40, 0x6f, 0x4a, 0x59, 0x46, 0x99, 0x1b, 0x06, 0x8c, 0xb8, 0x8c, 0xd3, 0x25, 0x71, - 0xf3, 0x7e, 0x48, 0x78, 0xd0, 0x77, 0xd9, 0x3c, 0x58, 0xb0, 0x98, 0x72, 0x67, 0xb1, 0xa4, 0x9c, - 0xa2, 0xff, 0x65, 0xd2, 0x11, 0x49, 0x47, 0x26, 0x9d, 0x2a, 0x79, 0xd0, 0x89, 0x68, 0x44, 0x65, - 0xca, 0x15, 0x55, 0x09, 0xd8, 0x8f, 0x1a, 0xb4, 0xc6, 0x95, 0x63, 0xc8, 0x49, 0x86, 0x6e, 0xa0, - 0x29, 0x39, 0x53, 0x3d, 0x54, 0x7b, 0xbf, 0xcf, 0x4f, 0x9d, 0x6f, 0x8d, 0xce, 0x9e, 0x1b, 0x8b, - 0xbf, 0x02, 0xf6, 0x15, 0x5c, 0xc2, 0x68, 0x04, 0x7a, 0x12, 0xe4, 0x33, 0x53, 0x93, 0x92, 0x93, - 0x1a, 0x92, 0xe1, 0xf5, 0xe4, 0x56, 0x38, 0xbc, 0x9f, 0xc5, 0xb6, 0xab, 0x8b, 0xce, 0x57, 0xb0, - 0x94, 0xa0, 0x01, 0x68, 0x69, 0x6e, 0x36, 0xa4, 0xea, 0xb8, 0x86, 0x6a, 0x34, 0x91, 0x22, 0xa3, - 0xd8, 0x76, 0xb5, 0xd1, 0xc4, 0x57, 0xb0, 0x96, 0xe6, 0x68, 0x00, 0x06, 0x9b, 0xc6, 0x24, 0x0b, - 0x4c, 0xbd, 0xb6, 0x68, 0x2c, 0x01, 0x5f, 0xc1, 0x15, 0xea, 0x19, 0xa0, 0x27, 0x9c, 0x64, 0xf6, - 0x11, 0xfc, 0xfd, 0x34, 0x3c, 0x42, 0xa0, 0xcf, 0x83, 0xac, 0x5c, 0xdc, 0x2f, 0x2c, 0x6b, 0x7b, - 0x06, 0xed, 0x8f, 0x03, 0xa2, 0x36, 0x34, 0x52, 0xb2, 0x96, 0xb1, 0x16, 0x16, 0x25, 0xea, 0x40, - 0x33, 0x0f, 0x66, 0x2b, 0x22, 0xd7, 0xd5, 0xc2, 0x65, 0x83, 0x4c, 0xf8, 0x91, 0x93, 0x25, 0x4b, - 0xe8, 0x5c, 0xce, 0xde, 0xc0, 0xfb, 0x16, 0xfd, 0x03, 0x23, 0x26, 0x49, 0x14, 0x73, 0x39, 0x4b, - 0x13, 0x57, 0x9d, 0x7d, 0x05, 0x7f, 0xde, 0xef, 0xa0, 0xee, 0x5b, 0xf6, 0xe5, 0x1b, 0x59, 0x0e, - 0xfd, 0x05, 0x89, 0x40, 0xe7, 0xeb, 0xc5, 0x1e, 0x94, 0xb5, 0xe7, 0x3d, 0x15, 0x96, 0xba, 0x29, - 0x2c, 0xf5, 0xa5, 0xb0, 0xd4, 0x87, 0x9d, 0xa5, 0x6c, 0x76, 0x96, 0xf2, 0xbc, 0xb3, 0x94, 0xbb, - 0x5e, 0x94, 0xf0, 0x78, 0x15, 0x3a, 0x53, 0x9a, 0xb9, 0xd5, 0xf9, 0x96, 0x9f, 0x33, 0x76, 0x9f, - 0x56, 0x47, 0x2c, 0x14, 0x2c, 0x34, 0xe4, 0x25, 0x5e, 0xbc, 0x06, 0x00, 0x00, 0xff, 0xff, 0x38, - 0x51, 0x1f, 0x3e, 0xe6, 0x02, 0x00, 0x00, + // 393 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x92, 0xcf, 0x4a, 0xeb, 0x40, + 0x18, 0xc5, 0xf3, 0xaf, 0xb9, 0xf7, 0xce, 0x0d, 0x52, 0x87, 0x22, 0xd1, 0x45, 0x5a, 0x82, 0x60, + 0x44, 0x4d, 0xa8, 0x6e, 0xdc, 0x9a, 0xba, 0x48, 0xa9, 0xab, 0x29, 0x74, 0xe1, 0x2e, 0xa9, 0x43, + 0x12, 0xd2, 0x74, 0x4a, 0x67, 0x1a, 0xe8, 0x5b, 0xf8, 0x2e, 0xbe, 0x84, 0xcb, 0x2e, 0x5d, 0x15, + 0x49, 0x5f, 0x44, 0x66, 0x92, 0x22, 0x2a, 0x42, 0x56, 0xf9, 0x4e, 0x38, 0xe7, 0x37, 0x9c, 0x8f, + 0x0f, 0x38, 0x53, 0x42, 0x73, 0x42, 0xbd, 0x28, 0xa4, 0xd8, 0xa3, 0x8c, 0x2c, 0xb1, 0x57, 0xf4, + 0x23, 0xcc, 0xc2, 0xbe, 0x47, 0xe7, 0xe1, 0x82, 0x26, 0x84, 0xb9, 0x8b, 0x25, 0x61, 0x04, 0x1e, + 0x57, 0x4e, 0x97, 0x3b, 0x5d, 0xe1, 0x74, 0x6b, 0xe7, 0x49, 0x27, 0x26, 0x31, 0x11, 0x2e, 0x8f, + 0x4f, 0x55, 0xc0, 0x7e, 0x51, 0x80, 0x31, 0xae, 0x19, 0x43, 0x86, 0x73, 0x78, 0x0f, 0x5a, 0x22, + 0x67, 0xca, 0x3d, 0xd9, 0xf9, 0x7f, 0x7d, 0xe9, 0xfe, 0x4a, 0x74, 0xf7, 0xb9, 0x31, 0xff, 0xcb, + 0xc3, 0x81, 0x84, 0xaa, 0x30, 0x1c, 0x01, 0x2d, 0x0d, 0x8b, 0x99, 0xa9, 0x08, 0xc8, 0x45, 0x03, + 0xc8, 0xf0, 0x6e, 0xf2, 0xc0, 0x19, 0xfe, 0xdf, 0x72, 0xdb, 0xd5, 0xb8, 0x0a, 0x24, 0x24, 0x20, + 0x70, 0x00, 0x94, 0xac, 0x30, 0x55, 0x81, 0x3a, 0x6f, 0x80, 0x1a, 0x4d, 0x04, 0x48, 0x2f, 0xb7, + 0x5d, 0x65, 0x34, 0x09, 0x24, 0xa4, 0x64, 0x05, 0x1c, 0x00, 0x9d, 0x4e, 0x13, 0x9c, 0x87, 0xa6, + 0xd6, 0x18, 0x34, 0x16, 0x81, 0x40, 0x42, 0x75, 0xd4, 0xd7, 0x81, 0x96, 0x32, 0x9c, 0xdb, 0x67, + 0xe0, 0xf0, 0x47, 0x79, 0x08, 0x81, 0x36, 0x0f, 0xf3, 0x6a, 0x71, 0xff, 0x90, 0x98, 0xed, 0x19, + 0x68, 0x7f, 0x2f, 0x08, 0xdb, 0x40, 0xcd, 0xf0, 0x5a, 0xd8, 0x0c, 0xc4, 0x47, 0xd8, 0x01, 0xad, + 0x22, 0x9c, 0xad, 0xb0, 0x58, 0x97, 0x81, 0x2a, 0x01, 0x4d, 0xf0, 0xa7, 0xc0, 0x4b, 0x9a, 0x92, + 0xb9, 0xe8, 0xae, 0xa2, 0xbd, 0x84, 0x47, 0x40, 0x4f, 0x70, 0x1a, 0x27, 0x4c, 0x74, 0x69, 0xa1, + 0x5a, 0xd9, 0xb7, 0xe0, 0xe0, 0xeb, 0x0e, 0x9a, 0xbe, 0x65, 0x9f, 0x7e, 0x26, 0xab, 0xd2, 0xbc, + 0x4d, 0x86, 0xd7, 0xd4, 0x94, 0x7b, 0xaa, 0x63, 0x20, 0x31, 0xfb, 0xfe, 0x6b, 0x69, 0xc9, 0x9b, + 0xd2, 0x92, 0xdf, 0x4b, 0x4b, 0x7e, 0xde, 0x59, 0xd2, 0x66, 0x67, 0x49, 0x6f, 0x3b, 0x4b, 0x7a, + 0x74, 0xe2, 0x94, 0x25, 0xab, 0xc8, 0x9d, 0x92, 0xdc, 0xab, 0x8f, 0xb5, 0xfa, 0x5c, 0xd1, 0xa7, + 0xac, 0x3e, 0x59, 0xb6, 0x5e, 0x60, 0x1a, 0xe9, 0xe2, 0xee, 0x6e, 0x3e, 0x02, 0x00, 0x00, 0xff, + 0xff, 0xe8, 0xed, 0x8f, 0x77, 0xd4, 0x02, 0x00, 0x00, } func (m *SnapshotItem) Marshal() (dAtA []byte, err error) { @@ -648,19 +640,14 @@ func (m *SnapshotSchema) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if len(m.Type) > 0 { - i -= len(m.Type) - copy(dAtA[i:], m.Type) - i = encodeVarintSnapshot(dAtA, i, uint64(len(m.Type))) - i-- - dAtA[i] = 0x12 - } - if len(m.Key) > 0 { - i -= len(m.Key) - copy(dAtA[i:], m.Key) - i = encodeVarintSnapshot(dAtA, i, uint64(len(m.Key))) - i-- - dAtA[i] = 0xa + if len(m.Keys) > 0 { + for iNdEx := len(m.Keys) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Keys[iNdEx]) + copy(dAtA[i:], m.Keys[iNdEx]) + i = encodeVarintSnapshot(dAtA, i, uint64(len(m.Keys[iNdEx]))) + i-- + dAtA[i] = 0xa + } } return len(dAtA) - i, nil } @@ -795,13 +782,11 @@ func (m *SnapshotSchema) Size() (n int) { } var l int _ = l - l = len(m.Key) - if l > 0 { - n += 1 + l + sovSnapshot(uint64(l)) - } - l = len(m.Type) - if l > 0 { - n += 1 + l + sovSnapshot(uint64(l)) + if len(m.Keys) > 0 { + for _, b := range m.Keys { + l = len(b) + n += 1 + l + sovSnapshot(uint64(l)) + } } return n } @@ -1389,41 +1374,7 @@ func (m *SnapshotSchema) Unmarshal(dAtA []byte) error { switch fieldNum { case 1: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Key", wireType) - } - var byteLen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowSnapshot - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - byteLen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if byteLen < 0 { - return ErrInvalidLengthSnapshot - } - postIndex := iNdEx + byteLen - if postIndex < 0 { - return ErrInvalidLengthSnapshot - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.Key = append(m.Key[:0], dAtA[iNdEx:postIndex]...) - if m.Key == nil { - m.Key = []byte{} - } - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Keys", wireType) } var byteLen int for shift := uint(0); ; shift += 7 { @@ -1450,10 +1401,8 @@ func (m *SnapshotSchema) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Type = append(m.Type[:0], dAtA[iNdEx:postIndex]...) - if m.Type == nil { - m.Type = []byte{} - } + m.Keys = append(m.Keys, make([]byte, postIndex-iNdEx)) + copy(m.Keys[len(m.Keys)-1], dAtA[iNdEx:postIndex]) iNdEx = postIndex default: iNdEx = preIndex diff --git a/store/v2/root/snapshot.go b/store/v2/root/snapshot.go new file mode 100644 index 000000000000..4cbec73347da --- /dev/null +++ b/store/v2/root/snapshot.go @@ -0,0 +1,216 @@ +package root + +import ( + "bufio" + "compress/zlib" + "fmt" + prefixdb "github.com/cosmos/cosmos-sdk/db/prefix" + "github.com/cosmos/cosmos-sdk/snapshots" + snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" + storetypes "github.com/cosmos/cosmos-sdk/store/types" + types "github.com/cosmos/cosmos-sdk/store/v2" + sdkerrors "github.com/cosmos/cosmos-sdk/types/errors" + protoio "github.com/gogo/protobuf/io" + "io" + "math" +) + +// Restore implements snapshottypes.Snapshotter. +func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser, ready chan<- struct{}) error { + if height == 0 { + return sdkerrors.Wrap(sdkerrors.ErrLogic, "cannot restore snapshot at height 0") + } + if height > uint64(math.MaxInt64) { + return sdkerrors.Wrapf(snapshottypes.ErrInvalidMetadata, + "snapshot height %v cannot exceed %v", height, int64(math.MaxInt64)) + } + + versions, err := rs.stateDB.Versions() + if versions.Count() != 0 { + return sdkerrors.Wrapf(sdkerrors.ErrLogic, "cannot restore snapshot %v", height) + } + + // Signal readiness. Must be done before the readers below are set up, since the zlib + // reader reads from the stream on initialization, potentially causing deadlocks. + if ready != nil { + close(ready) + } + + // Set up a restore stream pipeline + // chan io.ReadCloser -> chunkReader -> zlib -> delimited Protobuf -> ExportNode + chunkReader := snapshots.NewChunkReader(chunks) + defer chunkReader.Close() + zReader, err := zlib.NewReader(chunkReader) + if err != nil { + return sdkerrors.Wrap(err, "zlib failure") + } + defer zReader.Close() + protoReader := protoio.NewDelimitedReader(zReader, snapshotMaxItemSize) + defer protoReader.Close() + + var subStore *substore + // initialisation empty store-schema for snapshot + ret := prefixRegistry{StoreSchema: StoreSchema{}} + + for { + item := &storetypes.SnapshotItem{} + err := protoReader.ReadMsg(item) + if err == io.EOF { + break + } else if err != nil { + return sdkerrors.Wrap(err, "invalid protobuf message") + } + + switch item := item.Item.(type) { + case *storetypes.SnapshotItem_Schema: + schemaWriter := prefixdb.NewPrefixWriter(rs.stateTxn, schemaPrefix) + sKeys := item.Schema.GetKeys() + for _, sKey := range sKeys { + ret.StoreSchema[string(sKey)] = types.StoreTypePersistent + ret.reserved = append(ret.reserved, string(sKey)) + err := schemaWriter.Set(sKey, []byte{byte(types.StoreTypePersistent)}) + if err != nil { + return sdkerrors.Wrap(err, "error at set the store schema key values") + } + } + + case *storetypes.SnapshotItem_Store: + storeName := item.Store.GetName() + _, _, err := ret.storeInfo(storeName) + if err != nil { + return sdkerrors.Wrap(sdkerrors.ErrLogic, "received store name before store schema") + } + // set the new snapshot store schema to root-store + rs.schema = ret.StoreSchema + subStore, err = rs.getSubstore(storeName) + if err != nil { + return sdkerrors.Wrap(err, fmt.Sprintf("error while getting the substore for key %s", storeName)) + } + + case *storetypes.SnapshotItem_KV: + if subStore == nil { + return sdkerrors.Wrap(sdkerrors.ErrLogic, "received KV Item before store item") + } + // update the key/value SMT.Store + subStore.Set(item.KV.Key, item.KV.Value) + + default: + return sdkerrors.Wrapf(sdkerrors.ErrLogic, "unknown snapshot item %T", item) + } + } + + // commit the all key/values to store + _, err = rs.commit(height) + if err != nil { + return sdkerrors.Wrap(err, fmt.Sprintf("error while commit the store at height %d", height)) + } + + return nil +} + +// Snapshot implements snapshottypes.Snapshotter. +func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) { + if height == 0 { + return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "cannot snapshot height 0") + } + if height > uint64(rs.LastCommitID().Version) { + return nil, sdkerrors.Wrapf(sdkerrors.ErrLogic, "cannot snapshot future height %v", height) + } + versions, err := rs.stateDB.Versions() + if !versions.Exists(height) { + return nil, sdkerrors.Wrapf(sdkerrors.ErrNotFound, "cannot find snapshot at height %v", height) + } + + // get the saved snapshot at height + vs, err := rs.getView(int64(height)) + if err != nil { + return nil, sdkerrors.Wrap(err, fmt.Sprintf("error while get the version at height %d", height)) + } + + // Spawn goroutine to generate snapshot chunks and pass their io.ReadClosers through a channel + ch := make(chan io.ReadCloser) + go func() { + // Set up a stream pipeline to serialize snapshot nodes: + // ExportNode -> delimited Protobuf -> zlib -> buffer -> chunkWriter -> chan io.ReadCloser + chunkWriter := snapshots.NewChunkWriter(ch, snapshotChunkSize) + defer chunkWriter.Close() + bufWriter := bufio.NewWriterSize(chunkWriter, snapshotBufferSize) + defer func() { + if err := bufWriter.Flush(); err != nil { + chunkWriter.CloseWithError(err) + } + }() + zWriter, err := zlib.NewWriterLevel(bufWriter, 7) + if err != nil { + chunkWriter.CloseWithError(sdkerrors.Wrap(err, "zlib failure")) + return + } + defer func() { + if err := zWriter.Close(); err != nil { + chunkWriter.CloseWithError(err) + } + }() + protoWriter := protoio.NewDelimitedWriter(zWriter) + defer func() { + if err := protoWriter.Close(); err != nil { + chunkWriter.CloseWithError(err) + } + }() + + var sKeys [][]byte + // sending the snapshot store schema + for sKey := range vs.schema { + sKeys = append(sKeys, []byte(sKey)) + } + + err = protoWriter.WriteMsg(&storetypes.SnapshotItem{ + Item: &storetypes.SnapshotItem_Schema{ + Schema: &storetypes.SnapshotSchema{ + Keys: sKeys, + }, + }, + }) + if err != nil { + chunkWriter.CloseWithError(err) + return + } + + for sKey := range vs.schema { + subStore, err := vs.getSubstore(sKey) + if err := protoWriter.Close(); err != nil { + chunkWriter.CloseWithError(err) + } + + err = protoWriter.WriteMsg(&storetypes.SnapshotItem{ + Item: &storetypes.SnapshotItem_Store{ + Store: &storetypes.SnapshotStoreItem{ + Name: sKey, + }, + }, + }) + if err != nil { + chunkWriter.CloseWithError(err) + return + } + + iter := subStore.Iterator(nil, nil) + for ; iter.Valid(); iter.Next() { + err = protoWriter.WriteMsg(&storetypes.SnapshotItem{ + Item: &storetypes.SnapshotItem_KV{ + KV: &storetypes.SnapshotKVItem{ + Key: iter.Key(), + Value: iter.Value(), + }, + }, + }) + if err != nil { + chunkWriter.CloseWithError(err) + return + } + } + iter.Close() + } + }() + + return ch, nil +} diff --git a/store/v2/root/store.go b/store/v2/root/store.go index 3eedd009e27c..a80171ddc906 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -1,14 +1,8 @@ package root import ( - "bufio" - "compress/zlib" "errors" "fmt" - "github.com/cosmos/cosmos-sdk/snapshots" - snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" - storetypes "github.com/cosmos/cosmos-sdk/store/types" - protoio "github.com/gogo/protobuf/io" "io" "math" "strings" @@ -316,6 +310,7 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { return } } + schemaWriter := prefixdb.NewPrefixWriter(ret.stateTxn, schemaPrefix) it, err := schemaView.Iterator(nil, nil) if err != nil { @@ -910,182 +905,3 @@ func (tlm *traceListenMixin) wrapTraceListen(store types.KVStore, skey types.Sto func (s *Store) GetPruning() types.PruningOptions { return s.Pruning } func (s *Store) SetPruning(po types.PruningOptions) { s.Pruning = po } - -// Restore implements snapshottypes.Snapshotter. -func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser, ready chan<- struct{}) error { - if height == 0 { - return sdkerrors.Wrap(sdkerrors.ErrLogic, "cannot restore snapshot at height 0") - } - if height > uint64(math.MaxInt64) { - return sdkerrors.Wrapf(snapshottypes.ErrInvalidMetadata, - "snapshot height %v cannot exceed %v", height, int64(math.MaxInt64)) - } - - // Signal readiness. Must be done before the readers below are set up, since the zlib - // reader reads from the stream on initialization, potentially causing deadlocks. - if ready != nil { - close(ready) - } - - // Set up a restore stream pipeline - // chan io.ReadCloser -> chunkReader -> zlib -> delimited Protobuf -> ExportNode - chunkReader := snapshots.NewChunkReader(chunks) - defer chunkReader.Close() - zReader, err := zlib.NewReader(chunkReader) - if err != nil { - return sdkerrors.Wrap(err, "zlib failure") - } - defer zReader.Close() - protoReader := protoio.NewDelimitedReader(zReader, snapshotMaxItemSize) - defer protoReader.Close() - - var sstore *substore - // initialisation empty store-schema for snapshot - ret := prefixRegistry{StoreSchema: StoreSchema{}} - - for { - item := &storetypes.SnapshotItem{} - err := protoReader.ReadMsg(item) - if err == io.EOF { - break - } else if err != nil { - return sdkerrors.Wrap(err, "invalid protobuf message") - } - - switch item := item.Item.(type) { - case *storetypes.SnapshotItem_Schema: - sKey := item.Schema.GetKey() - sValue := item.Schema.GetType() - if len(sValue) != 1 || !validSubStoreType(types.StoreType(sValue[0])) { - return sdkerrors.Wrap(err, fmt.Sprintf("invalid mapping for store key: %v => %v", item.Schema.Key, item.Schema.Type)) - } - ret.StoreSchema[string(sKey)] = types.StoreType(sValue[0]) - ret.reserved = append(ret.reserved, string(sKey)) // assume iter yields keys sorted - - case *storetypes.SnapshotItem_Store: - // set the new snapshot store schema to root-store - rs.schema = ret.StoreSchema - sstore, err = rs.getSubstore(item.Store.Name) - if err != nil { - return sdkerrors.Wrap(err, fmt.Sprintf("error while getting the substore for key %s", item.Store.Name)) - } - - case *storetypes.SnapshotItem_KV: - // update the key/value SMT.Store - sstore.Set(item.KV.Key, item.KV.Value) - } - } - - // commit the all key/values to store - _, err = rs.commit(height) - if err != nil { - return sdkerrors.Wrap(err, fmt.Sprintf("error while commit the store at height %d", height)) - } - - return nil -} - -// Snapshot implements snapshottypes.Snapshotter. -func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) { - if height == 0 { - return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "cannot snapshot height 0") - } - if height > uint64(rs.LastCommitID().Version) { - return nil, sdkerrors.Wrapf(sdkerrors.ErrLogic, "cannot snapshot future height %v", height) - } - versions, err := rs.stateDB.Versions() - if !versions.Exists(height) { - return nil, sdkerrors.Wrapf(sdkerrors.ErrNotFound, "cannot find snapshot at height %v", height) - } - - // get the saved snapshot at height - vs, err := rs.getView(int64(height)) - if err != nil { - return nil, sdkerrors.Wrap(err, fmt.Sprintf("error while get the version at height %d", height)) - } - - // Spawn goroutine to generate snapshot chunks and pass their io.ReadClosers through a channel - ch := make(chan io.ReadCloser) - go func() { - // Set up a stream pipeline to serialize snapshot nodes: - // ExportNode -> delimited Protobuf -> zlib -> buffer -> chunkWriter -> chan io.ReadCloser - chunkWriter := snapshots.NewChunkWriter(ch, snapshotChunkSize) - defer chunkWriter.Close() - bufWriter := bufio.NewWriterSize(chunkWriter, snapshotBufferSize) - defer func() { - if err := bufWriter.Flush(); err != nil { - chunkWriter.CloseWithError(err) - } - }() - zWriter, err := zlib.NewWriterLevel(bufWriter, 7) - if err != nil { - chunkWriter.CloseWithError(sdkerrors.Wrap(err, "zlib failure")) - return - } - defer func() { - if err := zWriter.Close(); err != nil { - chunkWriter.CloseWithError(err) - } - }() - protoWriter := protoio.NewDelimitedWriter(zWriter) - defer func() { - if err := protoWriter.Close(); err != nil { - chunkWriter.CloseWithError(err) - } - }() - - // sending the snapshot store schema - for skey, typ := range vs.schema { - err = protoWriter.WriteMsg(&storetypes.SnapshotItem{ - Item: &storetypes.SnapshotItem_Schema{ - Schema: &storetypes.SnapshotSchema{ - Key: []byte(skey), - Type: []byte{byte(typ)}, - }, - }, - }) - if err != nil { - chunkWriter.CloseWithError(err) - return - } - } - - for key := range vs.schema { - getSubstore, err := vs.getSubstore(key) - if err := protoWriter.Close(); err != nil { - chunkWriter.CloseWithError(err) - } - - err = protoWriter.WriteMsg(&storetypes.SnapshotItem{ - Item: &storetypes.SnapshotItem_Store{ - Store: &storetypes.SnapshotStoreItem{ - Name: key, - }, - }, - }) - if err != nil { - chunkWriter.CloseWithError(err) - return - } - - iter := getSubstore.Iterator(nil, nil) - for ; iter.Valid(); iter.Next() { - err = protoWriter.WriteMsg(&storetypes.SnapshotItem{ - Item: &storetypes.SnapshotItem_KV{ - KV: &storetypes.SnapshotKVItem{ - Key: iter.Key(), - Value: iter.Value(), - }, - }, - }) - if err != nil { - chunkWriter.CloseWithError(err) - return - } - } - iter.Close() - } - }() - - return ch, nil -} From 551dd3b131eaa4c2d164e301674440c5d1f7390d Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Sat, 11 Dec 2021 16:58:41 +0530 Subject: [PATCH 40/47] address the pr comments --- store/v2/root/snapshot.go | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/store/v2/root/snapshot.go b/store/v2/root/snapshot.go index 4cbec73347da..7788b67bb4d3 100644 --- a/store/v2/root/snapshot.go +++ b/store/v2/root/snapshot.go @@ -27,7 +27,7 @@ func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadClos versions, err := rs.stateDB.Versions() if versions.Count() != 0 { - return sdkerrors.Wrapf(sdkerrors.ErrLogic, "cannot restore snapshot %v", height) + return sdkerrors.Wrapf(sdkerrors.ErrLogic, "cannot restore snapshot for non empty store at height %v", height) } // Signal readiness. Must be done before the readers below are set up, since the zlib @@ -50,7 +50,7 @@ func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadClos var subStore *substore // initialisation empty store-schema for snapshot - ret := prefixRegistry{StoreSchema: StoreSchema{}} + preg := prefixRegistry{StoreSchema: StoreSchema{}} for { item := &storetypes.SnapshotItem{} @@ -66,22 +66,24 @@ func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadClos schemaWriter := prefixdb.NewPrefixWriter(rs.stateTxn, schemaPrefix) sKeys := item.Schema.GetKeys() for _, sKey := range sKeys { - ret.StoreSchema[string(sKey)] = types.StoreTypePersistent - ret.reserved = append(ret.reserved, string(sKey)) + preg.StoreSchema[string(sKey)] = types.StoreTypePersistent + preg.reserved = append(preg.reserved, string(sKey)) err := schemaWriter.Set(sKey, []byte{byte(types.StoreTypePersistent)}) if err != nil { return sdkerrors.Wrap(err, "error at set the store schema key values") } } + // set the new snapshot store schema to root-store + rs.schema = preg.StoreSchema case *storetypes.SnapshotItem_Store: storeName := item.Store.GetName() - _, _, err := ret.storeInfo(storeName) - if err != nil { - return sdkerrors.Wrap(sdkerrors.ErrLogic, "received store name before store schema") + // checking the store schema exists or not + if _, has := rs.schema[storeName]; !has { + return sdkerrors.Wrapf(sdkerrors.ErrLogic, "received store name before store schema %s", storeName) } - // set the new snapshot store schema to root-store - rs.schema = ret.StoreSchema + + // get the substore subStore, err = rs.getSubstore(storeName) if err != nil { return sdkerrors.Wrap(err, fmt.Sprintf("error while getting the substore for key %s", storeName)) From 8f68cedf969d0a66007dfcda89b15f0979afd08a Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Mon, 13 Dec 2021 10:45:08 +0530 Subject: [PATCH 41/47] removed prefix registry from restore --- store/v2/root/snapshot.go | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/store/v2/root/snapshot.go b/store/v2/root/snapshot.go index 7788b67bb4d3..b126f343d883 100644 --- a/store/v2/root/snapshot.go +++ b/store/v2/root/snapshot.go @@ -49,8 +49,6 @@ func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadClos defer protoReader.Close() var subStore *substore - // initialisation empty store-schema for snapshot - preg := prefixRegistry{StoreSchema: StoreSchema{}} for { item := &storetypes.SnapshotItem{} @@ -66,15 +64,12 @@ func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadClos schemaWriter := prefixdb.NewPrefixWriter(rs.stateTxn, schemaPrefix) sKeys := item.Schema.GetKeys() for _, sKey := range sKeys { - preg.StoreSchema[string(sKey)] = types.StoreTypePersistent - preg.reserved = append(preg.reserved, string(sKey)) + rs.schema[string(sKey)] = types.StoreTypePersistent err := schemaWriter.Set(sKey, []byte{byte(types.StoreTypePersistent)}) if err != nil { return sdkerrors.Wrap(err, "error at set the store schema key values") } } - // set the new snapshot store schema to root-store - rs.schema = preg.StoreSchema case *storetypes.SnapshotItem_Store: storeName := item.Store.GetName() From b024b20958f923cd30362462086eb5132337f2da Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Mon, 13 Dec 2021 11:35:59 +0530 Subject: [PATCH 42/47] addressed the pr comments --- store/v2/root/snapshot.go | 21 ++++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/store/v2/root/snapshot.go b/store/v2/root/snapshot.go index b126f343d883..982c5e2f1a3e 100644 --- a/store/v2/root/snapshot.go +++ b/store/v2/root/snapshot.go @@ -26,6 +26,9 @@ func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadClos } versions, err := rs.stateDB.Versions() + if err != nil { + return sdkerrors.Wrapf(err, "error while getting the snapshot versions at height %v", height) + } if versions.Count() != 0 { return sdkerrors.Wrapf(sdkerrors.ErrLogic, "cannot restore snapshot for non empty store at height %v", height) } @@ -49,6 +52,7 @@ func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadClos defer protoReader.Close() var subStore *substore + var storeSchemaReceived bool = false for { item := &storetypes.SnapshotItem{} @@ -61,6 +65,7 @@ func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadClos switch item := item.Item.(type) { case *storetypes.SnapshotItem_Schema: + storeSchemaReceived = true schemaWriter := prefixdb.NewPrefixWriter(rs.stateTxn, schemaPrefix) sKeys := item.Schema.GetKeys() for _, sKey := range sKeys { @@ -73,9 +78,13 @@ func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadClos case *storetypes.SnapshotItem_Store: storeName := item.Store.GetName() + // checking the store schema is received or not + if !storeSchemaReceived { + return sdkerrors.Wrapf(sdkerrors.ErrLogic, "received store name before store schema %s", storeName) + } // checking the store schema exists or not if _, has := rs.schema[storeName]; !has { - return sdkerrors.Wrapf(sdkerrors.ErrLogic, "received store name before store schema %s", storeName) + return sdkerrors.Wrapf(sdkerrors.ErrLogic, "store is missing from schema %s", storeName) } // get the substore @@ -114,6 +123,9 @@ func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, e return nil, sdkerrors.Wrapf(sdkerrors.ErrLogic, "cannot snapshot future height %v", height) } versions, err := rs.stateDB.Versions() + if err != nil { + return nil, sdkerrors.Wrapf(err, "error while getting the snapshot versions at height %v", height) + } if !versions.Exists(height) { return nil, sdkerrors.Wrapf(sdkerrors.ErrNotFound, "cannot find snapshot at height %v", height) } @@ -157,7 +169,9 @@ func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, e var sKeys [][]byte // sending the snapshot store schema for sKey := range vs.schema { - sKeys = append(sKeys, []byte(sKey)) + if vs.schema[sKey] == storetypes.StoreTypePersistent { + sKeys = append(sKeys, []byte(sKey)) + } } err = protoWriter.WriteMsg(&storetypes.SnapshotItem{ @@ -174,8 +188,9 @@ func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, e for sKey := range vs.schema { subStore, err := vs.getSubstore(sKey) - if err := protoWriter.Close(); err != nil { + if err != nil { chunkWriter.CloseWithError(err) + return } err = protoWriter.WriteMsg(&storetypes.SnapshotItem{ From 4284df57ef3473d1945aaa4c903b859769d12352 Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Mon, 13 Dec 2021 17:15:13 +0530 Subject: [PATCH 43/47] test: add test cases for snapshot for v2 store --- store/v2/root/snapshot.go | 29 +++- store/v2/root/snapshot_test.go | 257 +++++++++++++++++++++++++++++++++ 2 files changed, 281 insertions(+), 5 deletions(-) create mode 100644 store/v2/root/snapshot_test.go diff --git a/store/v2/root/snapshot.go b/store/v2/root/snapshot.go index 982c5e2f1a3e..feb48762be93 100644 --- a/store/v2/root/snapshot.go +++ b/store/v2/root/snapshot.go @@ -13,10 +13,16 @@ import ( protoio "github.com/gogo/protobuf/io" "io" "math" + "sort" + "strings" ) // Restore implements snapshottypes.Snapshotter. func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadCloser, ready chan<- struct{}) error { + if format != snapshottypes.CurrentFormat { + return sdkerrors.Wrapf(snapshottypes.ErrUnknownFormat, "format %v", format) + } + if height == 0 { return sdkerrors.Wrap(sdkerrors.ErrLogic, "cannot restore snapshot at height 0") } @@ -52,7 +58,7 @@ func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadClos defer protoReader.Close() var subStore *substore - var storeSchemaReceived bool = false + var storeSchemaReceived = false for { item := &storetypes.SnapshotItem{} @@ -116,6 +122,10 @@ func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadClos // Snapshot implements snapshottypes.Snapshotter. func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, error) { + if format != snapshottypes.CurrentFormat { + return nil, sdkerrors.Wrapf(snapshottypes.ErrUnknownFormat, "format %v", format) + } + if height == 0 { return nil, sdkerrors.Wrap(sdkerrors.ErrLogic, "cannot snapshot height 0") } @@ -166,18 +176,27 @@ func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, e } }() - var sKeys [][]byte + var sKeys []string // sending the snapshot store schema for sKey := range vs.schema { if vs.schema[sKey] == storetypes.StoreTypePersistent { - sKeys = append(sKeys, []byte(sKey)) + sKeys = append(sKeys, sKey) } } + sort.Slice(sKeys, func(i, j int) bool { + return strings.Compare(sKeys[i], sKeys[j]) == -1 + }) + + var storeByteKeys [][]byte + for _, sKey := range sKeys { + storeByteKeys = append(storeByteKeys, []byte(sKey)) + } + err = protoWriter.WriteMsg(&storetypes.SnapshotItem{ Item: &storetypes.SnapshotItem_Schema{ Schema: &storetypes.SnapshotSchema{ - Keys: sKeys, + Keys: storeByteKeys, }, }, }) @@ -186,7 +205,7 @@ func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, e return } - for sKey := range vs.schema { + for _, sKey := range sKeys { subStore, err := vs.getSubstore(sKey) if err != nil { chunkWriter.CloseWithError(err) diff --git a/store/v2/root/snapshot_test.go b/store/v2/root/snapshot_test.go new file mode 100644 index 000000000000..d9482d8687e0 --- /dev/null +++ b/store/v2/root/snapshot_test.go @@ -0,0 +1,257 @@ +package root + +import ( + "crypto/sha256" + "encoding/binary" + "encoding/hex" + "errors" + "fmt" + dbm "github.com/cosmos/cosmos-sdk/db" + "github.com/cosmos/cosmos-sdk/db/memdb" + snapshottypes "github.com/cosmos/cosmos-sdk/snapshots/types" + "github.com/cosmos/cosmos-sdk/store/types" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "io" + "math/rand" + "sort" + "strings" + "testing" +) + +func multiStoreConfig(t *testing.T, stores int) StoreConfig { + opts := DefaultStoreConfig() + opts.Pruning = types.PruneNothing + + for i := 0; i < stores; i++ { + sKey := types.NewKVStoreKey(fmt.Sprintf("store%d", i)) + require.NoError(t, opts.ReservePrefix(sKey.Name(), types.StoreTypePersistent)) + } + + return opts +} + +func newMultiStoreWithGeneratedData(t *testing.T, db dbm.DBConnection, stores int, storeKeys uint64) *Store { + cfg := multiStoreConfig(t, stores) + store, err := NewStore(db, cfg) + require.NoError(t, err) + r := rand.New(rand.NewSource(49872768940)) // Fixed seed for deterministic tests + + var sKeys []string + for sKey := range store.schema { + sKeys = append(sKeys, sKey) + } + + sort.Slice(sKeys, func(i, j int) bool { + return strings.Compare(sKeys[i], sKeys[j]) == -1 + }) + + for _, sKey := range sKeys { + sStore, err := store.getSubstore(sKey) + require.NoError(t, err) + for i := uint64(0); i < storeKeys; i++ { + k := make([]byte, 8) + v := make([]byte, 1024) + binary.BigEndian.PutUint64(k, i) + _, err := r.Read(v) + if err != nil { + panic(err) + } + sStore.Set(k, v) + } + } + store.Commit() + return store +} + +func newMultiStoreWithBasicData(t *testing.T, db dbm.DBConnection, stores int) *Store { + cfg := multiStoreConfig(t, stores) + store, err := NewStore(db, cfg) + require.NoError(t, err) + + for sKey := range store.schema { + sStore, err := store.getSubstore(sKey) + require.NoError(t, err) + for k, v := range alohaData { + sStore.Set([]byte(k), []byte(v)) + } + } + + store.Commit() + return store +} + +func newMultiStore(t *testing.T, db dbm.DBConnection, stores int) *Store { + cfg := multiStoreConfig(t, stores) + store, err := NewStore(db, cfg) + require.NoError(t, err) + return store +} + +func TestMultistoreSnapshot_Errors(t *testing.T) { + store := newMultiStoreWithBasicData(t, memdb.NewDB(), 4) + testcases := map[string]struct { + height uint64 + format uint32 + expectType error + }{ + "0 height": {0, snapshottypes.CurrentFormat, nil}, + "0 format": {1, 0, snapshottypes.ErrUnknownFormat}, + "unknown format": {1, 9, snapshottypes.ErrUnknownFormat}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + _, err := store.Snapshot(tc.height, tc.format) + require.Error(t, err) + if tc.expectType != nil { + assert.True(t, errors.Is(err, tc.expectType)) + } + }) + } +} + +func TestMultistoreRestore_Errors(t *testing.T) { + store := newMultiStoreWithBasicData(t, memdb.NewDB(), 4) + testcases := map[string]struct { + height uint64 + format uint32 + expectType error + }{ + "0 height": {0, snapshottypes.CurrentFormat, nil}, + "0 format": {1, 0, snapshottypes.ErrUnknownFormat}, + "unknown format": {1, 9, snapshottypes.ErrUnknownFormat}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + err := store.Restore(tc.height, tc.format, nil, nil) + require.Error(t, err) + if tc.expectType != nil { + assert.True(t, errors.Is(err, tc.expectType)) + } + }) + } +} + +func TestMultistoreSnapshot_Checksum(t *testing.T) { + store := newMultiStoreWithGeneratedData(t, memdb.NewDB(), 5, 10000) + version := uint64(store.LastCommitID().Version) + + testcases := []struct { + format uint32 + chunkHashes []string + }{ + {1, []string{ + "28b9dd52156e7c46f42d6c2b390350be3c635a54446f6a6a553e1a6ecca5efca", + "8c32e05f312cf2dee6b7d2bdb41e1a2bb2372697f25504e676af1718245d8b63", + "05dfef0e32c34ef3900300f9de51f228d7fb204fa8f4e4d0d1529f083d122029", + "77d30aeeb427b0bdcedf3639adde1e822c15233d652782e171125280875aa492", + "c00c3801da889ea4370f0e647ffe1e291bd47f500e2a7269611eb4cc198b993f", + "3af4440d732225317644fa814dd8c0fb52adb7bf9046631af092af2c8cf9b512", + }}, + } + for _, tc := range testcases { + tc := tc + t.Run(fmt.Sprintf("Format %v", tc.format), func(t *testing.T) { + chunks, err := store.Snapshot(version, tc.format) + require.NoError(t, err) + hashes := []string{} + hasher := sha256.New() + for chunk := range chunks { + hasher.Reset() + _, err := io.Copy(hasher, chunk) + require.NoError(t, err) + hashes = append(hashes, hex.EncodeToString(hasher.Sum(nil))) + } + assert.Equal(t, tc.chunkHashes, hashes, "Snapshot output for format %v has changed", tc.format) + }) + } +} + +func TestMultistoreSnapshotRestore(t *testing.T) { + source := newMultiStoreWithBasicData(t, memdb.NewDB(), 4) + target := newMultiStore(t, memdb.NewDB(), 4) + require.Equal(t, source.LastCommitID().Version, int64(1)) + version := uint64(source.LastCommitID().Version) + chunks, err := source.Snapshot(version, snapshottypes.CurrentFormat) + require.NoError(t, err) + ready := make(chan struct{}) + err = target.Restore(version, snapshottypes.CurrentFormat, chunks, ready) + require.NoError(t, err) + assert.EqualValues(t, struct{}{}, <-ready) + + assert.Equal(t, source.LastCommitID(), target.LastCommitID()) + + for sKey := range source.schema { + sourceSubStore, err := source.getSubstore(sKey) + require.NoError(t, err) + targetSubStore, err := target.getSubstore(sKey) + require.NoError(t, err) + require.Equal(t, sourceSubStore, targetSubStore) + } +} + +func BenchmarkMultistoreSnapshot100K(b *testing.B) { + benchmarkMultistoreSnapshot(b, 10, 10000) +} + +func BenchmarkMultistoreSnapshot1M(b *testing.B) { + benchmarkMultistoreSnapshot(b, 10, 100000) +} + +func BenchmarkMultistoreSnapshotRestore100K(b *testing.B) { + benchmarkMultistoreSnapshotRestore(b, 10, 10000) +} + +func BenchmarkMultistoreSnapshotRestore1M(b *testing.B) { + benchmarkMultistoreSnapshotRestore(b, 10, 100000) +} + +func benchmarkMultistoreSnapshot(b *testing.B, stores int, storeKeys uint64) { + b.Skip("Noisy with slow setup time, please see https://github.com/cosmos/cosmos-sdk/issues/8855.") + + b.ReportAllocs() + b.StopTimer() + source := newMultiStoreWithGeneratedData(nil, memdb.NewDB(), stores, storeKeys) + + version := source.LastCommitID().Version + require.EqualValues(b, 1, version) + b.StartTimer() + + for i := 0; i < b.N; i++ { + target := newMultiStore(nil, memdb.NewDB(), stores) + require.EqualValues(b, 0, target.LastCommitID().Version) + + chunks, err := source.Snapshot(uint64(version), snapshottypes.CurrentFormat) + require.NoError(b, err) + for reader := range chunks { + _, err := io.Copy(io.Discard, reader) + require.NoError(b, err) + err = reader.Close() + require.NoError(b, err) + } + } +} + +func benchmarkMultistoreSnapshotRestore(b *testing.B, stores int, storeKeys uint64) { + b.Skip("Noisy with slow setup time, please see https://github.com/cosmos/cosmos-sdk/issues/8855.") + + b.ReportAllocs() + b.StopTimer() + source := newMultiStoreWithGeneratedData(nil, memdb.NewDB(), stores, storeKeys) + version := uint64(source.LastCommitID().Version) + require.EqualValues(b, 1, version) + b.StartTimer() + + for i := 0; i < b.N; i++ { + target := newMultiStore(nil, memdb.NewDB(), stores) + require.EqualValues(b, 0, target.LastCommitID().Version) + + chunks, err := source.Snapshot(version, snapshottypes.CurrentFormat) + require.NoError(b, err) + err = target.Restore(version, snapshottypes.CurrentFormat, chunks, nil) + require.NoError(b, err) + require.Equal(b, source.LastCommitID(), target.LastCommitID()) + } +} From db80903aa954125cfbcaef1cb01337d2b5ca2a36 Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Tue, 14 Dec 2021 14:15:41 +0530 Subject: [PATCH 44/47] test: add test for checking restore for non-empty store --- store/v2/root/snapshot_test.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/store/v2/root/snapshot_test.go b/store/v2/root/snapshot_test.go index d9482d8687e0..2dcf8c9dd8f1 100644 --- a/store/v2/root/snapshot_test.go +++ b/store/v2/root/snapshot_test.go @@ -171,7 +171,7 @@ func TestMultistoreSnapshot_Checksum(t *testing.T) { func TestMultistoreSnapshotRestore(t *testing.T) { source := newMultiStoreWithBasicData(t, memdb.NewDB(), 4) - target := newMultiStore(t, memdb.NewDB(), 4) + target := newMultiStore(t, memdb.NewDB(), 0) require.Equal(t, source.LastCommitID().Version, int64(1)) version := uint64(source.LastCommitID().Version) chunks, err := source.Snapshot(version, snapshottypes.CurrentFormat) @@ -190,6 +190,12 @@ func TestMultistoreSnapshotRestore(t *testing.T) { require.NoError(t, err) require.Equal(t, sourceSubStore, targetSubStore) } + + // restore checking for non-empty store + target2 := newMultiStoreWithBasicData(t, memdb.NewDB(), 0) + ready2 := make(chan struct{}) + err = target2.Restore(version, snapshottypes.CurrentFormat, chunks, ready2) + require.Error(t, err) } func BenchmarkMultistoreSnapshot100K(b *testing.B) { From 58e47950c682298def4d1b45c897b0bd76165d45 Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Tue, 14 Dec 2021 16:25:09 +0530 Subject: [PATCH 45/47] test: add test for checking restore for store with existed schema --- store/v2/root/snapshot.go | 8 ++++++-- store/v2/root/snapshot_test.go | 17 ++++++++++++++--- 2 files changed, 20 insertions(+), 5 deletions(-) diff --git a/store/v2/root/snapshot.go b/store/v2/root/snapshot.go index feb48762be93..c41bb55f1ad2 100644 --- a/store/v2/root/snapshot.go +++ b/store/v2/root/snapshot.go @@ -46,7 +46,7 @@ func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadClos } // Set up a restore stream pipeline - // chan io.ReadCloser -> chunkReader -> zlib -> delimited Protobuf -> ExportNode + // chan io.ReadCloser -> chunkReader -> zlib -> delimited Protobuf -> Exported KV Item chunkReader := snapshots.NewChunkReader(chunks) defer chunkReader.Close() zReader, err := zlib.NewReader(chunkReader) @@ -71,6 +71,10 @@ func (rs *Store) Restore(height uint64, format uint32, chunks <-chan io.ReadClos switch item := item.Item.(type) { case *storetypes.SnapshotItem_Schema: + if len(rs.schema) != 0 { + return sdkerrors.Wrap(sdkerrors.ErrLogic, "store schema is not empty") + } + storeSchemaReceived = true schemaWriter := prefixdb.NewPrefixWriter(rs.stateTxn, schemaPrefix) sKeys := item.Schema.GetKeys() @@ -150,7 +154,7 @@ func (rs *Store) Snapshot(height uint64, format uint32) (<-chan io.ReadCloser, e ch := make(chan io.ReadCloser) go func() { // Set up a stream pipeline to serialize snapshot nodes: - // ExportNode -> delimited Protobuf -> zlib -> buffer -> chunkWriter -> chan io.ReadCloser + // Export KV Item -> delimited Protobuf -> zlib -> buffer -> chunkWriter -> chan io.ReadCloser chunkWriter := snapshots.NewChunkWriter(ch, snapshotChunkSize) defer chunkWriter.Close() bufWriter := bufio.NewWriterSize(chunkWriter, snapshotBufferSize) diff --git a/store/v2/root/snapshot_test.go b/store/v2/root/snapshot_test.go index 2dcf8c9dd8f1..f0bedbae4482 100644 --- a/store/v2/root/snapshot_test.go +++ b/store/v2/root/snapshot_test.go @@ -170,10 +170,13 @@ func TestMultistoreSnapshot_Checksum(t *testing.T) { } func TestMultistoreSnapshotRestore(t *testing.T) { - source := newMultiStoreWithBasicData(t, memdb.NewDB(), 4) + source := newMultiStoreWithGeneratedData(t, memdb.NewDB(), 3, 4) target := newMultiStore(t, memdb.NewDB(), 0) require.Equal(t, source.LastCommitID().Version, int64(1)) version := uint64(source.LastCommitID().Version) + // check for target store restore + require.Equal(t, target.LastCommitID().Version, int64(0)) + chunks, err := source.Snapshot(version, snapshottypes.CurrentFormat) require.NoError(t, err) ready := make(chan struct{}) @@ -191,11 +194,19 @@ func TestMultistoreSnapshotRestore(t *testing.T) { require.Equal(t, sourceSubStore, targetSubStore) } - // restore checking for non-empty store - target2 := newMultiStoreWithBasicData(t, memdb.NewDB(), 0) + // checking snapshot restore for non-empty store with existed saved version + target2 := newMultiStoreWithBasicData(t, memdb.NewDB(), 4) ready2 := make(chan struct{}) err = target2.Restore(version, snapshottypes.CurrentFormat, chunks, ready2) require.Error(t, err) + + // checking snapshot restoring for store with existed schema and without existed versions + target3 := newMultiStore(t, memdb.NewDB(), 4) + ready3 := make(chan struct{}) + chunks, err = source.Snapshot(version, snapshottypes.CurrentFormat) + require.NoError(t, err) + err = target3.Restore(version, snapshottypes.CurrentFormat, chunks, ready3) + require.Error(t, err) } func BenchmarkMultistoreSnapshot100K(b *testing.B) { From 21715e9b199c88532ec892901d2179f754948e49 Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Tue, 14 Dec 2021 19:06:10 +0530 Subject: [PATCH 46/47] address the pr comments --- store/v2/root/snapshot_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/store/v2/root/snapshot_test.go b/store/v2/root/snapshot_test.go index f0bedbae4482..27afaa28d788 100644 --- a/store/v2/root/snapshot_test.go +++ b/store/v2/root/snapshot_test.go @@ -194,14 +194,14 @@ func TestMultistoreSnapshotRestore(t *testing.T) { require.Equal(t, sourceSubStore, targetSubStore) } - // checking snapshot restore for non-empty store with existed saved version + // checking snapshot restore for non-empty store with existing saved version target2 := newMultiStoreWithBasicData(t, memdb.NewDB(), 4) ready2 := make(chan struct{}) err = target2.Restore(version, snapshottypes.CurrentFormat, chunks, ready2) require.Error(t, err) - // checking snapshot restoring for store with existed schema and without existed versions - target3 := newMultiStore(t, memdb.NewDB(), 4) + // checking snapshot restoring for store with existed schema and without existing versions + target3 := newMultiStore(t, memdb.NewDB(), 0) ready3 := make(chan struct{}) chunks, err = source.Snapshot(version, snapshottypes.CurrentFormat) require.NoError(t, err) From 34311309d3640b8090b6cbcf993d9dbcfcd71a8c Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Tue, 14 Dec 2021 19:17:52 +0530 Subject: [PATCH 47/47] address the pr comments --- store/v2/root/snapshot_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/store/v2/root/snapshot_test.go b/store/v2/root/snapshot_test.go index 27afaa28d788..9bb29db33583 100644 --- a/store/v2/root/snapshot_test.go +++ b/store/v2/root/snapshot_test.go @@ -194,14 +194,14 @@ func TestMultistoreSnapshotRestore(t *testing.T) { require.Equal(t, sourceSubStore, targetSubStore) } - // checking snapshot restore for non-empty store with existing saved version - target2 := newMultiStoreWithBasicData(t, memdb.NewDB(), 4) + // checking snapshot restore for store with existing saved version + target2 := newMultiStoreWithBasicData(t, memdb.NewDB(), 0) ready2 := make(chan struct{}) err = target2.Restore(version, snapshottypes.CurrentFormat, chunks, ready2) require.Error(t, err) // checking snapshot restoring for store with existed schema and without existing versions - target3 := newMultiStore(t, memdb.NewDB(), 0) + target3 := newMultiStore(t, memdb.NewDB(), 4) ready3 := make(chan struct{}) chunks, err = source.Snapshot(version, snapshottypes.CurrentFormat) require.NoError(t, err)