From 9d2ef775cf496d97926ff5c1f24f7c0c0065fc04 Mon Sep 17 00:00:00 2001 From: Roman Khimov Date: Wed, 16 Feb 2022 17:48:15 +0300 Subject: [PATCH] storage: simplify (*MemCachedStore).Put/Delete interface They never return errors, so their interface should reflect that. This allows to remove quite a lot of useless and never tested code. Notice that Get still does return an error. It can be made not to do that, but usually we need to differentiate between successful/unsuccessful accesses anyway, so this doesn't help much. --- pkg/core/blockchain.go | 73 +++++--------------- pkg/core/blockchain_test.go | 26 +++---- pkg/core/dao/dao.go | 88 +++++++++++------------- pkg/core/dao/dao_test.go | 28 +++----- pkg/core/interop_system.go | 6 +- pkg/core/interop_system_test.go | 11 ++- pkg/core/mpt/batch_test.go | 4 +- pkg/core/mpt/billet.go | 6 +- pkg/core/mpt/proof.go | 3 +- pkg/core/mpt/trie.go | 8 +-- pkg/core/mpt/trie_test.go | 2 +- pkg/core/native/management.go | 33 +++------ pkg/core/native/management_test.go | 2 +- pkg/core/native/native_neo.go | 72 +++++++------------ pkg/core/native/native_nep17.go | 25 +++---- pkg/core/native/notary.go | 24 ++----- pkg/core/native/oracle.go | 22 ++---- pkg/core/native/policy.go | 37 +++------- pkg/core/native/util.go | 7 +- pkg/core/stateroot/module.go | 20 ++---- pkg/core/stateroot/store.go | 20 ++---- pkg/core/statesync/module.go | 12 +--- pkg/core/storage/memcached_store.go | 8 +-- pkg/core/storage/memcached_store_test.go | 65 +++++++++-------- pkg/core/storage/memory_store_test.go | 4 +- pkg/core/storage/storeandbatch_test.go | 2 +- 26 files changed, 228 insertions(+), 380 deletions(-) diff --git a/pkg/core/blockchain.go b/pkg/core/blockchain.go index 25de99481..034dff3c4 100644 --- a/pkg/core/blockchain.go +++ b/pkg/core/blockchain.go @@ -316,9 +316,7 @@ func (bc *Blockchain) init() error { KeepOnlyLatestState: bc.config.KeepOnlyLatestState, Value: version, } - if err = bc.dao.PutVersion(ver); err != nil { - return err - } + bc.dao.PutVersion(ver) bc.dao.Version = ver bc.persistent.Version = ver genesisBlock, err := createGenesisBlock(bc.config) @@ -326,10 +324,7 @@ func (bc *Blockchain) init() error { return err } bc.headerHashes = []util.Uint256{genesisBlock.Hash()} - err = bc.dao.PutCurrentHeader(hashAndIndexToBytes(genesisBlock.Hash(), genesisBlock.Index)) - if err != nil { - return err - } + bc.dao.PutCurrentHeader(hashAndIndexToBytes(genesisBlock.Hash(), genesisBlock.Index)) if err := bc.stateRoot.Init(0); err != nil { return fmt.Errorf("can't init MPT: %w", err) } @@ -508,10 +503,7 @@ func (bc *Blockchain) jumpToStateInternal(p uint32, stage stateJumpStage) error jumpStageKey := storage.SYSStateJumpStage.Bytes() switch stage { case none: - err := bc.dao.Store.Put(jumpStageKey, []byte{byte(stateJumpStarted)}) - if err != nil { - return fmt.Errorf("failed to store state jump stage: %w", err) - } + bc.dao.Store.Put(jumpStageKey, []byte{byte(stateJumpStarted)}) fallthrough case stateJumpStarted: newPrefix := statesync.TemporaryPrefix(bc.dao.Version.StoragePrefix) @@ -520,15 +512,10 @@ func (bc *Blockchain) jumpToStateInternal(p uint32, stage stateJumpStage) error return fmt.Errorf("failed to get dao.Version: %w", err) } v.StoragePrefix = newPrefix - if err := bc.dao.PutVersion(v); err != nil { - return fmt.Errorf("failed to update dao.Version: %w", err) - } + bc.dao.PutVersion(v) bc.persistent.Version = v - err = bc.dao.Store.Put(jumpStageKey, []byte{byte(newStorageItemsAdded)}) - if err != nil { - return fmt.Errorf("failed to store state jump stage: %w", err) - } + bc.dao.Store.Put(jumpStageKey, []byte{byte(newStorageItemsAdded)}) fallthrough case newStorageItemsAdded: @@ -536,7 +523,7 @@ func (bc *Blockchain) jumpToStateInternal(p uint32, stage stateJumpStage) error prefix := statesync.TemporaryPrefix(bc.dao.Version.StoragePrefix) bc.dao.Store.Seek(storage.SeekRange{Prefix: []byte{byte(prefix)}}, func(k, _ []byte) bool { // #1468, but don't need to copy here, because it is done by Store. - _ = cache.Store.Delete(k) + cache.Store.Delete(k) return true }) @@ -551,12 +538,12 @@ func (bc *Blockchain) jumpToStateInternal(p uint32, stage stateJumpStage) error prefixes := []byte{byte(storage.STNEP11Transfers), byte(storage.STNEP17Transfers), byte(storage.STTokenTransferInfo)} for i := range prefixes { cache.Store.Seek(storage.SeekRange{Prefix: prefixes[i : i+1]}, func(k, v []byte) bool { - _ = cache.Store.Delete(k) // It's MemCachedStore which never returns an error. + cache.Store.Delete(k) return true }) } } - _ = cache.Store.Put(jumpStageKey, []byte{byte(genesisStateRemoved)}) + cache.Store.Put(jumpStageKey, []byte{byte(genesisStateRemoved)}) _, err := cache.Persist() if err != nil { return fmt.Errorf("failed to persist old items removal: %w", err) @@ -573,10 +560,7 @@ func (bc *Blockchain) jumpToStateInternal(p uint32, stage stateJumpStage) error return fmt.Errorf("failed to get current block: %w", err) } writeBuf.Reset() - err = bc.dao.StoreAsCurrentBlock(block, writeBuf) - if err != nil { - return fmt.Errorf("failed to store current block: %w", err) - } + bc.dao.StoreAsCurrentBlock(block, writeBuf) bc.topBlock.Store(block) atomic.StoreUint32(&bc.blockHeight, p) atomic.StoreUint32(&bc.persistedHeight, p) @@ -585,12 +569,10 @@ func (bc *Blockchain) jumpToStateInternal(p uint32, stage stateJumpStage) error if err != nil { return fmt.Errorf("failed to get block to init MPT: %w", err) } - if err = bc.stateRoot.JumpToState(&state.MPTRoot{ + bc.stateRoot.JumpToState(&state.MPTRoot{ Index: p, Root: block.PrevStateRoot, - }); err != nil { - return fmt.Errorf("can't perform MPT jump to height %d: %w", p, err) - } + }) err = bc.contracts.NEO.InitializeCache(bc, bc.dao) if err != nil { @@ -608,10 +590,7 @@ func (bc *Blockchain) jumpToStateInternal(p uint32, stage stateJumpStage) error updateBlockHeightMetric(p) - err = bc.dao.Store.Delete(jumpStageKey) - if err != nil { - return fmt.Errorf("failed to remove outdated state jump stage: %w", err) - } + bc.dao.Store.Delete(jumpStageKey) return nil } @@ -973,7 +952,7 @@ func (bc *Blockchain) addHeaders(verify bool, headers ...*block.Header) error { } key := storage.AppendPrefix(storage.DataExecutable, h.Hash().BytesBE()) - _ = batch.Store.Put(key, buf.Bytes()) + batch.Store.Put(key, buf.Bytes()) buf.Reset() lastHeader = h } @@ -986,11 +965,11 @@ func (bc *Blockchain) addHeaders(verify bool, headers ...*block.Header) error { } key := storage.AppendPrefixInt(storage.IXHeaderHashList, int(bc.storedHeaderCount)) - _ = batch.Store.Put(key, buf.Bytes()) + batch.Store.Put(key, buf.Bytes()) bc.storedHeaderCount += headerBatchCount } - _ = batch.Store.Put(storage.SYSCurrentHeader.Bytes(), hashAndIndexToBytes(lastHeader.Hash(), lastHeader.Index)) + batch.Store.Put(storage.SYSCurrentHeader.Bytes(), hashAndIndexToBytes(lastHeader.Hash(), lastHeader.Index)) updateHeaderHeightMetric(len(bc.headerHashes) - 1) if _, err = batch.Persist(); err != nil { return err @@ -1033,10 +1012,7 @@ func (bc *Blockchain) storeBlock(block *block.Block, txpool *mempool.Pool) error baer1, baer2 *state.AppExecResult transCache = make(map[util.Uint160]transferData) ) - if err := kvcache.StoreAsCurrentBlock(block, writeBuf); err != nil { - aerdone <- err - return - } + kvcache.StoreAsCurrentBlock(block, writeBuf) writeBuf.Reset() if bc.config.RemoveUntraceableBlocks { var start, stop uint32 @@ -1102,18 +1078,10 @@ func (bc *Blockchain) storeBlock(block *block.Block, txpool *mempool.Pool) error return } if !trData.Info.NewNEP11Batch { - err = kvcache.PutTokenTransferLog(acc, trData.Info.NextNEP11NewestTimestamp, trData.Info.NextNEP11Batch, true, &trData.Log11) - if err != nil { - aerdone <- err - return - } + kvcache.PutTokenTransferLog(acc, trData.Info.NextNEP11NewestTimestamp, trData.Info.NextNEP11Batch, true, &trData.Log11) } if !trData.Info.NewNEP17Batch { - err = kvcache.PutTokenTransferLog(acc, trData.Info.NextNEP17NewestTimestamp, trData.Info.NextNEP17Batch, false, &trData.Log17) - if err != nil { - aerdone <- err - return - } + kvcache.PutTokenTransferLog(acc, trData.Info.NextNEP17NewestTimestamp, trData.Info.NextNEP17Batch, false, &trData.Log17) } } close(aerdone) @@ -1471,10 +1439,7 @@ func appendTokenTransfer(cache dao.DAO, transCache map[util.Uint160]transferData transferData.Info.LastUpdated[token] = bIndex *newBatch = log.Size() >= state.TokenTransferBatchSize if *newBatch { - err = cache.PutTokenTransferLog(addr, *currTimestamp, *nextBatch, isNEP11, log) - if err != nil { - return err - } + cache.PutTokenTransferLog(addr, *currTimestamp, *nextBatch, isNEP11, log) *nextBatch++ *currTimestamp = bTimestamp // Put makes a copy of it anyway. diff --git a/pkg/core/blockchain_test.go b/pkg/core/blockchain_test.go index 9c95b5e21..8272a130d 100644 --- a/pkg/core/blockchain_test.go +++ b/pkg/core/blockchain_test.go @@ -1385,7 +1385,7 @@ func TestClose(t *testing.T) { // It's a hack, but we use internal knowledge of MemoryStore // implementation which makes it completely unusable (up to panicing) // after Close(). - _ = bc.dao.Store.Put([]byte{0}, []byte{1}) + bc.dao.Store.Put([]byte{0}, []byte{1}) // This should never be executed. assert.Nil(t, t) @@ -1602,16 +1602,16 @@ func TestRemoveOldTransfers(t *testing.T) { ttl := state.TokenTransferLog{Raw: []byte{1}} // It's incorrect, but who cares. for i := uint32(0); i < 3; i++ { - require.NoError(t, bc.dao.PutTokenTransferLog(acc1, older, i, false, &ttl)) + bc.dao.PutTokenTransferLog(acc1, older, i, false, &ttl) } for i := uint32(0); i < 3; i++ { - require.NoError(t, bc.dao.PutTokenTransferLog(acc2, newer, i, false, &ttl)) + bc.dao.PutTokenTransferLog(acc2, newer, i, false, &ttl) } for i := uint32(0); i < 2; i++ { - require.NoError(t, bc.dao.PutTokenTransferLog(acc3, older, i, true, &ttl)) + bc.dao.PutTokenTransferLog(acc3, older, i, true, &ttl) } for i := uint32(0); i < 2; i++ { - require.NoError(t, bc.dao.PutTokenTransferLog(acc3, newer, i, true, &ttl)) + bc.dao.PutTokenTransferLog(acc3, newer, i, true, &ttl) } _, err = bc.dao.Persist() @@ -1855,7 +1855,7 @@ func TestBlockchain_InitWithIncompleteStateJump(t *testing.T) { key := slice.Copy(k) key[0] = byte(tempPrefix) value := slice.Copy(v) - _ = batch.Put(key, value) + batch.Put(key, value) return true }) _, err := batch.Persist() @@ -1879,33 +1879,33 @@ func TestBlockchain_InitWithIncompleteStateJump(t *testing.T) { } // manually store statejump stage to check statejump recover process t.Run("invalid RemoveUntraceableBlocks setting", func(t *testing.T) { - require.NoError(t, bcSpout.dao.Store.Put(storage.SYSStateJumpStage.Bytes(), []byte{byte(stateJumpStarted)})) + bcSpout.dao.Store.Put(storage.SYSStateJumpStage.Bytes(), []byte{byte(stateJumpStarted)}) checkNewBlockchainErr(t, func(c *config.Config) { boltCfg(c) c.ProtocolConfiguration.RemoveUntraceableBlocks = false }, bcSpout.dao.Store, true) }) t.Run("invalid state jump stage format", func(t *testing.T) { - require.NoError(t, bcSpout.dao.Store.Put(storage.SYSStateJumpStage.Bytes(), []byte{0x01, 0x02})) + bcSpout.dao.Store.Put(storage.SYSStateJumpStage.Bytes(), []byte{0x01, 0x02}) checkNewBlockchainErr(t, boltCfg, bcSpout.dao.Store, true) }) t.Run("missing state sync point", func(t *testing.T) { - require.NoError(t, bcSpout.dao.Store.Put(storage.SYSStateJumpStage.Bytes(), []byte{byte(stateJumpStarted)})) + bcSpout.dao.Store.Put(storage.SYSStateJumpStage.Bytes(), []byte{byte(stateJumpStarted)}) checkNewBlockchainErr(t, boltCfg, bcSpout.dao.Store, true) }) t.Run("invalid state sync point", func(t *testing.T) { - require.NoError(t, bcSpout.dao.Store.Put(storage.SYSStateJumpStage.Bytes(), []byte{byte(stateJumpStarted)})) + bcSpout.dao.Store.Put(storage.SYSStateJumpStage.Bytes(), []byte{byte(stateJumpStarted)}) point := make([]byte, 4) binary.LittleEndian.PutUint32(point, uint32(len(bcSpout.headerHashes))) - require.NoError(t, bcSpout.dao.Store.Put(storage.SYSStateSyncPoint.Bytes(), point)) + bcSpout.dao.Store.Put(storage.SYSStateSyncPoint.Bytes(), point) checkNewBlockchainErr(t, boltCfg, bcSpout.dao.Store, true) }) for _, stage := range []stateJumpStage{stateJumpStarted, newStorageItemsAdded, genesisStateRemoved, 0x03} { t.Run(fmt.Sprintf("state jump stage %d", stage), func(t *testing.T) { - require.NoError(t, bcSpout.dao.Store.Put(storage.SYSStateJumpStage.Bytes(), []byte{byte(stage)})) + bcSpout.dao.Store.Put(storage.SYSStateJumpStage.Bytes(), []byte{byte(stage)}) point := make([]byte, 4) binary.LittleEndian.PutUint32(point, uint32(stateSyncPoint)) - require.NoError(t, bcSpout.dao.Store.Put(storage.SYSStateSyncPoint.Bytes(), point)) + bcSpout.dao.Store.Put(storage.SYSStateSyncPoint.Bytes(), point) shouldFail := stage == 0x03 // unknown stage checkNewBlockchainErr(t, spountCfg, bcSpout.dao.Store, shouldFail) }) diff --git a/pkg/core/dao/dao.go b/pkg/core/dao/dao.go index 0573e1265..b3ff1c8b9 100644 --- a/pkg/core/dao/dao.go +++ b/pkg/core/dao/dao.go @@ -5,7 +5,6 @@ import ( "context" "encoding/binary" "errors" - "fmt" iocore "io" "sort" @@ -31,8 +30,8 @@ var ( // DAO is a data access object. type DAO interface { DeleteBlock(h util.Uint256, buf *io.BufBinWriter) error - DeleteContractID(id int32) error - DeleteStorageItem(id int32, key []byte) error + DeleteContractID(id int32) + DeleteStorageItem(id int32, key []byte) GetAndDecode(entity io.Serializable, key []byte) error GetAppExecResults(hash util.Uint256, trig trigger.Type) ([]state.AppExecResult, error) GetBatch() *storage.MemBatch @@ -54,18 +53,18 @@ type DAO interface { HasTransaction(hash util.Uint256) error Persist() (int, error) PersistSync() (int, error) - PutContractID(id int32, hash util.Uint160) error - PutCurrentHeader(hashAndIndex []byte) error + PutContractID(id int32, hash util.Uint160) + PutCurrentHeader(hashAndIndex []byte) PutTokenTransferInfo(acc util.Uint160, bs *state.TokenTransferInfo) error - PutTokenTransferLog(acc util.Uint160, start uint64, index uint32, isNEP11 bool, lg *state.TokenTransferLog) error - PutStateSyncPoint(p uint32) error - PutStateSyncCurrentBlockHeight(h uint32) error - PutStorageItem(id int32, key []byte, si state.StorageItem) error - PutVersion(v Version) error + PutTokenTransferLog(acc util.Uint160, start uint64, index uint32, isNEP11 bool, lg *state.TokenTransferLog) + PutStateSyncPoint(p uint32) + PutStateSyncCurrentBlockHeight(h uint32) + PutStorageItem(id int32, key []byte, si state.StorageItem) + PutVersion(v Version) Seek(id int32, rng storage.SeekRange, f func(k, v []byte) bool) SeekAsync(ctx context.Context, id int32, rng storage.SeekRange) chan storage.KeyValue StoreAsBlock(block *block.Block, aer1 *state.AppExecResult, aer2 *state.AppExecResult, buf *io.BufBinWriter) error - StoreAsCurrentBlock(block *block.Block, buf *io.BufBinWriter) error + StoreAsCurrentBlock(block *block.Block, buf *io.BufBinWriter) StoreAsTransaction(tx *transaction.Transaction, index uint32, aer *state.AppExecResult, buf *io.BufBinWriter) error putTokenTransferInfo(acc util.Uint160, bs *state.TokenTransferInfo, buf *io.BufBinWriter) error } @@ -113,18 +112,14 @@ func (dao *Simple) GetAndDecode(entity io.Serializable, key []byte) error { return reader.Err } -// Put performs put operation with serializable structures. -func (dao *Simple) Put(entity io.Serializable, key []byte) error { - return dao.putWithBuffer(entity, key, io.NewBufBinWriter()) -} - // putWithBuffer performs put operation using buf as a pre-allocated buffer for serialization. func (dao *Simple) putWithBuffer(entity io.Serializable, key []byte, buf *io.BufBinWriter) error { entity.EncodeBinary(buf.BinWriter) if buf.Err != nil { return buf.Err } - return dao.Store.Put(key, buf.Bytes()) + dao.Store.Put(key, buf.Bytes()) + return nil } func makeContractIDKey(id int32) []byte { @@ -135,13 +130,13 @@ func makeContractIDKey(id int32) []byte { } // DeleteContractID deletes contract's id to hash mapping. -func (dao *Simple) DeleteContractID(id int32) error { - return dao.Store.Delete(makeContractIDKey(id)) +func (dao *Simple) DeleteContractID(id int32) { + dao.Store.Delete(makeContractIDKey(id)) } // PutContractID adds a mapping from contract's ID to its hash. -func (dao *Simple) PutContractID(id int32, hash util.Uint160) error { - return dao.Store.Put(makeContractIDKey(id), hash.BytesBE()) +func (dao *Simple) PutContractID(id int32, hash util.Uint160) { + dao.Store.Put(makeContractIDKey(id), hash.BytesBE()) } // GetContractScriptHash retrieves contract's hash given its ID. @@ -251,9 +246,9 @@ func (dao *Simple) GetTokenTransferLog(acc util.Uint160, newestTimestamp uint64, } // PutTokenTransferLog saves given transfer log in the cache. -func (dao *Simple) PutTokenTransferLog(acc util.Uint160, start uint64, index uint32, isNEP11 bool, lg *state.TokenTransferLog) error { +func (dao *Simple) PutTokenTransferLog(acc util.Uint160, start uint64, index uint32, isNEP11 bool, lg *state.TokenTransferLog) { key := getTokenTransferLogKey(acc, start, index, isNEP11) - return dao.Store.Put(key, lg.Raw) + dao.Store.Put(key, lg.Raw) } // -- end transfer log. @@ -315,16 +310,16 @@ func (dao *Simple) GetStorageItem(id int32, key []byte) state.StorageItem { // PutStorageItem puts given StorageItem for given id with given // key into the given store. -func (dao *Simple) PutStorageItem(id int32, key []byte, si state.StorageItem) error { +func (dao *Simple) PutStorageItem(id int32, key []byte, si state.StorageItem) { stKey := makeStorageItemKey(dao.Version.StoragePrefix, id, key) - return dao.Store.Put(stKey, si) + dao.Store.Put(stKey, si) } // DeleteStorageItem drops storage item for the given id with the // given key from the store. -func (dao *Simple) DeleteStorageItem(id int32, key []byte) error { +func (dao *Simple) DeleteStorageItem(id int32, key []byte) { stKey := makeStorageItemKey(dao.Version.StoragePrefix, id, key) - return dao.Store.Delete(stKey) + dao.Store.Delete(stKey) } // GetStorageItems returns all storage items for a given id. @@ -582,28 +577,28 @@ func (dao *Simple) GetTransaction(hash util.Uint256) (*transaction.Transaction, } // PutVersion stores the given version in the underlying store. -func (dao *Simple) PutVersion(v Version) error { +func (dao *Simple) PutVersion(v Version) { dao.Version = v - return dao.Store.Put(storage.SYSVersion.Bytes(), v.Bytes()) + dao.Store.Put(storage.SYSVersion.Bytes(), v.Bytes()) } // PutCurrentHeader stores current header. -func (dao *Simple) PutCurrentHeader(hashAndIndex []byte) error { - return dao.Store.Put(storage.SYSCurrentHeader.Bytes(), hashAndIndex) +func (dao *Simple) PutCurrentHeader(hashAndIndex []byte) { + dao.Store.Put(storage.SYSCurrentHeader.Bytes(), hashAndIndex) } // PutStateSyncPoint stores current state synchronisation point P. -func (dao *Simple) PutStateSyncPoint(p uint32) error { +func (dao *Simple) PutStateSyncPoint(p uint32) { buf := make([]byte, 4) binary.LittleEndian.PutUint32(buf, p) - return dao.Store.Put(storage.SYSStateSyncPoint.Bytes(), buf) + dao.Store.Put(storage.SYSStateSyncPoint.Bytes(), buf) } // PutStateSyncCurrentBlockHeight stores current block height during state synchronisation process. -func (dao *Simple) PutStateSyncCurrentBlockHeight(h uint32) error { +func (dao *Simple) PutStateSyncCurrentBlockHeight(h uint32) { buf := make([]byte, 4) binary.LittleEndian.PutUint32(buf, h) - return dao.Store.Put(storage.SYSStateSyncCurrentBlockHeight.Bytes(), buf) + dao.Store.Put(storage.SYSStateSyncCurrentBlockHeight.Bytes(), buf) } // read2000Uint256Hashes attempts to read 2000 Uint256 hashes from @@ -662,7 +657,8 @@ func (dao *Simple) StoreAsBlock(block *block.Block, aer1 *state.AppExecResult, a if buf.Err != nil { return buf.Err } - return dao.Store.Put(key, buf.Bytes()) + dao.Store.Put(key, buf.Bytes()) + return nil } // DeleteBlock removes block from dao. It's not atomic, so make sure you're @@ -694,16 +690,16 @@ func (dao *Simple) DeleteBlock(h util.Uint256, w *io.BufBinWriter) error { if w.Err != nil { return w.Err } - _ = dao.Store.Put(key, w.Bytes()) + dao.Store.Put(key, w.Bytes()) for _, tx := range b.Transactions { copy(key[1:], tx.Hash().BytesBE()) - _ = dao.Store.Delete(key) + dao.Store.Delete(key) if dao.Version.P2PSigExtensions { for _, attr := range tx.GetAttributes(transaction.ConflictsT) { hash := attr.Value.(*transaction.Conflicts).Hash copy(key[1:], hash.BytesBE()) - _ = dao.Store.Delete(key) + dao.Store.Delete(key) } } } @@ -714,14 +710,14 @@ func (dao *Simple) DeleteBlock(h util.Uint256, w *io.BufBinWriter) error { // StoreAsCurrentBlock stores a hash of the given block with prefix // SYSCurrentBlock. It can reuse given buffer for the purpose of value // serialization. -func (dao *Simple) StoreAsCurrentBlock(block *block.Block, buf *io.BufBinWriter) error { +func (dao *Simple) StoreAsCurrentBlock(block *block.Block, buf *io.BufBinWriter) { if buf == nil { buf = io.NewBufBinWriter() } h := block.Hash() h.EncodeBinary(buf.BinWriter) buf.WriteU32LE(block.Index) - return dao.Store.Put(storage.SYSCurrentBlock.Bytes(), buf.Bytes()) + dao.Store.Put(storage.SYSCurrentBlock.Bytes(), buf.Bytes()) } // StoreAsTransaction stores given TX as DataTransaction. It also stores transactions @@ -741,10 +737,7 @@ func (dao *Simple) StoreAsTransaction(tx *transaction.Transaction, index uint32, if buf.Err != nil { return buf.Err } - err := dao.Store.Put(key, buf.Bytes()) - if err != nil { - return err - } + dao.Store.Put(key, buf.Bytes()) if dao.Version.P2PSigExtensions { var value []byte for _, attr := range tx.GetAttributes(transaction.ConflictsT) { @@ -757,10 +750,7 @@ func (dao *Simple) StoreAsTransaction(tx *transaction.Transaction, index uint32, buf.BinWriter.WriteB(transaction.DummyVersion) value = buf.Bytes() } - err = dao.Store.Put(key, value) - if err != nil { - return fmt.Errorf("failed to store conflicting transaction %s for transaction %s: %w", hash.StringLE(), tx.Hash().StringLE(), err) - } + dao.Store.Put(key, value) } } return nil diff --git a/pkg/core/dao/dao_test.go b/pkg/core/dao/dao_test.go index fa3fb9d41..4b7397235 100644 --- a/pkg/core/dao/dao_test.go +++ b/pkg/core/dao/dao_test.go @@ -22,11 +22,10 @@ func TestPutGetAndDecode(t *testing.T) { dao := NewSimple(storage.NewMemoryStore(), false, false) serializable := &TestSerializable{field: random.String(4)} hash := []byte{1} - err := dao.Put(serializable, hash) - require.NoError(t, err) + require.NoError(t, dao.putWithBuffer(serializable, hash, io.NewBufBinWriter())) gotAndDecoded := &TestSerializable{} - err = dao.GetAndDecode(gotAndDecoded, hash) + err := dao.GetAndDecode(gotAndDecoded, hash) require.NoError(t, err) } @@ -48,8 +47,7 @@ func TestPutGetStorageItem(t *testing.T) { id := int32(random.Int(0, 1024)) key := []byte{0} storageItem := state.StorageItem{} - err := dao.PutStorageItem(id, key, storageItem) - require.NoError(t, err) + dao.PutStorageItem(id, key, storageItem) gotStorageItem := dao.GetStorageItem(id, key) require.Equal(t, storageItem, gotStorageItem) } @@ -59,10 +57,8 @@ func TestDeleteStorageItem(t *testing.T) { id := int32(random.Int(0, 1024)) key := []byte{0} storageItem := state.StorageItem{} - err := dao.PutStorageItem(id, key, storageItem) - require.NoError(t, err) - err = dao.DeleteStorageItem(id, key) - require.NoError(t, err) + dao.PutStorageItem(id, key, storageItem) + dao.DeleteStorageItem(id, key) gotStorageItem := dao.GetStorageItem(id, key) require.Nil(t, gotStorageItem) } @@ -129,22 +125,21 @@ func TestGetVersion(t *testing.T) { StateRootInHeader: true, Value: "testVersion", } - err := dao.PutVersion(expected) - require.NoError(t, err) + dao.PutVersion(expected) actual, err := dao.GetVersion() require.NoError(t, err) require.Equal(t, expected, actual) t.Run("invalid", func(t *testing.T) { dao := NewSimple(storage.NewMemoryStore(), false, false) - require.NoError(t, dao.Store.Put(storage.SYSVersion.Bytes(), []byte("0.1.2\x00x"))) + dao.Store.Put(storage.SYSVersion.Bytes(), []byte("0.1.2\x00x")) _, err := dao.GetVersion() require.Error(t, err) }) t.Run("old format", func(t *testing.T) { dao := NewSimple(storage.NewMemoryStore(), false, false) - require.NoError(t, dao.Store.Put(storage.SYSVersion.Bytes(), []byte("0.1.2"))) + dao.Store.Put(storage.SYSVersion.Bytes(), []byte("0.1.2")) version, err := dao.GetVersion() require.NoError(t, err) @@ -169,8 +164,7 @@ func TestGetCurrentHeaderHeight_Store(t *testing.T) { }, }, } - err := dao.StoreAsCurrentBlock(b, nil) - require.NoError(t, err) + dao.StoreAsCurrentBlock(b, nil) height, err := dao.GetCurrentBlockHeight() require.NoError(t, err) require.Equal(t, uint32(0), height) @@ -305,7 +299,7 @@ func TestPutGetStateSyncPoint(t *testing.T) { // non-empty store var expected uint32 = 5 - require.NoError(t, dao.PutStateSyncPoint(expected)) + dao.PutStateSyncPoint(expected) actual, err := dao.GetStateSyncPoint() require.NoError(t, err) require.Equal(t, expected, actual) @@ -320,7 +314,7 @@ func TestPutGetStateSyncCurrentBlockHeight(t *testing.T) { // non-empty store var expected uint32 = 5 - require.NoError(t, dao.PutStateSyncCurrentBlockHeight(expected)) + dao.PutStateSyncCurrentBlockHeight(expected) actual, err := dao.GetStateSyncCurrentBlockHeight() require.NoError(t, err) require.Equal(t, expected, actual) diff --git a/pkg/core/interop_system.go b/pkg/core/interop_system.go index d5d3d70a3..94b9b4eee 100644 --- a/pkg/core/interop_system.go +++ b/pkg/core/interop_system.go @@ -59,7 +59,8 @@ func storageDelete(ic *interop.Context) error { return errors.New("StorageContext is read only") } key := ic.VM.Estack().Pop().Bytes() - return ic.DAO.DeleteStorageItem(stc.ID, key) + ic.DAO.DeleteStorageItem(stc.ID, key) + return nil } // storageGet returns stored key-value pair. @@ -128,7 +129,8 @@ func putWithContext(ic *interop.Context, stc *StorageContext, key []byte, value if !ic.VM.AddGas(int64(sizeInc) * ic.Chain.GetStoragePrice()) { return errGasLimitExceeded } - return ic.DAO.PutStorageItem(stc.ID, key, value) + ic.DAO.PutStorageItem(stc.ID, key, value) + return nil } // storagePut puts key-value pair into the storage. diff --git a/pkg/core/interop_system_test.go b/pkg/core/interop_system_test.go index de236b9d3..c2b2089be 100644 --- a/pkg/core/interop_system_test.go +++ b/pkg/core/interop_system_test.go @@ -362,8 +362,8 @@ func BenchmarkStorageFind(b *testing.B) { items["abc"+random.String(10)] = random.Bytes(10) } for k, v := range items { - require.NoError(b, context.DAO.PutStorageItem(contractState.ID, []byte(k), v)) - require.NoError(b, context.DAO.PutStorageItem(contractState.ID+1, []byte(k), v)) + context.DAO.PutStorageItem(contractState.ID, []byte(k), v) + context.DAO.PutStorageItem(contractState.ID+1, []byte(k), v) } changes, err := context.DAO.Persist() require.NoError(b, err) @@ -406,8 +406,8 @@ func BenchmarkStorageFindIteratorNext(b *testing.B) { items["abc"+random.String(10)] = random.Bytes(10) } for k, v := range items { - require.NoError(b, context.DAO.PutStorageItem(contractState.ID, []byte(k), v)) - require.NoError(b, context.DAO.PutStorageItem(contractState.ID+1, []byte(k), v)) + context.DAO.PutStorageItem(contractState.ID, []byte(k), v) + context.DAO.PutStorageItem(contractState.ID+1, []byte(k), v) } changes, err := context.DAO.Persist() require.NoError(b, err) @@ -487,8 +487,7 @@ func TestStorageFind(t *testing.T) { id := contractState.ID for i := range skeys { - err := context.DAO.PutStorageItem(id, skeys[i], items[i]) - require.NoError(t, err) + context.DAO.PutStorageItem(id, skeys[i], items[i]) } testFind := func(t *testing.T, prefix []byte, opts int64, expected []stackitem.Item) { diff --git a/pkg/core/mpt/batch_test.go b/pkg/core/mpt/batch_test.go index ced30c061..b2acf226a 100644 --- a/pkg/core/mpt/batch_test.go +++ b/pkg/core/mpt/batch_test.go @@ -275,8 +275,8 @@ func TestTrie_PutBatchHash(t *testing.T) { tr1.Collapse(1) tr2.Collapse(1) key := makeStorageKey(tr1.root.(*BranchNode).Children[2].Hash()) - require.NoError(t, tr1.Store.Delete(key)) - require.NoError(t, tr2.Store.Delete(key)) + tr1.Store.Delete(key) + tr2.Store.Delete(key) testIncompletePut(t, ps, 1, tr1, tr2) }) } diff --git a/pkg/core/mpt/billet.go b/pkg/core/mpt/billet.go index d50645fa1..af7edf2be 100644 --- a/pkg/core/mpt/billet.go +++ b/pkg/core/mpt/billet.go @@ -70,7 +70,7 @@ func (b *Billet) RestoreHashNode(path []byte, node Node) error { panic("invalid storage prefix") } k := append([]byte{byte(b.TempStoragePrefix)}, fromNibbles(path)...) - _ = b.Store.Put(k, leaf.value) + b.Store.Put(k, leaf.value) } return nil } @@ -194,9 +194,9 @@ func (b *Billet) incrementRefAndStore(h util.Uint256, bs []byte) { data = append(bs, 1, 0, 0, 0, 0) } binary.LittleEndian.PutUint32(data[len(data)-4:], uint32(cnt)) - _ = b.Store.Put(key, data) + b.Store.Put(key, data) } else { - _ = b.Store.Put(key, bs) + b.Store.Put(key, bs) } } diff --git a/pkg/core/mpt/proof.go b/pkg/core/mpt/proof.go index c3ed7d75c..74b25e7d4 100644 --- a/pkg/core/mpt/proof.go +++ b/pkg/core/mpt/proof.go @@ -69,8 +69,7 @@ func VerifyProof(rh util.Uint256, key []byte, proofs [][]byte) ([]byte, bool) { tr := NewTrie(NewHashNode(rh), ModeAll, storage.NewMemCachedStore(storage.NewMemoryStore())) for i := range proofs { h := hash.DoubleSha256(proofs[i]) - // no errors in Put to memory store - _ = tr.Store.Put(makeStorageKey(h), proofs[i]) + tr.Store.Put(makeStorageKey(h), proofs[i]) } _, leaf, _, err := tr.getWithPath(tr.root, path, true) if err != nil { diff --git a/pkg/core/mpt/trie.go b/pkg/core/mpt/trie.go index f1d2deebc..608a39cfb 100644 --- a/pkg/core/mpt/trie.go +++ b/pkg/core/mpt/trie.go @@ -420,7 +420,7 @@ func (t *Trie) Flush(index uint32) { delete(t.refcount, h) } } else if node.refcount > 0 { - _ = t.Store.Put(key, node.bytes) + t.Store.Put(key, node.bytes) } node.refcount = 0 } else { @@ -467,15 +467,15 @@ func (t *Trie) updateRefCount(h util.Uint256, key []byte, index uint32) int32 { panic(fmt.Sprintf("negative reference count: %s new %d, upd %d", h.StringBE(), cnt, t.refcount[h])) case cnt == 0: if !t.mode.GC() { - _ = t.Store.Delete(key) + t.Store.Delete(key) } else { data[len(data)-5] = 0 binary.LittleEndian.PutUint32(data[len(data)-4:], index) - _ = t.Store.Put(key, data) + t.Store.Put(key, data) } default: binary.LittleEndian.PutUint32(data[len(data)-4:], uint32(cnt)) - _ = t.Store.Put(key, data) + t.Store.Put(key, data) } return cnt } diff --git a/pkg/core/mpt/trie_test.go b/pkg/core/mpt/trie_test.go index 97c5ef4e4..3753f3886 100644 --- a/pkg/core/mpt/trie_test.go +++ b/pkg/core/mpt/trie_test.go @@ -251,7 +251,7 @@ func (tr *Trie) putToStore(n Node) { } tr.updateRefCount(n.Hash(), makeStorageKey(n.Hash()), 0) } else { - _ = tr.Store.Put(makeStorageKey(n.Hash()), n.Bytes()) + tr.Store.Put(makeStorageKey(n.Hash()), n.Bytes()) } } diff --git a/pkg/core/native/management.go b/pkg/core/native/management.go index d85613738..936ffbbf7 100644 --- a/pkg/core/native/management.go +++ b/pkg/core/native/management.go @@ -386,23 +386,14 @@ func (m *Management) Destroy(d dao.DAO, hash util.Uint160) error { return err } key := MakeContractKey(hash) - err = d.DeleteStorageItem(m.ID, key) - if err != nil { - return err - } - err = d.DeleteContractID(contract.ID) - if err != nil { - return err - } + d.DeleteStorageItem(m.ID, key) + d.DeleteContractID(contract.ID) siArr, err := d.GetStorageItems(contract.ID) if err != nil { return err } for _, kv := range siArr { - err := d.DeleteStorageItem(contract.ID, []byte(kv.Key)) - if err != nil { - return err - } + d.DeleteStorageItem(contract.ID, []byte(kv.Key)) } m.markUpdated(hash) return nil @@ -425,10 +416,7 @@ func (m *Management) setMinimumDeploymentFee(ic *interop.Context, args []stackit if !m.NEO.checkCommittee(ic) { panic("invalid committee signature") } - err := ic.DAO.PutStorageItem(m.ID, keyMinimumDeploymentFee, bigint.ToBytes(value)) - if err != nil { - panic(err) - } + ic.DAO.PutStorageItem(m.ID, keyMinimumDeploymentFee, bigint.ToBytes(value)) return stackitem.Null{} } @@ -564,10 +552,9 @@ func (m *Management) GetNEP17Contracts() []util.Uint160 { // Initialize implements Contract interface. func (m *Management) Initialize(ic *interop.Context) error { - if err := setIntWithKey(m.ID, ic.DAO, keyMinimumDeploymentFee, defaultMinimumDeploymentFee); err != nil { - return err - } - return setIntWithKey(m.ID, ic.DAO, keyNextAvailableID, 1) + setIntWithKey(m.ID, ic.DAO, keyMinimumDeploymentFee, defaultMinimumDeploymentFee) + setIntWithKey(m.ID, ic.DAO, keyNextAvailableID, 1) + return nil } // PutContractState saves given contract state into given DAO. @@ -580,7 +567,8 @@ func (m *Management) PutContractState(d dao.DAO, cs *state.Contract) error { if cs.UpdateCounter != 0 { // Update. return nil } - return d.PutContractID(cs.ID, cs.Hash) + d.PutContractID(cs.ID, cs.Hash) + return nil } func (m *Management) getNextContractID(d dao.DAO) (int32, error) { @@ -592,7 +580,8 @@ func (m *Management) getNextContractID(d dao.DAO) (int32, error) { ret := int32(id.Int64()) id.Add(id, intOne) si = bigint.ToPreallocatedBytes(id, si) - return ret, d.PutStorageItem(m.ID, keyNextAvailableID, si) + d.PutStorageItem(m.ID, keyNextAvailableID, si) + return ret, nil } func (m *Management) emitNotification(ic *interop.Context, name string, hash util.Uint160) { diff --git a/pkg/core/native/management_test.go b/pkg/core/native/management_test.go index 88498e00c..15b6cf0c1 100644 --- a/pkg/core/native/management_test.go +++ b/pkg/core/native/management_test.go @@ -79,7 +79,7 @@ func TestManagement_Initialize(t *testing.T) { t.Run("invalid contract state", func(t *testing.T) { d := dao.NewSimple(storage.NewMemoryStore(), false, false) mgmt := newManagement() - require.NoError(t, d.PutStorageItem(mgmt.ID, []byte{prefixContract}, state.StorageItem{0xFF})) + d.PutStorageItem(mgmt.ID, []byte{prefixContract}, state.StorageItem{0xFF}) require.Error(t, mgmt.InitializeCache(d)) }) } diff --git a/pkg/core/native/native_neo.go b/pkg/core/native/native_neo.go index b5499b2f1..f30ddc993 100644 --- a/pkg/core/native/native_neo.go +++ b/pkg/core/native/native_neo.go @@ -215,10 +215,7 @@ func (n *NEO) Initialize(ic *interop.Context) error { return err } - err = ic.DAO.PutStorageItem(n.ID, prefixCommittee, cvs.Bytes()) - if err != nil { - return err - } + ic.DAO.PutStorageItem(n.ID, prefixCommittee, cvs.Bytes()) h, err := getStandbyValidatorsHash(ic) if err != nil { @@ -228,22 +225,14 @@ func (n *NEO) Initialize(ic *interop.Context) error { var index uint32 value := big.NewInt(5 * GASFactor) - err = n.putGASRecord(ic.DAO, index, value) - if err != nil { - return err - } + n.putGASRecord(ic.DAO, index, value) + gr := &gasRecord{{Index: index, GASPerBlock: *value}} n.gasPerBlock.Store(*gr) n.gasPerBlockChanged.Store(false) - err = ic.DAO.PutStorageItem(n.ID, []byte{prefixVotersCount}, state.StorageItem{}) - if err != nil { - return err - } + ic.DAO.PutStorageItem(n.ID, []byte{prefixVotersCount}, state.StorageItem{}) - err = setIntWithKey(n.ID, ic.DAO, []byte{prefixRegisterPrice}, DefaultRegisterPrice) - if err != nil { - return err - } + setIntWithKey(n.ID, ic.DAO, []byte{prefixRegisterPrice}, DefaultRegisterPrice) n.registerPrice.Store(int64(DefaultRegisterPrice)) n.registerPriceChanged.Store(false) return nil @@ -305,7 +294,8 @@ func (n *NEO) updateCommittee(ic *interop.Context) error { if !votesChanged { // We need to put in storage anyway, as it affects dumps committee := n.committee.Load().(keysWithVotes) - return ic.DAO.PutStorageItem(n.ID, prefixCommittee, committee.Bytes()) + ic.DAO.PutStorageItem(n.ID, prefixCommittee, committee.Bytes()) + return nil } _, cvs, err := n.computeCommitteeMembers(ic.Chain, ic.DAO) @@ -316,7 +306,8 @@ func (n *NEO) updateCommittee(ic *interop.Context) error { return err } n.votesChanged.Store(false) - return ic.DAO.PutStorageItem(n.ID, prefixCommittee, cvs.Bytes()) + ic.DAO.PutStorageItem(n.ID, prefixCommittee, cvs.Bytes()) + return nil } // OnPersist implements Contract interface. @@ -379,9 +370,7 @@ func (n *NEO) PostPersist(ic *interop.Context) error { binary.BigEndian.PutUint32(key[34:], ic.Block.Index+1) n.gasPerVoteCache[cs[i].Key] = *tmp - if err := ic.DAO.PutStorageItem(n.ID, key, bigint.ToBytes(tmp)); err != nil { - return err - } + ic.DAO.PutStorageItem(n.ID, key, bigint.ToBytes(tmp)) } } } @@ -485,10 +474,7 @@ func (n *NEO) distributeGas(ic *interop.Context, h util.Uint160, acc *state.NEOB // Must store acc before GAS distribution to fix acc's BalanceHeight value in the storage for // further acc's queries from `onNEP17Payment` if so, see https://github.com/nspcc-dev/neo-go/pull/2181. key := makeAccountKey(h) - err = ic.DAO.PutStorageItem(n.ID, key, acc.Bytes()) - if err != nil { - return fmt.Errorf("failed to store acc before gas distribution: %w", err) - } + ic.DAO.PutStorageItem(n.ID, key, acc.Bytes()) n.GAS.mint(ic, h, gen, true) return nil @@ -581,7 +567,8 @@ func (n *NEO) SetGASPerBlock(ic *interop.Context, index uint32, gas *big.Int) er return errors.New("invalid committee signature") } n.gasPerBlockChanged.Store(true) - return n.putGASRecord(ic.DAO, index, gas) + n.putGASRecord(ic.DAO, index, gas) + return nil } func (n *NEO) getRegisterPrice(ic *interop.Context, _ []stackitem.Item) stackitem.Item { @@ -604,10 +591,7 @@ func (n *NEO) setRegisterPrice(ic *interop.Context, args []stackitem.Item) stack panic("invalid committee signature") } - err := setIntWithKey(n.ID, ic.DAO, []byte{prefixRegisterPrice}, price.Int64()) - if err != nil { - panic(err) - } + setIntWithKey(n.ID, ic.DAO, []byte{prefixRegisterPrice}, price.Int64()) n.registerPriceChanged.Store(true) return stackitem.Null{} } @@ -616,9 +600,7 @@ func (n *NEO) dropCandidateIfZero(d dao.DAO, pub *keys.PublicKey, c *candidate) if c.Registered || c.Votes.Sign() != 0 { return false, nil } - if err := d.DeleteStorageItem(n.ID, makeValidatorKey(pub)); err != nil { - return true, err - } + d.DeleteStorageItem(n.ID, makeValidatorKey(pub)) var toRemove []string voterKey := makeVoterKey(pub.Bytes()) @@ -627,9 +609,7 @@ func (n *NEO) dropCandidateIfZero(d dao.DAO, pub *keys.PublicKey, c *candidate) return true }) for i := range toRemove { - if err := d.DeleteStorageItem(n.ID, []byte(toRemove[i])); err != nil { - return true, err - } + d.DeleteStorageItem(n.ID, []byte(toRemove[i])) } delete(n.gasPerVoteCache, string(voterKey)) @@ -804,10 +784,7 @@ func (n *NEO) VoteInternal(ic *interop.Context, h util.Uint160, pub *keys.Public return err } // we should put it in storage anyway as it affects dumps - err = ic.DAO.PutStorageItem(n.ID, key, si) - if err != nil { - return err - } + ic.DAO.PutStorageItem(n.ID, key, si) if pub != nil { valKey := makeValidatorKey(pub) valSi := ic.DAO.GetStorageItem(n.ID, valKey) @@ -816,10 +793,7 @@ func (n *NEO) VoteInternal(ic *interop.Context, h util.Uint160, pub *keys.Public } cd := new(candidate).FromBytes(valSi) // we should put it in storage anyway as it affects dumps - err = ic.DAO.PutStorageItem(n.ID, valKey, valSi) - if err != nil { - return err - } + ic.DAO.PutStorageItem(n.ID, valKey, valSi) if !cd.Registered { return errors.New("validator must be registered") } @@ -844,7 +818,8 @@ func (n *NEO) VoteInternal(ic *interop.Context, h util.Uint160, pub *keys.Public if err := n.ModifyAccountVotes(acc, ic.DAO, &acc.Balance, true); err != nil { return err } - return ic.DAO.PutStorageItem(n.ID, key, acc.Bytes()) + ic.DAO.PutStorageItem(n.ID, key, acc.Bytes()) + return nil } // ModifyAccountVotes modifies votes of the specified account by value (can be negative). @@ -987,7 +962,8 @@ func (n *NEO) modifyVoterTurnout(d dao.DAO, amount *big.Int) error { votersCount := bigint.FromBytes(si) votersCount.Add(votersCount, amount) si = bigint.ToPreallocatedBytes(votersCount, si) - return d.PutStorageItem(n.ID, key, si) + d.PutStorageItem(n.ID, key, si) + return nil } // GetCommitteeMembers returns public keys of nodes in committee using cached value. @@ -1105,9 +1081,9 @@ func toPublicKey(s stackitem.Item) *keys.PublicKey { } // putGASRecord is a helper which creates key and puts GASPerBlock value into the storage. -func (n *NEO) putGASRecord(dao dao.DAO, index uint32, value *big.Int) error { +func (n *NEO) putGASRecord(dao dao.DAO, index uint32, value *big.Int) { key := make([]byte, 5) key[0] = prefixGASPerBlock binary.BigEndian.PutUint32(key[1:], index) - return dao.PutStorageItem(n.ID, key, bigint.ToBytes(value)) + dao.PutStorageItem(n.ID, key, bigint.ToBytes(value)) } diff --git a/pkg/core/native/native_nep17.go b/pkg/core/native/native_nep17.go index 0f88df1ad..8ed1bc570 100644 --- a/pkg/core/native/native_nep17.go +++ b/pkg/core/native/native_nep17.go @@ -107,9 +107,9 @@ func (c *nep17TokenNative) getTotalSupply(d dao.DAO) (state.StorageItem, *big.In return si, bigint.FromBytes(si) } -func (c *nep17TokenNative) saveTotalSupply(d dao.DAO, si state.StorageItem, supply *big.Int) error { +func (c *nep17TokenNative) saveTotalSupply(d dao.DAO, si state.StorageItem, supply *big.Int) { si = state.StorageItem(bigint.ToPreallocatedBytes(supply, si)) - return d.PutStorageItem(c.ID, totalSupplyKey, si) + d.PutStorageItem(c.ID, totalSupplyKey, si) } func (c *nep17TokenNative) Transfer(ic *interop.Context, args []stackitem.Item) stackitem.Item { @@ -184,16 +184,16 @@ func (c *nep17TokenNative) updateAccBalance(ic *interop.Context, acc util.Uint16 err := c.incBalance(ic, acc, &si, amount, requiredBalance) if err != nil { if si != nil && amount.Sign() <= 0 { - _ = ic.DAO.PutStorageItem(c.ID, key, si) + ic.DAO.PutStorageItem(c.ID, key, si) } return err } if si == nil { - err = ic.DAO.DeleteStorageItem(c.ID, key) + ic.DAO.DeleteStorageItem(c.ID, key) } else { - err = ic.DAO.PutStorageItem(c.ID, key, si) + ic.DAO.PutStorageItem(c.ID, key, si) } - return err + return nil } // TransferInternal transfers NEO between accounts. @@ -281,22 +281,15 @@ func (c *nep17TokenNative) addTokens(ic *interop.Context, h util.Uint160, amount if err := c.incBalance(ic, h, &si, amount, nil); err != nil { panic(err) } - var err error if si == nil { - err = ic.DAO.DeleteStorageItem(c.ID, key) + ic.DAO.DeleteStorageItem(c.ID, key) } else { - err = ic.DAO.PutStorageItem(c.ID, key, si) - } - if err != nil { - panic(err) + ic.DAO.PutStorageItem(c.ID, key, si) } buf, supply := c.getTotalSupply(ic.DAO) supply.Add(supply, amount) - err = c.saveTotalSupply(ic.DAO, buf, supply) - if err != nil { - panic(err) - } + c.saveTotalSupply(ic.DAO, buf, supply) } func newDescriptor(name string, ret smartcontract.ParamType, ps ...manifest.Parameter) *manifest.Method { diff --git a/pkg/core/native/notary.go b/pkg/core/native/notary.go index ad5b0a033..9b86f6c97 100644 --- a/pkg/core/native/notary.go +++ b/pkg/core/native/notary.go @@ -109,11 +109,7 @@ func (n *Notary) Metadata() *interop.ContractMD { // Initialize initializes Notary native contract and implements Contract interface. func (n *Notary) Initialize(ic *interop.Context) error { - err := setIntWithKey(n.ID, ic.DAO, maxNotValidBeforeDeltaKey, defaultMaxNotValidBeforeDelta) - if err != nil { - return err - } - + setIntWithKey(n.ID, ic.DAO, maxNotValidBeforeDeltaKey, defaultMaxNotValidBeforeDelta) n.isValid = true n.maxNotValidBeforeDelta = defaultMaxNotValidBeforeDelta return nil @@ -141,10 +137,7 @@ func (n *Notary) OnPersist(ic *interop.Context) error { balance := n.GetDepositFor(ic.DAO, payer.Account) balance.Amount.Sub(balance.Amount, big.NewInt(tx.SystemFee+tx.NetworkFee)) if balance.Amount.Sign() == 0 { - err := n.removeDepositFor(ic.DAO, payer.Account) - if err != nil { - return fmt.Errorf("failed to remove an empty deposit for %s from storage: %w", payer.Account.StringBE(), err) - } + n.removeDepositFor(ic.DAO, payer.Account) } else { err := n.putDepositFor(ic.DAO, balance, payer.Account) if err != nil { @@ -289,9 +282,7 @@ func (n *Notary) withdraw(ic *interop.Context, args []stackitem.Item) stackitem. if !ic.VM.Estack().Pop().Bool() { panic("failed to transfer GAS from Notary account: `transfer` returned false") } - if err := n.removeDepositFor(ic.DAO, from); err != nil { - panic(fmt.Errorf("failed to remove withdrawn deposit for %s from the storage: %w", from.StringBE(), err)) - } + n.removeDepositFor(ic.DAO, from) return stackitem.NewBool(true) } @@ -402,10 +393,7 @@ func (n *Notary) setMaxNotValidBeforeDelta(ic *interop.Context, args []stackitem } n.lock.Lock() defer n.lock.Unlock() - err := setIntWithKey(n.ID, ic.DAO, maxNotValidBeforeDeltaKey, int64(value)) - if err != nil { - panic(fmt.Errorf("failed to put value into the storage: %w", err)) - } + setIntWithKey(n.ID, ic.DAO, maxNotValidBeforeDeltaKey, int64(value)) n.isValid = false return stackitem.Null{} } @@ -432,9 +420,9 @@ func (n *Notary) putDepositFor(dao dao.DAO, deposit *state.Deposit, acc util.Uin } // removeDepositFor removes deposit from the storage. -func (n *Notary) removeDepositFor(dao dao.DAO, acc util.Uint160) error { +func (n *Notary) removeDepositFor(dao dao.DAO, acc util.Uint160) { key := append([]byte{prefixDeposit}, acc.BytesBE()...) - return dao.DeleteStorageItem(n.ID, key) + dao.DeleteStorageItem(n.ID, key) } // calculateNotaryReward calculates the reward for a single notary node based on FEE's count and Notary nodes count. diff --git a/pkg/core/native/oracle.go b/pkg/core/native/oracle.go index d0e2935b6..ad09b1d13 100644 --- a/pkg/core/native/oracle.go +++ b/pkg/core/native/oracle.go @@ -166,9 +166,7 @@ func (o *Oracle) PostPersist(ic *interop.Context) error { if err := o.getConvertibleFromDAO(ic.DAO, reqKey, req); err != nil { continue } - if err := ic.DAO.DeleteStorageItem(o.ID, reqKey); err != nil { - return err - } + ic.DAO.DeleteStorageItem(o.ID, reqKey) if orc != nil { removedIDs = append(removedIDs, resp.ID) } @@ -184,7 +182,7 @@ func (o *Oracle) PostPersist(ic *interop.Context) error { var err error if len(*idList) == 0 { - err = ic.DAO.DeleteStorageItem(o.ID, idKey) + ic.DAO.DeleteStorageItem(o.ID, idKey) } else { err = putConvertibleToDAO(o.ID, ic.DAO, idKey, idList) } @@ -222,12 +220,8 @@ func (o *Oracle) Metadata() *interop.ContractMD { // Initialize initializes Oracle contract. func (o *Oracle) Initialize(ic *interop.Context) error { - if err := setIntWithKey(o.ID, ic.DAO, prefixRequestID, 0); err != nil { - return err - } - if err := setIntWithKey(o.ID, ic.DAO, prefixRequestPrice, DefaultOracleRequestPrice); err != nil { - return err - } + setIntWithKey(o.ID, ic.DAO, prefixRequestID, 0) + setIntWithKey(o.ID, ic.DAO, prefixRequestPrice, DefaultOracleRequestPrice) o.requestPrice.Store(int64(DefaultOracleRequestPrice)) o.requestPriceChanged.Store(false) return nil @@ -348,9 +342,7 @@ func (o *Oracle) RequestInternal(ic *interop.Context, url string, filter *string id := itemID.Uint64() itemID.Add(itemID, intOne) si = bigint.ToPreallocatedBytes(itemID, si) - if err := ic.DAO.PutStorageItem(o.ID, prefixRequestID, si); err != nil { - return err - } + ic.DAO.PutStorageItem(o.ID, prefixRequestID, si) // Should be executed from contract. _, err := ic.GetContract(ic.VM.GetCallingScriptHash()) @@ -463,9 +455,7 @@ func (o *Oracle) setPrice(ic *interop.Context, args []stackitem.Item) stackitem. if !o.NEO.checkCommittee(ic) { panic("invalid committee signature") } - if err := setIntWithKey(o.ID, ic.DAO, prefixRequestPrice, price.Int64()); err != nil { - panic(err) - } + setIntWithKey(o.ID, ic.DAO, prefixRequestPrice, price.Int64()) o.requestPriceChanged.Store(true) return stackitem.Null{} } diff --git a/pkg/core/native/policy.go b/pkg/core/native/policy.go index dffe1371c..0bd818698 100644 --- a/pkg/core/native/policy.go +++ b/pkg/core/native/policy.go @@ -123,15 +123,9 @@ func (p *Policy) Metadata() *interop.ContractMD { // Initialize initializes Policy native contract and implements Contract interface. func (p *Policy) Initialize(ic *interop.Context) error { - if err := setIntWithKey(p.ID, ic.DAO, feePerByteKey, defaultFeePerByte); err != nil { - return err - } - if err := setIntWithKey(p.ID, ic.DAO, execFeeFactorKey, defaultExecFeeFactor); err != nil { - return err - } - if err := setIntWithKey(p.ID, ic.DAO, storagePriceKey, DefaultStoragePrice); err != nil { - return err - } + setIntWithKey(p.ID, ic.DAO, feePerByteKey, defaultFeePerByte) + setIntWithKey(p.ID, ic.DAO, execFeeFactorKey, defaultExecFeeFactor) + setIntWithKey(p.ID, ic.DAO, storagePriceKey, DefaultStoragePrice) p.isValid = true p.execFeeFactor = defaultExecFeeFactor @@ -229,10 +223,7 @@ func (p *Policy) setExecFeeFactor(ic *interop.Context, args []stackitem.Item) st } p.lock.Lock() defer p.lock.Unlock() - err := setIntWithKey(p.ID, ic.DAO, execFeeFactorKey, int64(value)) - if err != nil { - panic(err) - } + setIntWithKey(p.ID, ic.DAO, execFeeFactorKey, int64(value)) p.isValid = false return stackitem.Null{} } @@ -285,10 +276,7 @@ func (p *Policy) setStoragePrice(ic *interop.Context, args []stackitem.Item) sta } p.lock.Lock() defer p.lock.Unlock() - err := setIntWithKey(p.ID, ic.DAO, storagePriceKey, int64(value)) - if err != nil { - panic(err) - } + setIntWithKey(p.ID, ic.DAO, storagePriceKey, int64(value)) p.isValid = false return stackitem.Null{} } @@ -304,10 +292,7 @@ func (p *Policy) setFeePerByte(ic *interop.Context, args []stackitem.Item) stack } p.lock.Lock() defer p.lock.Unlock() - err := setIntWithKey(p.ID, ic.DAO, feePerByteKey, value) - if err != nil { - panic(err) - } + setIntWithKey(p.ID, ic.DAO, feePerByteKey, value) p.isValid = false return stackitem.Null{} } @@ -330,10 +315,7 @@ func (p *Policy) blockAccount(ic *interop.Context, args []stackitem.Item) stacki key := append([]byte{blockedAccountPrefix}, hash.BytesBE()...) p.lock.Lock() defer p.lock.Unlock() - err := ic.DAO.PutStorageItem(p.ID, key, state.StorageItem{}) - if err != nil { - panic(err) - } + ic.DAO.PutStorageItem(p.ID, key, state.StorageItem{}) p.isValid = false return stackitem.NewBool(true) } @@ -351,10 +333,7 @@ func (p *Policy) unblockAccount(ic *interop.Context, args []stackitem.Item) stac key := append([]byte{blockedAccountPrefix}, hash.BytesBE()...) p.lock.Lock() defer p.lock.Unlock() - err := ic.DAO.DeleteStorageItem(p.ID, key) - if err != nil { - panic(err) - } + ic.DAO.DeleteStorageItem(p.ID, key) p.isValid = false return stackitem.NewBool(true) } diff --git a/pkg/core/native/util.go b/pkg/core/native/util.go index 0fadf740d..a150d1e3e 100644 --- a/pkg/core/native/util.go +++ b/pkg/core/native/util.go @@ -28,11 +28,12 @@ func putConvertibleToDAO(id int32, d dao.DAO, key []byte, conv stackitem.Convert if err != nil { return err } - return d.PutStorageItem(id, key, data) + d.PutStorageItem(id, key, data) + return nil } -func setIntWithKey(id int32, dao dao.DAO, key []byte, value int64) error { - return dao.PutStorageItem(id, key, bigint.ToBytes(big.NewInt(value))) +func setIntWithKey(id int32, dao dao.DAO, key []byte, value int64) { + dao.PutStorageItem(id, key, bigint.ToBytes(big.NewInt(value))) } func getIntWithKey(id int32, dao dao.DAO, key []byte) int64 { diff --git a/pkg/core/stateroot/module.go b/pkg/core/stateroot/module.go index 30c6fe310..b09c5a953 100644 --- a/pkg/core/stateroot/module.go +++ b/pkg/core/stateroot/module.go @@ -149,7 +149,7 @@ func (s *Module) CleanStorage() error { b := storage.NewMemCachedStore(s.Store) s.Store.Seek(storage.SeekRange{Prefix: []byte{byte(storage.DataMPT)}}, func(k, _ []byte) bool { // #1468, but don't need to copy here, because it is done by Store. - _ = b.Delete(k) + b.Delete(k) return true }) _, err := b.Persist() @@ -160,22 +160,17 @@ func (s *Module) CleanStorage() error { } // JumpToState performs jump to the state specified by given stateroot index. -func (s *Module) JumpToState(sr *state.MPTRoot) error { - if err := s.addLocalStateRoot(s.Store, sr); err != nil { - return fmt.Errorf("failed to store local state root: %w", err) - } +func (s *Module) JumpToState(sr *state.MPTRoot) { + s.addLocalStateRoot(s.Store, sr) data := make([]byte, 4) binary.LittleEndian.PutUint32(data, sr.Index) - if err := s.Store.Put([]byte{byte(storage.DataMPTAux), prefixValidated}, data); err != nil { - return fmt.Errorf("failed to store validated height: %w", err) - } + s.Store.Put([]byte{byte(storage.DataMPTAux), prefixValidated}, data) s.validatedHeight.Store(sr.Index) s.currentLocal.Store(sr.Root) s.localHeight.Store(sr.Index) s.mpt = mpt.NewTrie(mpt.NewHashNode(sr.Root), s.mode, s.Store) - return nil } // GC performs garbage collection. @@ -225,11 +220,8 @@ func (s *Module) AddMPTBatch(index uint32, b mpt.Batch, cache *storage.MemCached Index: index, Root: mpt.StateRoot(), } - err := s.addLocalStateRoot(cache, sr) - if err != nil { - return nil, nil, err - } - return &mpt, sr, err + s.addLocalStateRoot(cache, sr) + return &mpt, sr, nil } // UpdateCurrentLocal updates local caches using provided state root. diff --git a/pkg/core/stateroot/store.go b/pkg/core/stateroot/store.go index 6c1494cc2..be6529ca4 100644 --- a/pkg/core/stateroot/store.go +++ b/pkg/core/stateroot/store.go @@ -21,21 +21,19 @@ const ( prefixValidated = 0x03 ) -func (s *Module) addLocalStateRoot(store *storage.MemCachedStore, sr *state.MPTRoot) error { +func (s *Module) addLocalStateRoot(store *storage.MemCachedStore, sr *state.MPTRoot) { key := makeStateRootKey(sr.Index) - if err := putStateRoot(store, key, sr); err != nil { - return err - } + putStateRoot(store, key, sr) data := make([]byte, 4) binary.LittleEndian.PutUint32(data, sr.Index) - return store.Put([]byte{byte(storage.DataMPTAux), prefixLocal}, data) + store.Put([]byte{byte(storage.DataMPTAux), prefixLocal}, data) } -func putStateRoot(store *storage.MemCachedStore, key []byte, sr *state.MPTRoot) error { +func putStateRoot(store *storage.MemCachedStore, key []byte, sr *state.MPTRoot) { w := io.NewBufBinWriter() sr.EncodeBinary(w.BinWriter) - return store.Put(key, w.Bytes()) + store.Put(key, w.Bytes()) } func (s *Module) getStateRoot(key []byte) (*state.MPTRoot, error) { @@ -73,15 +71,11 @@ func (s *Module) AddStateRoot(sr *state.MPTRoot) error { if len(local.Witness) != 0 { return nil } - if err := putStateRoot(s.Store, key, sr); err != nil { - return err - } + putStateRoot(s.Store, key, sr) data := make([]byte, 4) binary.LittleEndian.PutUint32(data, sr.Index) - if err := s.Store.Put([]byte{byte(storage.DataMPTAux), prefixValidated}, data); err != nil { - return err - } + s.Store.Put([]byte{byte(storage.DataMPTAux), prefixValidated}, data) s.validatedHeight.Store(sr.Index) if !s.srInHead { updateStateHeightMetric(sr.Index) diff --git a/pkg/core/statesync/module.go b/pkg/core/statesync/module.go index 4157ae983..912dcaaf1 100644 --- a/pkg/core/statesync/module.go +++ b/pkg/core/statesync/module.go @@ -167,10 +167,7 @@ func (s *Module) Init(currChainHeight uint32) error { } s.syncPoint = p - err = s.dao.PutStateSyncPoint(p) - if err != nil { - return fmt.Errorf("failed to store state synchronisation point %d: %w", p, err) - } + s.dao.PutStateSyncPoint(p) s.syncStage = initialized s.log.Info("try to sync state for the latest state synchronisation point", zap.Uint32("point", p), @@ -339,10 +336,7 @@ func (s *Module) AddBlock(block *block.Block) error { } writeBuf.Reset() - err := cache.PutStateSyncCurrentBlockHeight(block.Index) - if err != nil { - return fmt.Errorf("failed to store current block height: %w", err) - } + cache.PutStateSyncCurrentBlockHeight(block.Index) for _, tx := range block.Transactions { if err := cache.StoreAsTransaction(tx, block.Index, nil, writeBuf); err != nil { @@ -351,7 +345,7 @@ func (s *Module) AddBlock(block *block.Block) error { writeBuf.Reset() } - _, err = cache.Persist() + _, err := cache.Persist() if err != nil { return fmt.Errorf("failed to persist results: %w", err) } diff --git a/pkg/core/storage/memcached_store.go b/pkg/core/storage/memcached_store.go index 7602a113d..962bb68d8 100644 --- a/pkg/core/storage/memcached_store.go +++ b/pkg/core/storage/memcached_store.go @@ -65,23 +65,21 @@ func (s *MemCachedStore) Get(key []byte) ([]byte, error) { return s.ps.Get(key) } -// Put puts new KV pair into the store. Never returns an error. -func (s *MemCachedStore) Put(key, value []byte) error { +// Put puts new KV pair into the store. +func (s *MemCachedStore) Put(key, value []byte) { newKey := string(key) vcopy := slice.Copy(value) s.mut.Lock() put(s.chooseMap(key), newKey, vcopy) s.mut.Unlock() - return nil } // Delete drops KV pair from the store. Never returns an error. -func (s *MemCachedStore) Delete(key []byte) error { +func (s *MemCachedStore) Delete(key []byte) { newKey := string(key) s.mut.Lock() put(s.chooseMap(key), newKey, nil) s.mut.Unlock() - return nil } // GetBatch returns currently accumulated changeset. diff --git a/pkg/core/storage/memcached_store_test.go b/pkg/core/storage/memcached_store_test.go index 0b7565e6b..16a391e7c 100644 --- a/pkg/core/storage/memcached_store_test.go +++ b/pkg/core/storage/memcached_store_test.go @@ -18,26 +18,32 @@ func TestMemCachedPutGetDelete(t *testing.T) { key := []byte("foo") value := []byte("bar") - require.NoError(t, s.Put(key, value)) + s.Put(key, value) result, err := s.Get(key) assert.Nil(t, err) require.Equal(t, value, result) - err = s.Delete(key) - assert.Nil(t, err) + s.Delete(key) _, err = s.Get(key) assert.NotNil(t, err) assert.Equal(t, err, ErrKeyNotFound) // Double delete. - err = s.Delete(key) - assert.Nil(t, err) + s.Delete(key) + + _, err = s.Get(key) + assert.NotNil(t, err) + assert.Equal(t, err, ErrKeyNotFound) // Nonexistent. key = []byte("sparse") - assert.NoError(t, s.Delete(key)) + s.Delete(key) + + _, err = s.Get(key) + assert.NotNil(t, err) + assert.Equal(t, err, ErrKeyNotFound) } func testMemCachedStorePersist(t *testing.T, ps Store) { @@ -48,7 +54,7 @@ func testMemCachedStorePersist(t *testing.T, ps Store) { assert.Equal(t, nil, err) assert.Equal(t, 0, c) // persisting one key should result in one key in ps and nothing in ts - assert.NoError(t, ts.Put([]byte("key"), []byte("value"))) + ts.Put([]byte("key"), []byte("value")) checkBatch(t, ts, []KeyValueExists{{KeyValue: KeyValue{Key: []byte("key"), Value: []byte("value")}}}, nil) c, err = ts.Persist() checkBatch(t, ts, nil, nil) @@ -61,8 +67,8 @@ func testMemCachedStorePersist(t *testing.T, ps Store) { assert.Equal(t, ErrKeyNotFound, err) assert.Equal(t, []byte(nil), v) // now we overwrite the previous `key` contents and also add `key2`, - assert.NoError(t, ts.Put([]byte("key"), []byte("newvalue"))) - assert.NoError(t, ts.Put([]byte("key2"), []byte("value2"))) + ts.Put([]byte("key"), []byte("newvalue")) + ts.Put([]byte("key2"), []byte("value2")) // this is to check that now key is written into the ps before we do // persist v, err = ps.Get([]byte("key2")) @@ -96,8 +102,7 @@ func testMemCachedStorePersist(t *testing.T, ps Store) { assert.Equal(t, nil, err) assert.Equal(t, 0, c) // test persisting deletions - err = ts.Delete([]byte("key")) - assert.Equal(t, nil, err) + ts.Delete([]byte("key")) checkBatch(t, ts, nil, []KeyValueExists{{KeyValue: KeyValue{Key: []byte("key")}, Exists: true}}) c, err = ts.Persist() checkBatch(t, ts, nil, nil) @@ -155,7 +160,7 @@ func TestCachedGetFromPersistent(t *testing.T) { val, err := ts.Get(key) assert.Nil(t, err) assert.Equal(t, value, val) - assert.NoError(t, ts.Delete(key)) + ts.Delete(key) val, err = ts.Get(key) assert.Equal(t, err, ErrKeyNotFound) assert.Nil(t, val) @@ -188,11 +193,11 @@ func TestCachedSeek(t *testing.T) { } for _, v := range deletedKVs { require.NoError(t, ps.PutChangeSet(map[string][]byte{string(v.Key): v.Value}, nil)) - require.NoError(t, ts.Delete(v.Key)) + ts.Delete(v.Key) } for _, v := range updatedKVs { require.NoError(t, ps.PutChangeSet(map[string][]byte{string(v.Key): v.Value}, nil)) - require.NoError(t, ts.Put(v.Key, v.Value)) + ts.Put(v.Key, v.Value) } foundKVs := make(map[string][]byte) ts.Seek(SeekRange{Prefix: goodPrefix}, func(k, v []byte) bool { @@ -227,34 +232,34 @@ func benchmarkCachedSeek(t *testing.B, ps Store, psElementsCount, tsElementsCoun ) for i := 0; i < psElementsCount; i++ { // lower KVs with matching prefix that should be found - require.NoError(t, ts.Put(append(lowerPrefixGood, random.Bytes(10)...), []byte("value"))) + ts.Put(append(lowerPrefixGood, random.Bytes(10)...), []byte("value")) // lower KVs with non-matching prefix that shouldn't be found - require.NoError(t, ts.Put(append(lowerPrefixBad, random.Bytes(10)...), []byte("value"))) + ts.Put(append(lowerPrefixBad, random.Bytes(10)...), []byte("value")) // deleted KVs with matching prefix that shouldn't be found key := append(deletedPrefixGood, random.Bytes(10)...) - require.NoError(t, ts.Put(key, []byte("deleted"))) + ts.Put(key, []byte("deleted")) if i < tsElementsCount { - require.NoError(t, ts.Delete(key)) + ts.Delete(key) } // deleted KVs with non-matching prefix that shouldn't be found key = append(deletedPrefixBad, random.Bytes(10)...) - require.NoError(t, ts.Put(key, []byte("deleted"))) + ts.Put(key, []byte("deleted")) if i < tsElementsCount { - require.NoError(t, ts.Delete(key)) + ts.Delete(key) } // updated KVs with matching prefix that should be found key = append(updatedPrefixGood, random.Bytes(10)...) - require.NoError(t, ts.Put(key, []byte("stub"))) + ts.Put(key, []byte("stub")) if i < tsElementsCount { - require.NoError(t, ts.Put(key, []byte("updated"))) + ts.Put(key, []byte("updated")) } // updated KVs with non-matching prefix that shouldn't be found key = append(updatedPrefixBad, random.Bytes(10)...) - require.NoError(t, ts.Put(key, []byte("stub"))) + ts.Put(key, []byte("stub")) if i < tsElementsCount { - require.NoError(t, ts.Put(key, []byte("updated"))) + ts.Put(key, []byte("updated")) } } _, err := ts.PersistSync() @@ -329,13 +334,13 @@ func TestMemCachedPersistFailing(t *testing.T) { // cached Store ts := NewMemCachedStore(&bs) // Set a pair of keys. - require.NoError(t, ts.Put(t1, t1)) - require.NoError(t, ts.Put(t2, t2)) + ts.Put(t1, t1) + ts.Put(t2, t2) // This will be called during Persist(). bs.onPutBatch = func() { // Drop one, add one. - require.NoError(t, ts.Put(b1, b1)) - require.NoError(t, ts.Delete(t1)) + ts.Put(b1, b1) + ts.Delete(t1) } _, err := ts.Persist() require.Error(t, err) @@ -381,11 +386,11 @@ func TestCachedSeekSorting(t *testing.T) { } for _, v := range deletedKVs { require.NoError(t, ps.PutChangeSet(map[string][]byte{string(v.Key): v.Value}, nil)) - require.NoError(t, ts.Delete(v.Key)) + ts.Delete(v.Key) } for _, v := range updatedKVs { require.NoError(t, ps.PutChangeSet(map[string][]byte{string(v.Key): v.Value}, nil)) - require.NoError(t, ts.Put(v.Key, v.Value)) + ts.Put(v.Key, v.Value) } var foundKVs []KeyValue ts.Seek(SeekRange{Prefix: goodPrefix}, func(k, v []byte) bool { diff --git a/pkg/core/storage/memory_store_test.go b/pkg/core/storage/memory_store_test.go index eb3770c6a..544887ac1 100644 --- a/pkg/core/storage/memory_store_test.go +++ b/pkg/core/storage/memory_store_test.go @@ -22,8 +22,8 @@ func BenchmarkMemorySeek(t *testing.B) { ) ts := NewMemCachedStore(ms) for i := 0; i < count; i++ { - require.NoError(t, ts.Put(append(searchPrefix, random.Bytes(10)...), random.Bytes(10))) - require.NoError(t, ts.Put(append(badPrefix, random.Bytes(10)...), random.Bytes(10))) + ts.Put(append(searchPrefix, random.Bytes(10)...), random.Bytes(10)) + ts.Put(append(badPrefix, random.Bytes(10)...), random.Bytes(10)) } _, err := ts.PersistSync() require.NoError(t, err) diff --git a/pkg/core/storage/storeandbatch_test.go b/pkg/core/storage/storeandbatch_test.go index 0bc8917c5..88724fe10 100644 --- a/pkg/core/storage/storeandbatch_test.go +++ b/pkg/core/storage/storeandbatch_test.go @@ -39,7 +39,7 @@ func pushSeekDataSet(t *testing.T, s Store) []KeyValue { } up := NewMemCachedStore(s) for _, v := range kvs { - require.NoError(t, up.Put(v.Key, v.Value)) + up.Put(v.Key, v.Value) } _, err := up.PersistSync() require.NoError(t, err)