From 0b732a44abe67f1eb4c606aa01cf524e52c6fa95 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Mon, 30 Aug 2021 23:34:58 +0800 Subject: [PATCH 01/45] 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 00000000000..ba1f4af8fcb --- /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/45] 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 00000000000..2f0ceb5df54 --- /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 00000000000..9f8ac71b25c --- /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/45] 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 00000000000..6b2665839c3 --- /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 00000000000..51ab85cff01 --- /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 00000000000..e46ce085825 --- /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/45] 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 00000000000..e7ac348b922 --- /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/45] 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 6cd06b6a2aa..b00ff684939 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 00000000000..6206a3dc6f9 --- /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/45] 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 6116c79203d..3f97a91fa92 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 2f0ceb5df54..152314de811 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 00000000000..75f0f490aee --- /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 00000000000..de26f9fdd14 --- /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 b00ff684939..926b4688c53 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 c3f559b31c2..f6fd5e59daa 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 6206a3dc6f9..76b57869885 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 ba1f4af8fcb..6762eed69cf 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/45] 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 75f0f490aee..dc1baf01cac 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 926b4688c53..64ba923080f 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 f6fd5e59daa..1fda4c52a55 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 76b57869885..bcc5e08be39 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/45] 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 dc1baf01cac..2a88374b544 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 de26f9fdd14..bc10e7e36b8 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 64ba923080f..a177f43833f 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 1fda4c52a55..e70974c3f98 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 bcc5e08be39..a5ed1cded9b 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/45] 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 afded48f76a..34fcb57f4e5 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/45] 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 7af13914bc8..19d7f258617 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 6762eed69cf..f47376ba5cc 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/45] 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 ce413017433..953a08b297e 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 1bb18fb13fd..9a7c9192a07 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/45] 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 e7ac348b922..3ce99e12f62 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 295278a0a85..941c920f48c 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 00000000000..e1b817c4eb7 --- /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 2a88374b544..00000000000 --- 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 bc10e7e36b8..00000000000 --- 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 a177f43833f..0d8abaa4785 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 e70974c3f98..d2c9a38cf27 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 a5ed1cded9b..00000000000 --- 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 00000000000..f41b31115d4 --- /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 00000000000..7f75957f4d0 --- /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 f47376ba5cc..743bb91eccc 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/45] 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 3f97a91fa92..af4ae2aa10f 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 152314de811..fa59d90fc5f 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 e1b817c4eb7..00000000000 --- 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 0d8abaa4785..d7d2d07032d 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 d2c9a38cf27..b7078fcd538 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 00000000000..e11e8b0d544 --- /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 f41b31115d4..2fcdbd2f09e 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 7f75957f4d0..ae02296d6dd 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/45] 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 ef67961d2a5..9b40475d5e5 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 34fcb57f4e5..54c636ba2cf 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/45] 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 941c920f48c..295278a0a85 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/45] 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 313bfde5b43..d28aa1cae5f 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/45] 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 d28aa1cae5f..5e8bae42510 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/45] 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 d7d2d07032d..a2b0cc4d795 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/45] 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 a2b0cc4d795..e4e8ac72051 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/45] 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 b656d60edad..ddae7f74db8 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 8541f69673a..578d1e5f8dd 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 b7078fcd538..385c8b162fd 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/45] 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 af4ae2aa10f..66f90ef7b24 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 fa59d90fc5f..1cbd6c83585 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 9f8ac71b25c..cc78d8ac850 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/45] 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 990d22ae835..973807324df 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 cc78d8ac850..a5ca7510847 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 00000000000..269094f7d0d --- /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/45] 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 e4e8ac72051..076bc2afbe0 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 ae02296d6dd..b0f5438ed7e 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 743bb91eccc..1ab79e16406 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/45] 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 076bc2afbe0..0258985fae7 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/45] 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 385c8b162fd..228a2a58d86 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 2fcdbd2f09e..777e59cc2b0 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/45] 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 54c636ba2cf..99123a5a3f8 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/45] 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 0258985fae7..fdef6ee2ff3 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 b0f5438ed7e..85eaeb6e7cf 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 1ab79e16406..7e9db7697e2 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/45] 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 66f90ef7b24..38a8e003474 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 00000000000..0d49e765521 --- /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 fdef6ee2ff3..a3652145474 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/45] 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 38a8e003474..52f044d18b0 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 00000000000..57ccb28931d --- /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 a3652145474..7d49535d6fb 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 953a08b297e..b63d0e65ecd 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/45] 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 632b561b618..a1c8ea90ba9 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 6b2665839c3..88d4697ca1d 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 00000000000..822f5eaeb34 --- /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 7d49535d6fb..5f9e5ef1adf 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 51ab85cff01..1c68e15e169 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 e46ce085825..4b051245be8 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/45] 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 5f9e5ef1adf..8c529410f56 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/45] 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 57ccb28931d..5630acfc6db 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 8c529410f56..ad1b89ac8cb 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 228a2a58d86..7b5caaf67d3 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/45] 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 7b5caaf67d3..958f8e300f9 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/45] 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 9b40475d5e5..03cec141976 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 99123a5a3f8..1a688a5df05 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 5630acfc6db..915ea156c20 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 0d49e765521..76469ab11a6 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 ad1b89ac8cb..1d94a23eaf2 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 958f8e300f9..c631ad9d4cb 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 7e9db7697e2..6975cbdc49c 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/45] 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 e6294909c48..5494d719c1d 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 973807324df..3c4b844eb48 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 e78d71eaa0e..54be2ec2add 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 4b69172b517..058bf9a5fc8 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 5e8bae42510..d1d8f192679 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 2b7f7b74e48..c0b83778e20 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/45] 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 5494d719c1d..e6294909c48 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 86998b5aa99..b6be64240e3 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/45] 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 b745bfd6cbe..0010c3c5c89 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 e7fef9f14ff..ed305b1961d 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 61bddc6767e9e9752a4e752c82ac49042cfc5739 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Wed, 8 Dec 2021 17:12:05 +0800 Subject: [PATCH 38/45] godoc --- store/v2/root/store.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/store/v2/root/store.go b/store/v2/root/store.go index 1d94a23eaf2..36786d76225 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -474,7 +474,8 @@ func (rs *Store) GetKVStore(skey types.StoreKey) types.KVStore { return rs.wrapTraceListen(ret, skey) } -// Gets a persistent substore. This reads, but does not update the substore cache +// Gets a persistent substore. This reads, but does not update the substore cache. +// Use it in cases where we need to access a store internally (e.g. read/write Merkle keys, queries) func (rs *Store) getSubstore(key string) (*substore, error) { if cached, has := rs.substoreCache[key]; has { return cached, nil From ac8abb48c041a53e07d75ff2527f7d2536dd431c Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 9 Dec 2021 15:01:47 +0800 Subject: [PATCH 39/45] clean up, nits --- store/types/store.go | 3 +++ store/v2/root/store.go | 16 ++++++++-------- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/store/types/store.go b/store/types/store.go index 19d7f258617..bbf1b875803 100644 --- a/store/types/store.go +++ b/store/types/store.go @@ -319,6 +319,9 @@ func (st StoreType) String() string { case StoreTypeSMT: return "StoreTypeSMT" + + case StoreTypePersistent: + return "StoreTypePersistent" } return "unknown store type" diff --git a/store/v2/root/store.go b/store/v2/root/store.go index 36786d76225..3962c67e2b8 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -240,13 +240,13 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { }() stateCommitmentTxn := stateTxn if opts.StateCommitmentDB != nil { - var mversions dbm.VersionSet - mversions, err = opts.StateCommitmentDB.Versions() + var scVersions dbm.VersionSet + scVersions, err = opts.StateCommitmentDB.Versions() if err != nil { return } // Version sets of each DB must match - if !versions.Equal(mversions) { + if !versions.Equal(scVersions) { err = fmt.Errorf("Storage and StateCommitment DB have different version history") //nolint:stylecheck return } @@ -262,6 +262,8 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { stateTxn: stateTxn, StateCommitmentDB: opts.StateCommitmentDB, stateCommitmentTxn: stateCommitmentTxn, + mem: mem.NewStore(), + tran: transient.NewStore(), substoreCache: map[string]*substore{}, @@ -284,7 +286,7 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { if err != nil { return } - // If the loaded schema is empty, just copy the config schema; + // If the loaded schema is empty (for new store), 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 { @@ -331,8 +333,6 @@ func NewStore(db dbm.DBConnection, opts StoreConfig) (ret *Store, err error) { return } } - ret.mem = mem.NewStore() - ret.tran = transient.NewStore() ret.schema = reg.StoreSchema return } @@ -530,7 +530,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 } @@ -560,7 +560,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) From 438d384e80903068da419ae67b4c21c48e241bb5 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 9 Dec 2021 18:37:36 +0800 Subject: [PATCH 40/45] PR revisions --- CHANGELOG.md | 2 +- internal/db/iterator_adapter.go | 2 +- store/v2/dbadapter/store_test.go | 1 - store/v2/root/cache_store.go | 3 +-- 4 files changed, 3 insertions(+), 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 308766791e7..a7556424138 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -53,7 +53,7 @@ Ref: https://keepachangelog.com/en/1.0.0/ * [\#10208](https://github.com/cosmos/cosmos-sdk/pull/10208) Add `TipsTxMiddleware` for transferring tips. * [\#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. +* [\#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 store/v2 `MultiStore` implementation ### Improvements diff --git a/internal/db/iterator_adapter.go b/internal/db/iterator_adapter.go index 3ce99e12f62..6e7f366249f 100644 --- a/internal/db/iterator_adapter.go +++ b/internal/db/iterator_adapter.go @@ -16,7 +16,7 @@ type dbAsStoreIter struct { // (store/types).Iterator interface. func DBToStoreIterator(source dbm.Iterator) *dbAsStoreIter { ret := &dbAsStoreIter{Iterator: source} - ret.Next() + ret.Next() // The DB iterator must be primed before it can access the first element, because Next also returns the validity status return ret } diff --git a/store/v2/dbadapter/store_test.go b/store/v2/dbadapter/store_test.go index a5ca7510847..f28fa5f0655 100644 --- a/store/v2/dbadapter/store_test.go +++ b/store/v2/dbadapter/store_test.go @@ -29,7 +29,6 @@ func TestAccessors(t *testing.T) { 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) diff --git a/store/v2/root/cache_store.go b/store/v2/root/cache_store.go index 915ea156c20..58e261c2020 100644 --- a/store/v2/root/cache_store.go +++ b/store/v2/root/cache_store.go @@ -20,9 +20,8 @@ func (cs *cacheStore) GetKVStore(skey types.StoreKey) types.KVStore { // Write implements CacheMultiStore. func (cs *cacheStore) Write() { - for skey, sub := range cs.substores { + for _, sub := range cs.substores { sub.Write() - delete(cs.substores, skey) } } From 02a6fb1b42c5821257f7ff471b0a5c28d352a7a0 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Fri, 10 Dec 2021 13:53:21 +0800 Subject: [PATCH 41/45] use t.Run more, some cleanup --- store/v2/root/store_test.go | 496 +++++++++++++++++++----------------- 1 file changed, 259 insertions(+), 237 deletions(-) diff --git a/store/v2/root/store_test.go b/store/v2/root/store_test.go index c631ad9d4cb..aa9576019b1 100644 --- a/store/v2/root/store_test.go +++ b/store/v2/root/store_test.go @@ -100,52 +100,58 @@ func TestConstructors(t *testing.T) { store.Commit() require.NoError(t, store.Close()) - // Loading with an initial version beyond the lowest should error - opts := StoreConfig{InitialVersion: 5, Pruning: types.PruneNothing} - store, err = NewStore(db, opts) - require.Error(t, err) - db.Close() + t.Run("fail to load if InitialVersion > lowest existing version", func(t *testing.T) { + opts := StoreConfig{InitialVersion: 5, Pruning: types.PruneNothing} + store, err = NewStore(db, opts) + require.Error(t, err) + db.Close() + }) - store, err = NewStore(dbVersionsFails{memdb.NewDB()}, DefaultStoreConfig()) - require.Error(t, err) - store, err = NewStore(db, StoreConfig{StateCommitmentDB: dbVersionsFails{memdb.NewDB()}}) - require.Error(t, err) + t.Run("can't load store when db.Versions fails", func(t *testing.T) { + store, err = NewStore(dbVersionsFails{memdb.NewDB()}, DefaultStoreConfig()) + require.Error(t, err) + store, err = NewStore(db, StoreConfig{StateCommitmentDB: dbVersionsFails{memdb.NewDB()}}) + require.Error(t, err) + }) - // can't use a DB with open writers db = memdb.NewDB() merkledb := memdb.NewDB() w := db.Writer() - store, err = NewStore(db, DefaultStoreConfig()) - require.Error(t, err) - w.Discard() - w = merkledb.Writer() - 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{StateCommitmentDB: merkledb}) - require.Error(t, err) + t.Run("can't use a DB with open writers", func(t *testing.T) { + store, err = NewStore(db, DefaultStoreConfig()) + require.Error(t, err) + w.Discard() + w = merkledb.Writer() + store, err = NewStore(db, StoreConfig{StateCommitmentDB: merkledb}) + require.Error(t, err) + w.Discard() + }) + + t.Run("can't use DBs with different version history", func(t *testing.T) { + merkledb.SaveNextVersion() + store, err = NewStore(db, StoreConfig{StateCommitmentDB: merkledb}) + require.Error(t, err) + }) merkledb.Close() - // can't load existing store when we can't access the latest Merkle root hash - 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() - s1RootKey := append(contentPrefix, substorePrefix(skey_1.Name())...) - s1RootKey = append(s1RootKey, merkleRootKey...) - w.Delete(s1RootKey) - w.Commit() - db.SaveNextVersion() - store, err = NewStore(db, DefaultStoreConfig()) - require.Error(t, err) - // ...or, because of an error - store, err = NewStore(dbRWCrudFails{db}, DefaultStoreConfig()) - require.Error(t, err) + t.Run("can't load existing store if we can't access root hash", func(t *testing.T) { + 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() + s1RootKey := append(contentPrefix, substorePrefix(skey_1.Name())...) + s1RootKey = append(s1RootKey, merkleRootKey...) + w.Delete(s1RootKey) + w.Commit() + db.SaveNextVersion() + store, err = NewStore(db, DefaultStoreConfig()) + require.Error(t, err) + // ...or, because of an error + store, err = NewStore(dbRWCrudFails{db}, DefaultStoreConfig()) + require.Error(t, err) + }) } func TestIterators(t *testing.T) { @@ -211,7 +217,6 @@ func TestIterators(t *testing.T) { func TestCommit(t *testing.T) { testBasic := func(opts StoreConfig) { - // Sanity test for Merkle hashing db := memdb.NewDB() store, err := NewStore(db, opts) require.NoError(t, err) @@ -243,10 +248,15 @@ func TestCommit(t *testing.T) { } basicOpts := simpleStoreConfig(t) basicOpts.Pruning = types.PruneNothing - testBasic(basicOpts) - basicOpts.StateCommitmentDB = memdb.NewDB() - testBasic(basicOpts) + t.Run("sanity tests for Merkle hashing", func(t *testing.T) { + testBasic(basicOpts) + }) + t.Run("sanity tests for Merkle hashing with separate DBs", func(t *testing.T) { + basicOpts.StateCommitmentDB = memdb.NewDB() + testBasic(basicOpts) + }) + // test that we can recover from a failed commit testFailedCommit := func(t *testing.T, store *Store, db dbm.DBConnection, @@ -347,17 +357,19 @@ func TestCommit(t *testing.T) { require.Equal(t, int64(math.MaxInt64), store.LastCommitID().Version) // version history not modified }) - // setting initial version - 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) + t.Run("first commit version matches InitialVersion", func(t *testing.T) { + 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) + }) + // test improbable failures to fill out test coverage opts = simpleStoreConfig(t) - store, err = NewStore(memdb.NewDB(), opts) + store, err := NewStore(memdb.NewDB(), opts) require.NoError(t, err) store.Commit() store.stateDB = dbVersionsFails{store.stateDB} @@ -490,6 +502,7 @@ func TestQuery(t *testing.T) { ver := cid.Version query := abci.RequestQuery{Path: queryPath(skey_1, "/key"), Data: k1, Height: ver} querySub := abci.RequestQuery{Path: queryPath(skey_1, "/subspace"), Data: ksub, Height: ver} + queryHeight0 := abci.RequestQuery{Path: queryPath(skey_1, "/key"), Data: k1} // query subspace before anything set qres := store.Query(querySub) @@ -502,120 +515,125 @@ func TestQuery(t *testing.T) { 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(), 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(), qres.Log) - require.Nil(t, qres.Value) + t.Run("basic queries", func(t *testing.T) { + // set data without commit, doesn't show up + qres = store.Query(query) + 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(), 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(), qres.Log) - require.Equal(t, valExpSub1, qres.Value) + // commit it, but still don't see on old version + cid = store.Commit() + qres = store.Query(query) + require.True(t, qres.IsOK(), qres.Log) + require.Nil(t, qres.Value) - // modify - sub.Set(k1, v3) - cid = store.Commit() + // but yes on the new version + query.Height = cid.Version + qres = store.Query(query) + 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(), qres.Log) + require.Equal(t, valExpSub1, qres.Value) - // query will return old values, as height is fixed - qres = store.Query(query) - require.True(t, qres.IsOK(), qres.Log) - require.Equal(t, v1, qres.Value) + // modify + sub.Set(k1, v3) + cid = store.Commit() - // update to latest in the query and we are happy - query.Height = cid.Version - qres = store.Query(query) - require.True(t, qres.IsOK(), qres.Log) - require.Equal(t, v3, qres.Value) + // query will return old values, as height is fixed + qres = store.Query(query) + require.True(t, qres.IsOK(), qres.Log) + require.Equal(t, v1, qres.Value) - 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) - // and for the subspace - querySub.Height = cid.Version - qres = store.Query(querySub) - require.True(t, qres.IsOK(), qres.Log) - require.Equal(t, valExpSub2, qres.Value) + // update to latest height in the query and we are happy + query.Height = cid.Version + qres = store.Query(query) + require.True(t, qres.IsOK(), qres.Log) + require.Equal(t, v3, qres.Value) + // try other key + 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) + // and for the subspace + querySub.Height = cid.Version + qres = store.Query(querySub) + require.True(t, qres.IsOK(), qres.Log) + require.Equal(t, valExpSub2, qres.Value) - // default (height 0) will show latest-1 - 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) + // default (height 0) will show latest-1 + qres = store.Query(queryHeight0) + require.True(t, qres.IsOK(), qres.Log) + require.Equal(t, v1, qres.Value) + }) // querying an empty store will fail store2, err := NewStore(memdb.NewDB(), simpleStoreConfig(t)) require.NoError(t, err) - qres = store2.Query(query0) + qres = store2.Query(queryHeight0) require.True(t, qres.IsErr()) // default shows latest, if latest-1 does not exist store2.GetKVStore(skey_1).Set(k1, v1) store2.Commit() - qres = store2.Query(query0) + qres = store2.Query(queryHeight0) require.True(t, qres.IsOK(), qres.Log) require.Equal(t, v1, qres.Value) store2.Close() - // artificial error cases for coverage (should never happen with defined usage) - // ensure that height overflow triggers an error - require.NoError(t, err) - store2.stateDB = dbVersionsIs{store2.stateDB, dbm.NewVersionManager([]uint64{uint64(math.MaxInt64) + 1})} - qres = store2.Query(query0) - require.True(t, qres.IsErr()) - // failure to access versions triggers an error - store2.stateDB = dbVersionsFails{store.stateDB} - qres = store2.Query(query0) - require.True(t, qres.IsErr()) - store2.Close() - - // query with a nil or empty key fails - 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: 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: queryPath(skey_1, "/badpath"), Data: k1} - qres = store.Query(badquery) - require.True(t, qres.IsErr()) + t.Run("failed queries", func(t *testing.T) { + // artificial error cases for coverage (should never happen with prescribed usage) + // ensure that height overflow triggers an error + require.NoError(t, err) + store2.stateDB = dbVersionsIs{store2.stateDB, dbm.NewVersionManager([]uint64{uint64(math.MaxInt64) + 1})} + qres = store2.Query(queryHeight0) + require.True(t, qres.IsErr()) + // failure to access versions triggers an error + store2.stateDB = dbVersionsFails{store.stateDB} + qres = store2.Query(queryHeight0) + require.True(t, qres.IsErr()) + store2.Close() + + // query with a nil or empty key fails + 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: 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: 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: 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) - require.NotNil(t, qres.ProofOps) - } - testProve() - store.Close() + t.Run("queries with proof", func(t *testing.T) { + // test that proofs are generated with single and separate DBs + testProve := func() { + 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) + require.NotNil(t, qres.ProofOps) + } + testProve() + store.Close() - opts := simpleStoreConfig(t) - opts.StateCommitmentDB = memdb.NewDB() - store, err = NewStore(memdb.NewDB(), opts) - require.NoError(t, err) - store.GetKVStore(skey_1).Set(k1, v1) - store.Commit() - testProve() - store.Close() + opts := simpleStoreConfig(t) + opts.StateCommitmentDB = memdb.NewDB() + store, err = NewStore(memdb.NewDB(), opts) + require.NoError(t, err) + store.GetKVStore(skey_1).Set(k1, v1) + store.Commit() + testProve() + store.Close() + }) } func TestStoreConfig(t *testing.T) { @@ -672,7 +690,7 @@ func TestGetVersion(t *testing.T) { substore := store.GetKVStore(skey_1) require.NotNil(t, substore) substore.Set([]byte{0}, []byte{0}) - // setting a new value shouldn't affect old version + // setting a value shouldn't affect old version require.False(t, subview.Has([]byte{0})) cid = store.Commit() @@ -680,7 +698,7 @@ func TestGetVersion(t *testing.T) { require.NoError(t, err) subview = view.GetKVStore(skey_1) require.NotNil(t, subview) - + // deleting a value shouldn't affect old version substore.Delete([]byte{0}) require.Equal(t, []byte{0}, subview.Get([]byte{0})) } @@ -707,10 +725,12 @@ func TestMultiStoreMigration(t *testing.T) { require.NotNil(t, s3) s3.Set(k3, v3) + k4, v4 := []byte("fourth"), []byte("created") require.Panics(t, func() { store.GetKVStore(skey_4) }) cid := store.Commit() require.NoError(t, store.Close()) + var migratedID types.CommitID // Load without changes and make sure it is sensible store, err = NewStore(db, opts) @@ -722,104 +742,106 @@ func TestMultiStoreMigration(t *testing.T) { 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 valid - 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 valid - 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()) + t.Run("basic migration", func(t *testing.T) { + // 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()}, + }, + } + store, err = NewStore(db, opts) + require.Nil(t, err) - // query this new store - rl1 := reload.GetKVStore(skey_1) - require.NotNil(t, rl1) - require.Equal(t, v1, rl1.Get(k1)) + // s1 was not changed + s1 = store.GetKVStore(skey_1) + require.NotNil(t, s1) + require.Equal(t, v1, s1.Get(k1)) + + // store2 is no longer valid + require.Panics(t, func() { store.GetKVStore(skey_2) }) + // store2b has the old data + rs2 := store.GetKVStore(skey_2b) + require.NotNil(t, rs2) + require.Equal(t, v2, rs2.Get(k2)) + + // store3 is gone + require.Panics(t, func() { s3 = store.GetKVStore(skey_3) }) + + // store4 is valid + s4 := store.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 + s4.Set(k4, v4) + + // store this migrated data, and load it again without migrations + migratedID = store.Commit() + require.Equal(t, migratedID.Version, int64(2)) + require.NoError(t, store.Close()) + }) - rl2 := reload.GetKVStore(skey_2b) - require.NotNil(t, rl2) - require.Equal(t, v2, rl2.Get(k2)) + t.Run("reload after migrations", func(t *testing.T) { + // fail to load the migrated store with the old schema + store, err = NewStore(db, storeConfig123(t)) + require.Error(t, err) - rl4 := reload.GetKVStore(skey_4) - require.NotNil(t, rl4) - require.Equal(t, v4, rl4.Get(k4)) + // 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) + store, err = NewStore(db, migratedOpts) + require.Nil(t, err) + require.Equal(t, migratedID, store.LastCommitID()) + + // query this new store + rl1 := store.GetKVStore(skey_1) + require.NotNil(t, rl1) + require.Equal(t, v1, rl1.Get(k1)) + + rl2 := store.GetKVStore(skey_2b) + require.NotNil(t, rl2) + require.Equal(t, v2, rl2.Get(k2)) + + rl4 := store.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) + t.Run("load view from before migrations", func(t *testing.T) { + // 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)) + 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)) + 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)) + s3 = view.GetKVStore(skey_3) + 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) + }) }) } From db7501ee64a1041cd1f442ab3c9845da15a0c003 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 16 Dec 2021 00:04:54 +0800 Subject: [PATCH 42/45] Apply suggestions from code review Co-authored-by: Ian Norden --- docs/core/store.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/core/store.md b/docs/core/store.md index 1a688a5df05..b3b3292fb75 100644 --- a/docs/core/store.md +++ b/docs/core/store.md @@ -242,14 +242,14 @@ See [ADR-040](../architecture/adr-040-storage-and-smt-state-commitments.md) for ## `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`. +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 `root.Store`. ## 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. + * The set of valid substores is defined at initialization and cannot be updated dynamically in an existing store instance. ### `CommitMultiStore` From 746fb298b66d8ab18841a2b7dc3663ad7080db5e Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 16 Dec 2021 00:31:21 +0800 Subject: [PATCH 43/45] revision --- docs/core/store.md | 14 +++++++------- store/v2/root/store.go | 10 +++------- 2 files changed, 10 insertions(+), 14 deletions(-) diff --git a/docs/core/store.md b/docs/core/store.md index b3b3292fb75..ae79aa8ddc8 100644 --- a/docs/core/store.md +++ b/docs/core/store.md @@ -236,7 +236,7 @@ When `KVStore.Set` or `KVStore.Delete` methods are called, `listenkv.Store` auto 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. -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`). +These types use the new `db` sub-module of Cosmos-SDK (`github.com/cosmos/cosmos-sdk/db`), rather than `tmdb` (`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. @@ -246,7 +246,7 @@ An interface providing only the basic CRUD functionality (`Get`, `Set`, `Has`, a ## 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`: +This is the new interface (or, set of interfaces) for the main client store, replacing the role of `store/types.MultiStore` (v1). There are a few significant differences in behavior compared with v1: * 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 at initialization and cannot be updated dynamically in an existing store instance. @@ -254,21 +254,21 @@ This is the new interface (or set of interfaces) for the main client store, repl ### `CommitMultiStore` This is the main interface for persisent application state, analogous to the original `CommitMultiStore`. - * Past versions are accessed with `GetVersion`, which returns a `BasicMultiStore`. + * Past version views 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. ### `BasicMultiStore` -A minimal interface that only allows accessing substores. Note: substores returned by `BasicMultiStore.GetKVStore` are read-only. +A minimal interface that only allows accessing substores. Note: substores returned by `BasicMultiStore.GetKVStore` are read-only and will panic on `Set` or `Delete` calls. ### Implementation (`root.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 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 (state storage, or *SS*), but are also mapped in a logically separate database which generates cryptographic proofs (for state-commitment or *SC*). -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. +The state-commitment component of each substore is implemented as an independent `smt.Store` (see below). Internally, each substore is allocated in a logically separate partition within the same backing DB, such that commits apply to the state of all substores. Therefore, views of past versions also include the state of all substores (including *SS* and *SC* data). -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). +This store can optionally be configured to use a different backend database instance for *SC* (e.g., `badgerdb` for the state storage DB and `memdb` for the state-commitment DB; see `StoreConfig.StateCommitmentDB`). ## SMT Store diff --git a/store/v2/root/store.go b/store/v2/root/store.go index 3962c67e2b8..626c63ba81f 100644 --- a/store/v2/root/store.go +++ b/store/v2/root/store.go @@ -863,11 +863,7 @@ func (pr *prefixRegistry) ReservePrefix(key string, typ types.StoreType) error { 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 { - tlm.listeners[key] = listeners - } + tlm.listeners[key] = append(tlm.listeners[key], listeners...) } // ListeningEnabled returns if listening is enabled for a specific KVStore @@ -892,8 +888,8 @@ func (tlm *traceListenMixin) wrapTraceListen(store types.KVStore, skey types.Sto 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) + if tlm.ListeningEnabled(skey) { + store = listenkv.NewStore(store, skey, tlm.listeners[skey.Name()]) } return store } From 038f0eb6c1b754064a9837d9a8e686bbf40e3e76 Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 16 Dec 2021 00:32:22 +0800 Subject: [PATCH 44/45] rename store/v2/{root=>multi} --- store/v2/{root => multi}/cache_store.go | 0 store/v2/{root => multi}/doc.go | 0 store/v2/{root => multi}/store.go | 0 store/v2/{root => multi}/store_test.go | 0 store/v2/{root => multi}/sub_store.go | 0 store/v2/{root => multi}/test_util.go | 0 store/v2/{root => multi}/view_store.go | 0 7 files changed, 0 insertions(+), 0 deletions(-) rename store/v2/{root => multi}/cache_store.go (100%) rename store/v2/{root => multi}/doc.go (100%) rename store/v2/{root => multi}/store.go (100%) rename store/v2/{root => multi}/store_test.go (100%) rename store/v2/{root => multi}/sub_store.go (100%) rename store/v2/{root => multi}/test_util.go (100%) rename store/v2/{root => multi}/view_store.go (100%) diff --git a/store/v2/root/cache_store.go b/store/v2/multi/cache_store.go similarity index 100% rename from store/v2/root/cache_store.go rename to store/v2/multi/cache_store.go diff --git a/store/v2/root/doc.go b/store/v2/multi/doc.go similarity index 100% rename from store/v2/root/doc.go rename to store/v2/multi/doc.go diff --git a/store/v2/root/store.go b/store/v2/multi/store.go similarity index 100% rename from store/v2/root/store.go rename to store/v2/multi/store.go diff --git a/store/v2/root/store_test.go b/store/v2/multi/store_test.go similarity index 100% rename from store/v2/root/store_test.go rename to store/v2/multi/store_test.go diff --git a/store/v2/root/sub_store.go b/store/v2/multi/sub_store.go similarity index 100% rename from store/v2/root/sub_store.go rename to store/v2/multi/sub_store.go diff --git a/store/v2/root/test_util.go b/store/v2/multi/test_util.go similarity index 100% rename from store/v2/root/test_util.go rename to store/v2/multi/test_util.go diff --git a/store/v2/root/view_store.go b/store/v2/multi/view_store.go similarity index 100% rename from store/v2/root/view_store.go rename to store/v2/multi/view_store.go From 312924fc7c7f4582dd298bebb5c4e5e98eff1e7a Mon Sep 17 00:00:00 2001 From: Roy Crihfield Date: Thu, 16 Dec 2021 00:34:58 +0800 Subject: [PATCH 45/45] rename ReservePrefix => RegisterSubstore --- store/v2/multi/store.go | 6 +++--- store/v2/multi/store_test.go | 40 ++++++++++++++++++------------------ 2 files changed, 23 insertions(+), 23 deletions(-) diff --git a/store/v2/multi/store.go b/store/v2/multi/store.go index 626c63ba81f..08555da03c6 100644 --- a/store/v2/multi/store.go +++ b/store/v2/multi/store.go @@ -397,7 +397,7 @@ func (pr *prefixRegistry) migrate(store *Store, upgrades types.StoreUpgrades) er } pr.reserved = append(pr.reserved[:ix], pr.reserved[ix+1:]...) delete(pr.StoreSchema, rename.OldKey) - err = pr.ReservePrefix(rename.NewKey, types.StoreTypePersistent) + err = pr.RegisterSubstore(rename.NewKey, types.StoreTypePersistent) if err != nil { return err } @@ -429,7 +429,7 @@ func (pr *prefixRegistry) migrate(store *Store, upgrades types.StoreUpgrades) er } for _, key := range upgrades.Added { - err := pr.ReservePrefix(key, types.StoreTypePersistent) + err := pr.RegisterSubstore(key, types.StoreTypePersistent) if err != nil { return err } @@ -838,7 +838,7 @@ func (pr *prefixRegistry) storeInfo(key string) (sst types.StoreType, ix int, er return } -func (pr *prefixRegistry) ReservePrefix(key string, typ types.StoreType) error { +func (pr *prefixRegistry) RegisterSubstore(key string, typ types.StoreType) error { if !validSubStoreType(typ) { return fmt.Errorf("StoreType not supported: %v", typ) } diff --git a/store/v2/multi/store_test.go b/store/v2/multi/store_test.go index aa9576019b1..435277fcbae 100644 --- a/store/v2/multi/store_test.go +++ b/store/v2/multi/store_test.go @@ -34,16 +34,16 @@ var ( func simpleStoreConfig(t *testing.T) StoreConfig { opts := DefaultStoreConfig() - require.NoError(t, opts.ReservePrefix(skey_1.Name(), types.StoreTypePersistent)) + require.NoError(t, opts.RegisterSubstore(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)) + require.NoError(t, opts.RegisterSubstore(skey_1.Name(), types.StoreTypePersistent)) + require.NoError(t, opts.RegisterSubstore(skey_2.Name(), types.StoreTypePersistent)) + require.NoError(t, opts.RegisterSubstore(skey_3.Name(), types.StoreTypePersistent)) return opts } @@ -639,20 +639,20 @@ func TestQuery(t *testing.T) { 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)) + require.Error(t, opts.RegisterSubstore(skey_1.Name(), types.StoreTypeDB)) + require.Error(t, opts.RegisterSubstore(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)) + require.NoError(t, opts.RegisterSubstore(skey_1.Name(), types.StoreTypePersistent)) + require.NoError(t, opts.RegisterSubstore(skey_2.Name(), types.StoreTypeMemory)) + require.NoError(t, opts.RegisterSubstore(skey_3b.Name(), types.StoreTypeTransient)) + require.Error(t, opts.RegisterSubstore(skey_1b.Name(), types.StoreTypePersistent)) + require.Error(t, opts.RegisterSubstore(skey_2b.Name(), types.StoreTypePersistent)) + require.Error(t, opts.RegisterSubstore(skey_3.Name(), types.StoreTypePersistent)) } func TestMultiStoreBasic(t *testing.T) { opts := DefaultStoreConfig() - err := opts.ReservePrefix(skey_1.Name(), types.StoreTypePersistent) + err := opts.RegisterSubstore(skey_1.Name(), types.StoreTypePersistent) require.NoError(t, err) db := memdb.NewDB() store, err := NewStore(db, opts) @@ -798,11 +798,11 @@ func TestMultiStoreMigration(t *testing.T) { // pass in a schema reflecting the migrations migratedOpts := DefaultStoreConfig() - err = migratedOpts.ReservePrefix(skey_1.Name(), types.StoreTypePersistent) + err = migratedOpts.RegisterSubstore(skey_1.Name(), types.StoreTypePersistent) require.NoError(t, err) - err = migratedOpts.ReservePrefix(skey_2b.Name(), types.StoreTypePersistent) + err = migratedOpts.RegisterSubstore(skey_2b.Name(), types.StoreTypePersistent) require.NoError(t, err) - err = migratedOpts.ReservePrefix(skey_4.Name(), types.StoreTypePersistent) + err = migratedOpts.RegisterSubstore(skey_4.Name(), types.StoreTypePersistent) require.NoError(t, err) store, err = NewStore(db, migratedOpts) require.Nil(t, err) @@ -858,8 +858,8 @@ func TestTrace(t *testing.T) { 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)) + require.NoError(t, opts.RegisterSubstore(skey_2.Name(), types.StoreTypeMemory)) + require.NoError(t, opts.RegisterSubstore(skey_3.Name(), types.StoreTypeTransient)) store, err := NewStore(db, opts) require.NoError(t, err) @@ -936,8 +936,8 @@ func TestListeners(t *testing.T) { 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)) + require.NoError(t, opts.RegisterSubstore(skey_2.Name(), types.StoreTypeMemory)) + require.NoError(t, opts.RegisterSubstore(skey_3.Name(), types.StoreTypeTransient)) store, err := NewStore(db, opts) require.NoError(t, err)