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.
This commit is contained in:
Roman Khimov 2022-02-16 17:48:15 +03:00
parent be24bf6412
commit 9d2ef775cf
26 changed files with 228 additions and 380 deletions

View file

@ -316,9 +316,7 @@ func (bc *Blockchain) init() error {
KeepOnlyLatestState: bc.config.KeepOnlyLatestState, KeepOnlyLatestState: bc.config.KeepOnlyLatestState,
Value: version, Value: version,
} }
if err = bc.dao.PutVersion(ver); err != nil { bc.dao.PutVersion(ver)
return err
}
bc.dao.Version = ver bc.dao.Version = ver
bc.persistent.Version = ver bc.persistent.Version = ver
genesisBlock, err := createGenesisBlock(bc.config) genesisBlock, err := createGenesisBlock(bc.config)
@ -326,10 +324,7 @@ func (bc *Blockchain) init() error {
return err return err
} }
bc.headerHashes = []util.Uint256{genesisBlock.Hash()} bc.headerHashes = []util.Uint256{genesisBlock.Hash()}
err = bc.dao.PutCurrentHeader(hashAndIndexToBytes(genesisBlock.Hash(), genesisBlock.Index)) bc.dao.PutCurrentHeader(hashAndIndexToBytes(genesisBlock.Hash(), genesisBlock.Index))
if err != nil {
return err
}
if err := bc.stateRoot.Init(0); err != nil { if err := bc.stateRoot.Init(0); err != nil {
return fmt.Errorf("can't init MPT: %w", err) 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() jumpStageKey := storage.SYSStateJumpStage.Bytes()
switch stage { switch stage {
case none: case none:
err := bc.dao.Store.Put(jumpStageKey, []byte{byte(stateJumpStarted)}) bc.dao.Store.Put(jumpStageKey, []byte{byte(stateJumpStarted)})
if err != nil {
return fmt.Errorf("failed to store state jump stage: %w", err)
}
fallthrough fallthrough
case stateJumpStarted: case stateJumpStarted:
newPrefix := statesync.TemporaryPrefix(bc.dao.Version.StoragePrefix) 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) return fmt.Errorf("failed to get dao.Version: %w", err)
} }
v.StoragePrefix = newPrefix v.StoragePrefix = newPrefix
if err := bc.dao.PutVersion(v); err != nil { bc.dao.PutVersion(v)
return fmt.Errorf("failed to update dao.Version: %w", err)
}
bc.persistent.Version = v bc.persistent.Version = v
err = bc.dao.Store.Put(jumpStageKey, []byte{byte(newStorageItemsAdded)}) bc.dao.Store.Put(jumpStageKey, []byte{byte(newStorageItemsAdded)})
if err != nil {
return fmt.Errorf("failed to store state jump stage: %w", err)
}
fallthrough fallthrough
case newStorageItemsAdded: case newStorageItemsAdded:
@ -536,7 +523,7 @@ func (bc *Blockchain) jumpToStateInternal(p uint32, stage stateJumpStage) error
prefix := statesync.TemporaryPrefix(bc.dao.Version.StoragePrefix) prefix := statesync.TemporaryPrefix(bc.dao.Version.StoragePrefix)
bc.dao.Store.Seek(storage.SeekRange{Prefix: []byte{byte(prefix)}}, func(k, _ []byte) bool { 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. // #1468, but don't need to copy here, because it is done by Store.
_ = cache.Store.Delete(k) cache.Store.Delete(k)
return true 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)} prefixes := []byte{byte(storage.STNEP11Transfers), byte(storage.STNEP17Transfers), byte(storage.STTokenTransferInfo)}
for i := range prefixes { for i := range prefixes {
cache.Store.Seek(storage.SeekRange{Prefix: prefixes[i : i+1]}, func(k, v []byte) bool { 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 return true
}) })
} }
} }
_ = cache.Store.Put(jumpStageKey, []byte{byte(genesisStateRemoved)}) cache.Store.Put(jumpStageKey, []byte{byte(genesisStateRemoved)})
_, err := cache.Persist() _, err := cache.Persist()
if err != nil { if err != nil {
return fmt.Errorf("failed to persist old items removal: %w", err) 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) return fmt.Errorf("failed to get current block: %w", err)
} }
writeBuf.Reset() writeBuf.Reset()
err = bc.dao.StoreAsCurrentBlock(block, writeBuf) bc.dao.StoreAsCurrentBlock(block, writeBuf)
if err != nil {
return fmt.Errorf("failed to store current block: %w", err)
}
bc.topBlock.Store(block) bc.topBlock.Store(block)
atomic.StoreUint32(&bc.blockHeight, p) atomic.StoreUint32(&bc.blockHeight, p)
atomic.StoreUint32(&bc.persistedHeight, p) atomic.StoreUint32(&bc.persistedHeight, p)
@ -585,12 +569,10 @@ func (bc *Blockchain) jumpToStateInternal(p uint32, stage stateJumpStage) error
if err != nil { if err != nil {
return fmt.Errorf("failed to get block to init MPT: %w", err) 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, Index: p,
Root: block.PrevStateRoot, 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) err = bc.contracts.NEO.InitializeCache(bc, bc.dao)
if err != nil { if err != nil {
@ -608,10 +590,7 @@ func (bc *Blockchain) jumpToStateInternal(p uint32, stage stateJumpStage) error
updateBlockHeightMetric(p) updateBlockHeightMetric(p)
err = bc.dao.Store.Delete(jumpStageKey) bc.dao.Store.Delete(jumpStageKey)
if err != nil {
return fmt.Errorf("failed to remove outdated state jump stage: %w", err)
}
return nil return nil
} }
@ -973,7 +952,7 @@ func (bc *Blockchain) addHeaders(verify bool, headers ...*block.Header) error {
} }
key := storage.AppendPrefix(storage.DataExecutable, h.Hash().BytesBE()) key := storage.AppendPrefix(storage.DataExecutable, h.Hash().BytesBE())
_ = batch.Store.Put(key, buf.Bytes()) batch.Store.Put(key, buf.Bytes())
buf.Reset() buf.Reset()
lastHeader = h lastHeader = h
} }
@ -986,11 +965,11 @@ func (bc *Blockchain) addHeaders(verify bool, headers ...*block.Header) error {
} }
key := storage.AppendPrefixInt(storage.IXHeaderHashList, int(bc.storedHeaderCount)) key := storage.AppendPrefixInt(storage.IXHeaderHashList, int(bc.storedHeaderCount))
_ = batch.Store.Put(key, buf.Bytes()) batch.Store.Put(key, buf.Bytes())
bc.storedHeaderCount += headerBatchCount 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) updateHeaderHeightMetric(len(bc.headerHashes) - 1)
if _, err = batch.Persist(); err != nil { if _, err = batch.Persist(); err != nil {
return err return err
@ -1033,10 +1012,7 @@ func (bc *Blockchain) storeBlock(block *block.Block, txpool *mempool.Pool) error
baer1, baer2 *state.AppExecResult baer1, baer2 *state.AppExecResult
transCache = make(map[util.Uint160]transferData) transCache = make(map[util.Uint160]transferData)
) )
if err := kvcache.StoreAsCurrentBlock(block, writeBuf); err != nil { kvcache.StoreAsCurrentBlock(block, writeBuf)
aerdone <- err
return
}
writeBuf.Reset() writeBuf.Reset()
if bc.config.RemoveUntraceableBlocks { if bc.config.RemoveUntraceableBlocks {
var start, stop uint32 var start, stop uint32
@ -1102,18 +1078,10 @@ func (bc *Blockchain) storeBlock(block *block.Block, txpool *mempool.Pool) error
return return
} }
if !trData.Info.NewNEP11Batch { if !trData.Info.NewNEP11Batch {
err = kvcache.PutTokenTransferLog(acc, trData.Info.NextNEP11NewestTimestamp, trData.Info.NextNEP11Batch, true, &trData.Log11) kvcache.PutTokenTransferLog(acc, trData.Info.NextNEP11NewestTimestamp, trData.Info.NextNEP11Batch, true, &trData.Log11)
if err != nil {
aerdone <- err
return
}
} }
if !trData.Info.NewNEP17Batch { if !trData.Info.NewNEP17Batch {
err = kvcache.PutTokenTransferLog(acc, trData.Info.NextNEP17NewestTimestamp, trData.Info.NextNEP17Batch, false, &trData.Log17) kvcache.PutTokenTransferLog(acc, trData.Info.NextNEP17NewestTimestamp, trData.Info.NextNEP17Batch, false, &trData.Log17)
if err != nil {
aerdone <- err
return
}
} }
} }
close(aerdone) close(aerdone)
@ -1471,10 +1439,7 @@ func appendTokenTransfer(cache dao.DAO, transCache map[util.Uint160]transferData
transferData.Info.LastUpdated[token] = bIndex transferData.Info.LastUpdated[token] = bIndex
*newBatch = log.Size() >= state.TokenTransferBatchSize *newBatch = log.Size() >= state.TokenTransferBatchSize
if *newBatch { if *newBatch {
err = cache.PutTokenTransferLog(addr, *currTimestamp, *nextBatch, isNEP11, log) cache.PutTokenTransferLog(addr, *currTimestamp, *nextBatch, isNEP11, log)
if err != nil {
return err
}
*nextBatch++ *nextBatch++
*currTimestamp = bTimestamp *currTimestamp = bTimestamp
// Put makes a copy of it anyway. // Put makes a copy of it anyway.

View file

@ -1385,7 +1385,7 @@ func TestClose(t *testing.T) {
// It's a hack, but we use internal knowledge of MemoryStore // It's a hack, but we use internal knowledge of MemoryStore
// implementation which makes it completely unusable (up to panicing) // implementation which makes it completely unusable (up to panicing)
// after Close(). // after Close().
_ = bc.dao.Store.Put([]byte{0}, []byte{1}) bc.dao.Store.Put([]byte{0}, []byte{1})
// This should never be executed. // This should never be executed.
assert.Nil(t, t) 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. ttl := state.TokenTransferLog{Raw: []byte{1}} // It's incorrect, but who cares.
for i := uint32(0); i < 3; i++ { 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++ { 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++ { 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++ { 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() _, err = bc.dao.Persist()
@ -1855,7 +1855,7 @@ func TestBlockchain_InitWithIncompleteStateJump(t *testing.T) {
key := slice.Copy(k) key := slice.Copy(k)
key[0] = byte(tempPrefix) key[0] = byte(tempPrefix)
value := slice.Copy(v) value := slice.Copy(v)
_ = batch.Put(key, value) batch.Put(key, value)
return true return true
}) })
_, err := batch.Persist() _, err := batch.Persist()
@ -1879,33 +1879,33 @@ func TestBlockchain_InitWithIncompleteStateJump(t *testing.T) {
} }
// manually store statejump stage to check statejump recover process // manually store statejump stage to check statejump recover process
t.Run("invalid RemoveUntraceableBlocks setting", func(t *testing.T) { 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) { checkNewBlockchainErr(t, func(c *config.Config) {
boltCfg(c) boltCfg(c)
c.ProtocolConfiguration.RemoveUntraceableBlocks = false c.ProtocolConfiguration.RemoveUntraceableBlocks = false
}, bcSpout.dao.Store, true) }, bcSpout.dao.Store, true)
}) })
t.Run("invalid state jump stage format", func(t *testing.T) { 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) checkNewBlockchainErr(t, boltCfg, bcSpout.dao.Store, true)
}) })
t.Run("missing state sync point", func(t *testing.T) { 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) checkNewBlockchainErr(t, boltCfg, bcSpout.dao.Store, true)
}) })
t.Run("invalid state sync point", func(t *testing.T) { 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) point := make([]byte, 4)
binary.LittleEndian.PutUint32(point, uint32(len(bcSpout.headerHashes))) 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) checkNewBlockchainErr(t, boltCfg, bcSpout.dao.Store, true)
}) })
for _, stage := range []stateJumpStage{stateJumpStarted, newStorageItemsAdded, genesisStateRemoved, 0x03} { for _, stage := range []stateJumpStage{stateJumpStarted, newStorageItemsAdded, genesisStateRemoved, 0x03} {
t.Run(fmt.Sprintf("state jump stage %d", stage), func(t *testing.T) { 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) point := make([]byte, 4)
binary.LittleEndian.PutUint32(point, uint32(stateSyncPoint)) 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 shouldFail := stage == 0x03 // unknown stage
checkNewBlockchainErr(t, spountCfg, bcSpout.dao.Store, shouldFail) checkNewBlockchainErr(t, spountCfg, bcSpout.dao.Store, shouldFail)
}) })

View file

@ -5,7 +5,6 @@ import (
"context" "context"
"encoding/binary" "encoding/binary"
"errors" "errors"
"fmt"
iocore "io" iocore "io"
"sort" "sort"
@ -31,8 +30,8 @@ var (
// DAO is a data access object. // DAO is a data access object.
type DAO interface { type DAO interface {
DeleteBlock(h util.Uint256, buf *io.BufBinWriter) error DeleteBlock(h util.Uint256, buf *io.BufBinWriter) error
DeleteContractID(id int32) error DeleteContractID(id int32)
DeleteStorageItem(id int32, key []byte) error DeleteStorageItem(id int32, key []byte)
GetAndDecode(entity io.Serializable, key []byte) error GetAndDecode(entity io.Serializable, key []byte) error
GetAppExecResults(hash util.Uint256, trig trigger.Type) ([]state.AppExecResult, error) GetAppExecResults(hash util.Uint256, trig trigger.Type) ([]state.AppExecResult, error)
GetBatch() *storage.MemBatch GetBatch() *storage.MemBatch
@ -54,18 +53,18 @@ type DAO interface {
HasTransaction(hash util.Uint256) error HasTransaction(hash util.Uint256) error
Persist() (int, error) Persist() (int, error)
PersistSync() (int, error) PersistSync() (int, error)
PutContractID(id int32, hash util.Uint160) error PutContractID(id int32, hash util.Uint160)
PutCurrentHeader(hashAndIndex []byte) error PutCurrentHeader(hashAndIndex []byte)
PutTokenTransferInfo(acc util.Uint160, bs *state.TokenTransferInfo) error PutTokenTransferInfo(acc util.Uint160, bs *state.TokenTransferInfo) error
PutTokenTransferLog(acc util.Uint160, start uint64, index uint32, isNEP11 bool, lg *state.TokenTransferLog) error PutTokenTransferLog(acc util.Uint160, start uint64, index uint32, isNEP11 bool, lg *state.TokenTransferLog)
PutStateSyncPoint(p uint32) error PutStateSyncPoint(p uint32)
PutStateSyncCurrentBlockHeight(h uint32) error PutStateSyncCurrentBlockHeight(h uint32)
PutStorageItem(id int32, key []byte, si state.StorageItem) error PutStorageItem(id int32, key []byte, si state.StorageItem)
PutVersion(v Version) error PutVersion(v Version)
Seek(id int32, rng storage.SeekRange, f func(k, v []byte) bool) Seek(id int32, rng storage.SeekRange, f func(k, v []byte) bool)
SeekAsync(ctx context.Context, id int32, rng storage.SeekRange) chan storage.KeyValue 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 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 StoreAsTransaction(tx *transaction.Transaction, index uint32, aer *state.AppExecResult, buf *io.BufBinWriter) error
putTokenTransferInfo(acc util.Uint160, bs *state.TokenTransferInfo, 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 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. // 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 { func (dao *Simple) putWithBuffer(entity io.Serializable, key []byte, buf *io.BufBinWriter) error {
entity.EncodeBinary(buf.BinWriter) entity.EncodeBinary(buf.BinWriter)
if buf.Err != nil { if buf.Err != nil {
return buf.Err return buf.Err
} }
return dao.Store.Put(key, buf.Bytes()) dao.Store.Put(key, buf.Bytes())
return nil
} }
func makeContractIDKey(id int32) []byte { func makeContractIDKey(id int32) []byte {
@ -135,13 +130,13 @@ func makeContractIDKey(id int32) []byte {
} }
// DeleteContractID deletes contract's id to hash mapping. // DeleteContractID deletes contract's id to hash mapping.
func (dao *Simple) DeleteContractID(id int32) error { func (dao *Simple) DeleteContractID(id int32) {
return dao.Store.Delete(makeContractIDKey(id)) dao.Store.Delete(makeContractIDKey(id))
} }
// PutContractID adds a mapping from contract's ID to its hash. // PutContractID adds a mapping from contract's ID to its hash.
func (dao *Simple) PutContractID(id int32, hash util.Uint160) error { func (dao *Simple) PutContractID(id int32, hash util.Uint160) {
return dao.Store.Put(makeContractIDKey(id), hash.BytesBE()) dao.Store.Put(makeContractIDKey(id), hash.BytesBE())
} }
// GetContractScriptHash retrieves contract's hash given its ID. // 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. // 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) key := getTokenTransferLogKey(acc, start, index, isNEP11)
return dao.Store.Put(key, lg.Raw) dao.Store.Put(key, lg.Raw)
} }
// -- end transfer log. // -- 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 // PutStorageItem puts given StorageItem for given id with given
// key into the given store. // 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) 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 // DeleteStorageItem drops storage item for the given id with the
// given key from the store. // 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) 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. // 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. // 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 dao.Version = v
return dao.Store.Put(storage.SYSVersion.Bytes(), v.Bytes()) dao.Store.Put(storage.SYSVersion.Bytes(), v.Bytes())
} }
// PutCurrentHeader stores current header. // PutCurrentHeader stores current header.
func (dao *Simple) PutCurrentHeader(hashAndIndex []byte) error { func (dao *Simple) PutCurrentHeader(hashAndIndex []byte) {
return dao.Store.Put(storage.SYSCurrentHeader.Bytes(), hashAndIndex) dao.Store.Put(storage.SYSCurrentHeader.Bytes(), hashAndIndex)
} }
// PutStateSyncPoint stores current state synchronisation point P. // PutStateSyncPoint stores current state synchronisation point P.
func (dao *Simple) PutStateSyncPoint(p uint32) error { func (dao *Simple) PutStateSyncPoint(p uint32) {
buf := make([]byte, 4) buf := make([]byte, 4)
binary.LittleEndian.PutUint32(buf, p) 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. // 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) buf := make([]byte, 4)
binary.LittleEndian.PutUint32(buf, h) 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 // 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 { if buf.Err != nil {
return buf.Err 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 // 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 { if w.Err != nil {
return w.Err return w.Err
} }
_ = dao.Store.Put(key, w.Bytes()) dao.Store.Put(key, w.Bytes())
for _, tx := range b.Transactions { for _, tx := range b.Transactions {
copy(key[1:], tx.Hash().BytesBE()) copy(key[1:], tx.Hash().BytesBE())
_ = dao.Store.Delete(key) dao.Store.Delete(key)
if dao.Version.P2PSigExtensions { if dao.Version.P2PSigExtensions {
for _, attr := range tx.GetAttributes(transaction.ConflictsT) { for _, attr := range tx.GetAttributes(transaction.ConflictsT) {
hash := attr.Value.(*transaction.Conflicts).Hash hash := attr.Value.(*transaction.Conflicts).Hash
copy(key[1:], hash.BytesBE()) 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 // StoreAsCurrentBlock stores a hash of the given block with prefix
// SYSCurrentBlock. It can reuse given buffer for the purpose of value // SYSCurrentBlock. It can reuse given buffer for the purpose of value
// serialization. // 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 { if buf == nil {
buf = io.NewBufBinWriter() buf = io.NewBufBinWriter()
} }
h := block.Hash() h := block.Hash()
h.EncodeBinary(buf.BinWriter) h.EncodeBinary(buf.BinWriter)
buf.WriteU32LE(block.Index) 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 // 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 { if buf.Err != nil {
return buf.Err return buf.Err
} }
err := dao.Store.Put(key, buf.Bytes()) dao.Store.Put(key, buf.Bytes())
if err != nil {
return err
}
if dao.Version.P2PSigExtensions { if dao.Version.P2PSigExtensions {
var value []byte var value []byte
for _, attr := range tx.GetAttributes(transaction.ConflictsT) { 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) buf.BinWriter.WriteB(transaction.DummyVersion)
value = buf.Bytes() value = buf.Bytes()
} }
err = dao.Store.Put(key, value) 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)
}
} }
} }
return nil return nil

View file

@ -22,11 +22,10 @@ func TestPutGetAndDecode(t *testing.T) {
dao := NewSimple(storage.NewMemoryStore(), false, false) dao := NewSimple(storage.NewMemoryStore(), false, false)
serializable := &TestSerializable{field: random.String(4)} serializable := &TestSerializable{field: random.String(4)}
hash := []byte{1} hash := []byte{1}
err := dao.Put(serializable, hash) require.NoError(t, dao.putWithBuffer(serializable, hash, io.NewBufBinWriter()))
require.NoError(t, err)
gotAndDecoded := &TestSerializable{} gotAndDecoded := &TestSerializable{}
err = dao.GetAndDecode(gotAndDecoded, hash) err := dao.GetAndDecode(gotAndDecoded, hash)
require.NoError(t, err) require.NoError(t, err)
} }
@ -48,8 +47,7 @@ func TestPutGetStorageItem(t *testing.T) {
id := int32(random.Int(0, 1024)) id := int32(random.Int(0, 1024))
key := []byte{0} key := []byte{0}
storageItem := state.StorageItem{} storageItem := state.StorageItem{}
err := dao.PutStorageItem(id, key, storageItem) dao.PutStorageItem(id, key, storageItem)
require.NoError(t, err)
gotStorageItem := dao.GetStorageItem(id, key) gotStorageItem := dao.GetStorageItem(id, key)
require.Equal(t, storageItem, gotStorageItem) require.Equal(t, storageItem, gotStorageItem)
} }
@ -59,10 +57,8 @@ func TestDeleteStorageItem(t *testing.T) {
id := int32(random.Int(0, 1024)) id := int32(random.Int(0, 1024))
key := []byte{0} key := []byte{0}
storageItem := state.StorageItem{} storageItem := state.StorageItem{}
err := dao.PutStorageItem(id, key, storageItem) dao.PutStorageItem(id, key, storageItem)
require.NoError(t, err) dao.DeleteStorageItem(id, key)
err = dao.DeleteStorageItem(id, key)
require.NoError(t, err)
gotStorageItem := dao.GetStorageItem(id, key) gotStorageItem := dao.GetStorageItem(id, key)
require.Nil(t, gotStorageItem) require.Nil(t, gotStorageItem)
} }
@ -129,22 +125,21 @@ func TestGetVersion(t *testing.T) {
StateRootInHeader: true, StateRootInHeader: true,
Value: "testVersion", Value: "testVersion",
} }
err := dao.PutVersion(expected) dao.PutVersion(expected)
require.NoError(t, err)
actual, err := dao.GetVersion() actual, err := dao.GetVersion()
require.NoError(t, err) require.NoError(t, err)
require.Equal(t, expected, actual) require.Equal(t, expected, actual)
t.Run("invalid", func(t *testing.T) { t.Run("invalid", func(t *testing.T) {
dao := NewSimple(storage.NewMemoryStore(), false, false) 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() _, err := dao.GetVersion()
require.Error(t, err) require.Error(t, err)
}) })
t.Run("old format", func(t *testing.T) { t.Run("old format", func(t *testing.T) {
dao := NewSimple(storage.NewMemoryStore(), false, false) 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() version, err := dao.GetVersion()
require.NoError(t, err) require.NoError(t, err)
@ -169,8 +164,7 @@ func TestGetCurrentHeaderHeight_Store(t *testing.T) {
}, },
}, },
} }
err := dao.StoreAsCurrentBlock(b, nil) dao.StoreAsCurrentBlock(b, nil)
require.NoError(t, err)
height, err := dao.GetCurrentBlockHeight() height, err := dao.GetCurrentBlockHeight()
require.NoError(t, err) require.NoError(t, err)
require.Equal(t, uint32(0), height) require.Equal(t, uint32(0), height)
@ -305,7 +299,7 @@ func TestPutGetStateSyncPoint(t *testing.T) {
// non-empty store // non-empty store
var expected uint32 = 5 var expected uint32 = 5
require.NoError(t, dao.PutStateSyncPoint(expected)) dao.PutStateSyncPoint(expected)
actual, err := dao.GetStateSyncPoint() actual, err := dao.GetStateSyncPoint()
require.NoError(t, err) require.NoError(t, err)
require.Equal(t, expected, actual) require.Equal(t, expected, actual)
@ -320,7 +314,7 @@ func TestPutGetStateSyncCurrentBlockHeight(t *testing.T) {
// non-empty store // non-empty store
var expected uint32 = 5 var expected uint32 = 5
require.NoError(t, dao.PutStateSyncCurrentBlockHeight(expected)) dao.PutStateSyncCurrentBlockHeight(expected)
actual, err := dao.GetStateSyncCurrentBlockHeight() actual, err := dao.GetStateSyncCurrentBlockHeight()
require.NoError(t, err) require.NoError(t, err)
require.Equal(t, expected, actual) require.Equal(t, expected, actual)

View file

@ -59,7 +59,8 @@ func storageDelete(ic *interop.Context) error {
return errors.New("StorageContext is read only") return errors.New("StorageContext is read only")
} }
key := ic.VM.Estack().Pop().Bytes() 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. // 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()) { if !ic.VM.AddGas(int64(sizeInc) * ic.Chain.GetStoragePrice()) {
return errGasLimitExceeded 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. // storagePut puts key-value pair into the storage.

View file

@ -362,8 +362,8 @@ func BenchmarkStorageFind(b *testing.B) {
items["abc"+random.String(10)] = random.Bytes(10) items["abc"+random.String(10)] = random.Bytes(10)
} }
for k, v := range items { for k, v := range items {
require.NoError(b, context.DAO.PutStorageItem(contractState.ID, []byte(k), v)) 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+1, []byte(k), v)
} }
changes, err := context.DAO.Persist() changes, err := context.DAO.Persist()
require.NoError(b, err) require.NoError(b, err)
@ -406,8 +406,8 @@ func BenchmarkStorageFindIteratorNext(b *testing.B) {
items["abc"+random.String(10)] = random.Bytes(10) items["abc"+random.String(10)] = random.Bytes(10)
} }
for k, v := range items { for k, v := range items {
require.NoError(b, context.DAO.PutStorageItem(contractState.ID, []byte(k), v)) 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+1, []byte(k), v)
} }
changes, err := context.DAO.Persist() changes, err := context.DAO.Persist()
require.NoError(b, err) require.NoError(b, err)
@ -487,8 +487,7 @@ func TestStorageFind(t *testing.T) {
id := contractState.ID id := contractState.ID
for i := range skeys { for i := range skeys {
err := context.DAO.PutStorageItem(id, skeys[i], items[i]) context.DAO.PutStorageItem(id, skeys[i], items[i])
require.NoError(t, err)
} }
testFind := func(t *testing.T, prefix []byte, opts int64, expected []stackitem.Item) { testFind := func(t *testing.T, prefix []byte, opts int64, expected []stackitem.Item) {

View file

@ -275,8 +275,8 @@ func TestTrie_PutBatchHash(t *testing.T) {
tr1.Collapse(1) tr1.Collapse(1)
tr2.Collapse(1) tr2.Collapse(1)
key := makeStorageKey(tr1.root.(*BranchNode).Children[2].Hash()) key := makeStorageKey(tr1.root.(*BranchNode).Children[2].Hash())
require.NoError(t, tr1.Store.Delete(key)) tr1.Store.Delete(key)
require.NoError(t, tr2.Store.Delete(key)) tr2.Store.Delete(key)
testIncompletePut(t, ps, 1, tr1, tr2) testIncompletePut(t, ps, 1, tr1, tr2)
}) })
} }

View file

@ -70,7 +70,7 @@ func (b *Billet) RestoreHashNode(path []byte, node Node) error {
panic("invalid storage prefix") panic("invalid storage prefix")
} }
k := append([]byte{byte(b.TempStoragePrefix)}, fromNibbles(path)...) k := append([]byte{byte(b.TempStoragePrefix)}, fromNibbles(path)...)
_ = b.Store.Put(k, leaf.value) b.Store.Put(k, leaf.value)
} }
return nil return nil
} }
@ -194,9 +194,9 @@ func (b *Billet) incrementRefAndStore(h util.Uint256, bs []byte) {
data = append(bs, 1, 0, 0, 0, 0) data = append(bs, 1, 0, 0, 0, 0)
} }
binary.LittleEndian.PutUint32(data[len(data)-4:], uint32(cnt)) binary.LittleEndian.PutUint32(data[len(data)-4:], uint32(cnt))
_ = b.Store.Put(key, data) b.Store.Put(key, data)
} else { } else {
_ = b.Store.Put(key, bs) b.Store.Put(key, bs)
} }
} }

View file

@ -69,8 +69,7 @@ func VerifyProof(rh util.Uint256, key []byte, proofs [][]byte) ([]byte, bool) {
tr := NewTrie(NewHashNode(rh), ModeAll, storage.NewMemCachedStore(storage.NewMemoryStore())) tr := NewTrie(NewHashNode(rh), ModeAll, storage.NewMemCachedStore(storage.NewMemoryStore()))
for i := range proofs { for i := range proofs {
h := hash.DoubleSha256(proofs[i]) 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) _, leaf, _, err := tr.getWithPath(tr.root, path, true)
if err != nil { if err != nil {

View file

@ -420,7 +420,7 @@ func (t *Trie) Flush(index uint32) {
delete(t.refcount, h) delete(t.refcount, h)
} }
} else if node.refcount > 0 { } else if node.refcount > 0 {
_ = t.Store.Put(key, node.bytes) t.Store.Put(key, node.bytes)
} }
node.refcount = 0 node.refcount = 0
} else { } 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])) panic(fmt.Sprintf("negative reference count: %s new %d, upd %d", h.StringBE(), cnt, t.refcount[h]))
case cnt == 0: case cnt == 0:
if !t.mode.GC() { if !t.mode.GC() {
_ = t.Store.Delete(key) t.Store.Delete(key)
} else { } else {
data[len(data)-5] = 0 data[len(data)-5] = 0
binary.LittleEndian.PutUint32(data[len(data)-4:], index) binary.LittleEndian.PutUint32(data[len(data)-4:], index)
_ = t.Store.Put(key, data) t.Store.Put(key, data)
} }
default: default:
binary.LittleEndian.PutUint32(data[len(data)-4:], uint32(cnt)) binary.LittleEndian.PutUint32(data[len(data)-4:], uint32(cnt))
_ = t.Store.Put(key, data) t.Store.Put(key, data)
} }
return cnt return cnt
} }

View file

@ -251,7 +251,7 @@ func (tr *Trie) putToStore(n Node) {
} }
tr.updateRefCount(n.Hash(), makeStorageKey(n.Hash()), 0) tr.updateRefCount(n.Hash(), makeStorageKey(n.Hash()), 0)
} else { } else {
_ = tr.Store.Put(makeStorageKey(n.Hash()), n.Bytes()) tr.Store.Put(makeStorageKey(n.Hash()), n.Bytes())
} }
} }

View file

@ -386,23 +386,14 @@ func (m *Management) Destroy(d dao.DAO, hash util.Uint160) error {
return err return err
} }
key := MakeContractKey(hash) key := MakeContractKey(hash)
err = d.DeleteStorageItem(m.ID, key) d.DeleteStorageItem(m.ID, key)
if err != nil { d.DeleteContractID(contract.ID)
return err
}
err = d.DeleteContractID(contract.ID)
if err != nil {
return err
}
siArr, err := d.GetStorageItems(contract.ID) siArr, err := d.GetStorageItems(contract.ID)
if err != nil { if err != nil {
return err return err
} }
for _, kv := range siArr { for _, kv := range siArr {
err := d.DeleteStorageItem(contract.ID, []byte(kv.Key)) d.DeleteStorageItem(contract.ID, []byte(kv.Key))
if err != nil {
return err
}
} }
m.markUpdated(hash) m.markUpdated(hash)
return nil return nil
@ -425,10 +416,7 @@ func (m *Management) setMinimumDeploymentFee(ic *interop.Context, args []stackit
if !m.NEO.checkCommittee(ic) { if !m.NEO.checkCommittee(ic) {
panic("invalid committee signature") panic("invalid committee signature")
} }
err := ic.DAO.PutStorageItem(m.ID, keyMinimumDeploymentFee, bigint.ToBytes(value)) ic.DAO.PutStorageItem(m.ID, keyMinimumDeploymentFee, bigint.ToBytes(value))
if err != nil {
panic(err)
}
return stackitem.Null{} return stackitem.Null{}
} }
@ -564,10 +552,9 @@ func (m *Management) GetNEP17Contracts() []util.Uint160 {
// Initialize implements Contract interface. // Initialize implements Contract interface.
func (m *Management) Initialize(ic *interop.Context) error { func (m *Management) Initialize(ic *interop.Context) error {
if err := setIntWithKey(m.ID, ic.DAO, keyMinimumDeploymentFee, defaultMinimumDeploymentFee); err != nil { setIntWithKey(m.ID, ic.DAO, keyMinimumDeploymentFee, defaultMinimumDeploymentFee)
return err setIntWithKey(m.ID, ic.DAO, keyNextAvailableID, 1)
} return nil
return setIntWithKey(m.ID, ic.DAO, keyNextAvailableID, 1)
} }
// PutContractState saves given contract state into given DAO. // 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. if cs.UpdateCounter != 0 { // Update.
return nil 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) { 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()) ret := int32(id.Int64())
id.Add(id, intOne) id.Add(id, intOne)
si = bigint.ToPreallocatedBytes(id, si) 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) { func (m *Management) emitNotification(ic *interop.Context, name string, hash util.Uint160) {

View file

@ -79,7 +79,7 @@ func TestManagement_Initialize(t *testing.T) {
t.Run("invalid contract state", func(t *testing.T) { t.Run("invalid contract state", func(t *testing.T) {
d := dao.NewSimple(storage.NewMemoryStore(), false, false) d := dao.NewSimple(storage.NewMemoryStore(), false, false)
mgmt := newManagement() 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)) require.Error(t, mgmt.InitializeCache(d))
}) })
} }

View file

@ -215,10 +215,7 @@ func (n *NEO) Initialize(ic *interop.Context) error {
return err return err
} }
err = ic.DAO.PutStorageItem(n.ID, prefixCommittee, cvs.Bytes()) ic.DAO.PutStorageItem(n.ID, prefixCommittee, cvs.Bytes())
if err != nil {
return err
}
h, err := getStandbyValidatorsHash(ic) h, err := getStandbyValidatorsHash(ic)
if err != nil { if err != nil {
@ -228,22 +225,14 @@ func (n *NEO) Initialize(ic *interop.Context) error {
var index uint32 var index uint32
value := big.NewInt(5 * GASFactor) value := big.NewInt(5 * GASFactor)
err = n.putGASRecord(ic.DAO, index, value) n.putGASRecord(ic.DAO, index, value)
if err != nil {
return err
}
gr := &gasRecord{{Index: index, GASPerBlock: *value}} gr := &gasRecord{{Index: index, GASPerBlock: *value}}
n.gasPerBlock.Store(*gr) n.gasPerBlock.Store(*gr)
n.gasPerBlockChanged.Store(false) n.gasPerBlockChanged.Store(false)
err = ic.DAO.PutStorageItem(n.ID, []byte{prefixVotersCount}, state.StorageItem{}) ic.DAO.PutStorageItem(n.ID, []byte{prefixVotersCount}, state.StorageItem{})
if err != nil {
return err
}
err = setIntWithKey(n.ID, ic.DAO, []byte{prefixRegisterPrice}, DefaultRegisterPrice) setIntWithKey(n.ID, ic.DAO, []byte{prefixRegisterPrice}, DefaultRegisterPrice)
if err != nil {
return err
}
n.registerPrice.Store(int64(DefaultRegisterPrice)) n.registerPrice.Store(int64(DefaultRegisterPrice))
n.registerPriceChanged.Store(false) n.registerPriceChanged.Store(false)
return nil return nil
@ -305,7 +294,8 @@ func (n *NEO) updateCommittee(ic *interop.Context) error {
if !votesChanged { if !votesChanged {
// We need to put in storage anyway, as it affects dumps // We need to put in storage anyway, as it affects dumps
committee := n.committee.Load().(keysWithVotes) 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) _, cvs, err := n.computeCommitteeMembers(ic.Chain, ic.DAO)
@ -316,7 +306,8 @@ func (n *NEO) updateCommittee(ic *interop.Context) error {
return err return err
} }
n.votesChanged.Store(false) 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. // 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) binary.BigEndian.PutUint32(key[34:], ic.Block.Index+1)
n.gasPerVoteCache[cs[i].Key] = *tmp n.gasPerVoteCache[cs[i].Key] = *tmp
if err := ic.DAO.PutStorageItem(n.ID, key, bigint.ToBytes(tmp)); err != nil { ic.DAO.PutStorageItem(n.ID, key, bigint.ToBytes(tmp))
return err
}
} }
} }
} }
@ -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 // 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. // further acc's queries from `onNEP17Payment` if so, see https://github.com/nspcc-dev/neo-go/pull/2181.
key := makeAccountKey(h) key := makeAccountKey(h)
err = ic.DAO.PutStorageItem(n.ID, key, acc.Bytes()) ic.DAO.PutStorageItem(n.ID, key, acc.Bytes())
if err != nil {
return fmt.Errorf("failed to store acc before gas distribution: %w", err)
}
n.GAS.mint(ic, h, gen, true) n.GAS.mint(ic, h, gen, true)
return nil 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") return errors.New("invalid committee signature")
} }
n.gasPerBlockChanged.Store(true) 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 { 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") panic("invalid committee signature")
} }
err := setIntWithKey(n.ID, ic.DAO, []byte{prefixRegisterPrice}, price.Int64()) setIntWithKey(n.ID, ic.DAO, []byte{prefixRegisterPrice}, price.Int64())
if err != nil {
panic(err)
}
n.registerPriceChanged.Store(true) n.registerPriceChanged.Store(true)
return stackitem.Null{} 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 { if c.Registered || c.Votes.Sign() != 0 {
return false, nil return false, nil
} }
if err := d.DeleteStorageItem(n.ID, makeValidatorKey(pub)); err != nil { d.DeleteStorageItem(n.ID, makeValidatorKey(pub))
return true, err
}
var toRemove []string var toRemove []string
voterKey := makeVoterKey(pub.Bytes()) voterKey := makeVoterKey(pub.Bytes())
@ -627,9 +609,7 @@ func (n *NEO) dropCandidateIfZero(d dao.DAO, pub *keys.PublicKey, c *candidate)
return true return true
}) })
for i := range toRemove { for i := range toRemove {
if err := d.DeleteStorageItem(n.ID, []byte(toRemove[i])); err != nil { d.DeleteStorageItem(n.ID, []byte(toRemove[i]))
return true, err
}
} }
delete(n.gasPerVoteCache, string(voterKey)) delete(n.gasPerVoteCache, string(voterKey))
@ -804,10 +784,7 @@ func (n *NEO) VoteInternal(ic *interop.Context, h util.Uint160, pub *keys.Public
return err return err
} }
// we should put it in storage anyway as it affects dumps // we should put it in storage anyway as it affects dumps
err = ic.DAO.PutStorageItem(n.ID, key, si) ic.DAO.PutStorageItem(n.ID, key, si)
if err != nil {
return err
}
if pub != nil { if pub != nil {
valKey := makeValidatorKey(pub) valKey := makeValidatorKey(pub)
valSi := ic.DAO.GetStorageItem(n.ID, valKey) 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) cd := new(candidate).FromBytes(valSi)
// we should put it in storage anyway as it affects dumps // we should put it in storage anyway as it affects dumps
err = ic.DAO.PutStorageItem(n.ID, valKey, valSi) ic.DAO.PutStorageItem(n.ID, valKey, valSi)
if err != nil {
return err
}
if !cd.Registered { if !cd.Registered {
return errors.New("validator must be 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 { if err := n.ModifyAccountVotes(acc, ic.DAO, &acc.Balance, true); err != nil {
return err 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). // 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 := bigint.FromBytes(si)
votersCount.Add(votersCount, amount) votersCount.Add(votersCount, amount)
si = bigint.ToPreallocatedBytes(votersCount, si) 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. // 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. // 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 := make([]byte, 5)
key[0] = prefixGASPerBlock key[0] = prefixGASPerBlock
binary.BigEndian.PutUint32(key[1:], index) binary.BigEndian.PutUint32(key[1:], index)
return dao.PutStorageItem(n.ID, key, bigint.ToBytes(value)) dao.PutStorageItem(n.ID, key, bigint.ToBytes(value))
} }

View file

@ -107,9 +107,9 @@ func (c *nep17TokenNative) getTotalSupply(d dao.DAO) (state.StorageItem, *big.In
return si, bigint.FromBytes(si) 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)) 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 { 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) err := c.incBalance(ic, acc, &si, amount, requiredBalance)
if err != nil { if err != nil {
if si != nil && amount.Sign() <= 0 { if si != nil && amount.Sign() <= 0 {
_ = ic.DAO.PutStorageItem(c.ID, key, si) ic.DAO.PutStorageItem(c.ID, key, si)
} }
return err return err
} }
if si == nil { if si == nil {
err = ic.DAO.DeleteStorageItem(c.ID, key) ic.DAO.DeleteStorageItem(c.ID, key)
} else { } 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. // 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 { if err := c.incBalance(ic, h, &si, amount, nil); err != nil {
panic(err) panic(err)
} }
var err error
if si == nil { if si == nil {
err = ic.DAO.DeleteStorageItem(c.ID, key) ic.DAO.DeleteStorageItem(c.ID, key)
} else { } else {
err = ic.DAO.PutStorageItem(c.ID, key, si) ic.DAO.PutStorageItem(c.ID, key, si)
}
if err != nil {
panic(err)
} }
buf, supply := c.getTotalSupply(ic.DAO) buf, supply := c.getTotalSupply(ic.DAO)
supply.Add(supply, amount) supply.Add(supply, amount)
err = c.saveTotalSupply(ic.DAO, buf, supply) c.saveTotalSupply(ic.DAO, buf, supply)
if err != nil {
panic(err)
}
} }
func newDescriptor(name string, ret smartcontract.ParamType, ps ...manifest.Parameter) *manifest.Method { func newDescriptor(name string, ret smartcontract.ParamType, ps ...manifest.Parameter) *manifest.Method {

View file

@ -109,11 +109,7 @@ func (n *Notary) Metadata() *interop.ContractMD {
// Initialize initializes Notary native contract and implements Contract interface. // Initialize initializes Notary native contract and implements Contract interface.
func (n *Notary) Initialize(ic *interop.Context) error { func (n *Notary) Initialize(ic *interop.Context) error {
err := setIntWithKey(n.ID, ic.DAO, maxNotValidBeforeDeltaKey, defaultMaxNotValidBeforeDelta) setIntWithKey(n.ID, ic.DAO, maxNotValidBeforeDeltaKey, defaultMaxNotValidBeforeDelta)
if err != nil {
return err
}
n.isValid = true n.isValid = true
n.maxNotValidBeforeDelta = defaultMaxNotValidBeforeDelta n.maxNotValidBeforeDelta = defaultMaxNotValidBeforeDelta
return nil return nil
@ -141,10 +137,7 @@ func (n *Notary) OnPersist(ic *interop.Context) error {
balance := n.GetDepositFor(ic.DAO, payer.Account) balance := n.GetDepositFor(ic.DAO, payer.Account)
balance.Amount.Sub(balance.Amount, big.NewInt(tx.SystemFee+tx.NetworkFee)) balance.Amount.Sub(balance.Amount, big.NewInt(tx.SystemFee+tx.NetworkFee))
if balance.Amount.Sign() == 0 { if balance.Amount.Sign() == 0 {
err := n.removeDepositFor(ic.DAO, payer.Account) 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)
}
} else { } else {
err := n.putDepositFor(ic.DAO, balance, payer.Account) err := n.putDepositFor(ic.DAO, balance, payer.Account)
if err != nil { if err != nil {
@ -289,9 +282,7 @@ func (n *Notary) withdraw(ic *interop.Context, args []stackitem.Item) stackitem.
if !ic.VM.Estack().Pop().Bool() { if !ic.VM.Estack().Pop().Bool() {
panic("failed to transfer GAS from Notary account: `transfer` returned false") panic("failed to transfer GAS from Notary account: `transfer` returned false")
} }
if err := n.removeDepositFor(ic.DAO, from); err != nil { n.removeDepositFor(ic.DAO, from)
panic(fmt.Errorf("failed to remove withdrawn deposit for %s from the storage: %w", from.StringBE(), err))
}
return stackitem.NewBool(true) return stackitem.NewBool(true)
} }
@ -402,10 +393,7 @@ func (n *Notary) setMaxNotValidBeforeDelta(ic *interop.Context, args []stackitem
} }
n.lock.Lock() n.lock.Lock()
defer n.lock.Unlock() defer n.lock.Unlock()
err := setIntWithKey(n.ID, ic.DAO, maxNotValidBeforeDeltaKey, int64(value)) setIntWithKey(n.ID, ic.DAO, maxNotValidBeforeDeltaKey, int64(value))
if err != nil {
panic(fmt.Errorf("failed to put value into the storage: %w", err))
}
n.isValid = false n.isValid = false
return stackitem.Null{} 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. // 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()...) 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. // calculateNotaryReward calculates the reward for a single notary node based on FEE's count and Notary nodes count.

View file

@ -166,9 +166,7 @@ func (o *Oracle) PostPersist(ic *interop.Context) error {
if err := o.getConvertibleFromDAO(ic.DAO, reqKey, req); err != nil { if err := o.getConvertibleFromDAO(ic.DAO, reqKey, req); err != nil {
continue continue
} }
if err := ic.DAO.DeleteStorageItem(o.ID, reqKey); err != nil { ic.DAO.DeleteStorageItem(o.ID, reqKey)
return err
}
if orc != nil { if orc != nil {
removedIDs = append(removedIDs, resp.ID) removedIDs = append(removedIDs, resp.ID)
} }
@ -184,7 +182,7 @@ func (o *Oracle) PostPersist(ic *interop.Context) error {
var err error var err error
if len(*idList) == 0 { if len(*idList) == 0 {
err = ic.DAO.DeleteStorageItem(o.ID, idKey) ic.DAO.DeleteStorageItem(o.ID, idKey)
} else { } else {
err = putConvertibleToDAO(o.ID, ic.DAO, idKey, idList) err = putConvertibleToDAO(o.ID, ic.DAO, idKey, idList)
} }
@ -222,12 +220,8 @@ func (o *Oracle) Metadata() *interop.ContractMD {
// Initialize initializes Oracle contract. // Initialize initializes Oracle contract.
func (o *Oracle) Initialize(ic *interop.Context) error { func (o *Oracle) Initialize(ic *interop.Context) error {
if err := setIntWithKey(o.ID, ic.DAO, prefixRequestID, 0); err != nil { setIntWithKey(o.ID, ic.DAO, prefixRequestID, 0)
return err setIntWithKey(o.ID, ic.DAO, prefixRequestPrice, DefaultOracleRequestPrice)
}
if err := setIntWithKey(o.ID, ic.DAO, prefixRequestPrice, DefaultOracleRequestPrice); err != nil {
return err
}
o.requestPrice.Store(int64(DefaultOracleRequestPrice)) o.requestPrice.Store(int64(DefaultOracleRequestPrice))
o.requestPriceChanged.Store(false) o.requestPriceChanged.Store(false)
return nil return nil
@ -348,9 +342,7 @@ func (o *Oracle) RequestInternal(ic *interop.Context, url string, filter *string
id := itemID.Uint64() id := itemID.Uint64()
itemID.Add(itemID, intOne) itemID.Add(itemID, intOne)
si = bigint.ToPreallocatedBytes(itemID, si) si = bigint.ToPreallocatedBytes(itemID, si)
if err := ic.DAO.PutStorageItem(o.ID, prefixRequestID, si); err != nil { ic.DAO.PutStorageItem(o.ID, prefixRequestID, si)
return err
}
// Should be executed from contract. // Should be executed from contract.
_, err := ic.GetContract(ic.VM.GetCallingScriptHash()) _, 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) { if !o.NEO.checkCommittee(ic) {
panic("invalid committee signature") panic("invalid committee signature")
} }
if err := setIntWithKey(o.ID, ic.DAO, prefixRequestPrice, price.Int64()); err != nil { setIntWithKey(o.ID, ic.DAO, prefixRequestPrice, price.Int64())
panic(err)
}
o.requestPriceChanged.Store(true) o.requestPriceChanged.Store(true)
return stackitem.Null{} return stackitem.Null{}
} }

View file

@ -123,15 +123,9 @@ func (p *Policy) Metadata() *interop.ContractMD {
// Initialize initializes Policy native contract and implements Contract interface. // Initialize initializes Policy native contract and implements Contract interface.
func (p *Policy) Initialize(ic *interop.Context) error { func (p *Policy) Initialize(ic *interop.Context) error {
if err := setIntWithKey(p.ID, ic.DAO, feePerByteKey, defaultFeePerByte); err != nil { setIntWithKey(p.ID, ic.DAO, feePerByteKey, defaultFeePerByte)
return err setIntWithKey(p.ID, ic.DAO, execFeeFactorKey, defaultExecFeeFactor)
} setIntWithKey(p.ID, ic.DAO, storagePriceKey, DefaultStoragePrice)
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
}
p.isValid = true p.isValid = true
p.execFeeFactor = defaultExecFeeFactor p.execFeeFactor = defaultExecFeeFactor
@ -229,10 +223,7 @@ func (p *Policy) setExecFeeFactor(ic *interop.Context, args []stackitem.Item) st
} }
p.lock.Lock() p.lock.Lock()
defer p.lock.Unlock() defer p.lock.Unlock()
err := setIntWithKey(p.ID, ic.DAO, execFeeFactorKey, int64(value)) setIntWithKey(p.ID, ic.DAO, execFeeFactorKey, int64(value))
if err != nil {
panic(err)
}
p.isValid = false p.isValid = false
return stackitem.Null{} return stackitem.Null{}
} }
@ -285,10 +276,7 @@ func (p *Policy) setStoragePrice(ic *interop.Context, args []stackitem.Item) sta
} }
p.lock.Lock() p.lock.Lock()
defer p.lock.Unlock() defer p.lock.Unlock()
err := setIntWithKey(p.ID, ic.DAO, storagePriceKey, int64(value)) setIntWithKey(p.ID, ic.DAO, storagePriceKey, int64(value))
if err != nil {
panic(err)
}
p.isValid = false p.isValid = false
return stackitem.Null{} return stackitem.Null{}
} }
@ -304,10 +292,7 @@ func (p *Policy) setFeePerByte(ic *interop.Context, args []stackitem.Item) stack
} }
p.lock.Lock() p.lock.Lock()
defer p.lock.Unlock() defer p.lock.Unlock()
err := setIntWithKey(p.ID, ic.DAO, feePerByteKey, value) setIntWithKey(p.ID, ic.DAO, feePerByteKey, value)
if err != nil {
panic(err)
}
p.isValid = false p.isValid = false
return stackitem.Null{} return stackitem.Null{}
} }
@ -330,10 +315,7 @@ func (p *Policy) blockAccount(ic *interop.Context, args []stackitem.Item) stacki
key := append([]byte{blockedAccountPrefix}, hash.BytesBE()...) key := append([]byte{blockedAccountPrefix}, hash.BytesBE()...)
p.lock.Lock() p.lock.Lock()
defer p.lock.Unlock() defer p.lock.Unlock()
err := ic.DAO.PutStorageItem(p.ID, key, state.StorageItem{}) ic.DAO.PutStorageItem(p.ID, key, state.StorageItem{})
if err != nil {
panic(err)
}
p.isValid = false p.isValid = false
return stackitem.NewBool(true) 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()...) key := append([]byte{blockedAccountPrefix}, hash.BytesBE()...)
p.lock.Lock() p.lock.Lock()
defer p.lock.Unlock() defer p.lock.Unlock()
err := ic.DAO.DeleteStorageItem(p.ID, key) ic.DAO.DeleteStorageItem(p.ID, key)
if err != nil {
panic(err)
}
p.isValid = false p.isValid = false
return stackitem.NewBool(true) return stackitem.NewBool(true)
} }

View file

@ -28,11 +28,12 @@ func putConvertibleToDAO(id int32, d dao.DAO, key []byte, conv stackitem.Convert
if err != nil { if err != nil {
return err 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 { func setIntWithKey(id int32, dao dao.DAO, key []byte, value int64) {
return dao.PutStorageItem(id, key, bigint.ToBytes(big.NewInt(value))) dao.PutStorageItem(id, key, bigint.ToBytes(big.NewInt(value)))
} }
func getIntWithKey(id int32, dao dao.DAO, key []byte) int64 { func getIntWithKey(id int32, dao dao.DAO, key []byte) int64 {

View file

@ -149,7 +149,7 @@ func (s *Module) CleanStorage() error {
b := storage.NewMemCachedStore(s.Store) b := storage.NewMemCachedStore(s.Store)
s.Store.Seek(storage.SeekRange{Prefix: []byte{byte(storage.DataMPT)}}, func(k, _ []byte) bool { 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. // #1468, but don't need to copy here, because it is done by Store.
_ = b.Delete(k) b.Delete(k)
return true return true
}) })
_, err := b.Persist() _, err := b.Persist()
@ -160,22 +160,17 @@ func (s *Module) CleanStorage() error {
} }
// JumpToState performs jump to the state specified by given stateroot index. // JumpToState performs jump to the state specified by given stateroot index.
func (s *Module) JumpToState(sr *state.MPTRoot) error { func (s *Module) JumpToState(sr *state.MPTRoot) {
if err := s.addLocalStateRoot(s.Store, sr); err != nil { s.addLocalStateRoot(s.Store, sr)
return fmt.Errorf("failed to store local state root: %w", err)
}
data := make([]byte, 4) data := make([]byte, 4)
binary.LittleEndian.PutUint32(data, sr.Index) binary.LittleEndian.PutUint32(data, sr.Index)
if err := s.Store.Put([]byte{byte(storage.DataMPTAux), prefixValidated}, data); err != nil { s.Store.Put([]byte{byte(storage.DataMPTAux), prefixValidated}, data)
return fmt.Errorf("failed to store validated height: %w", err)
}
s.validatedHeight.Store(sr.Index) s.validatedHeight.Store(sr.Index)
s.currentLocal.Store(sr.Root) s.currentLocal.Store(sr.Root)
s.localHeight.Store(sr.Index) s.localHeight.Store(sr.Index)
s.mpt = mpt.NewTrie(mpt.NewHashNode(sr.Root), s.mode, s.Store) s.mpt = mpt.NewTrie(mpt.NewHashNode(sr.Root), s.mode, s.Store)
return nil
} }
// GC performs garbage collection. // GC performs garbage collection.
@ -225,11 +220,8 @@ func (s *Module) AddMPTBatch(index uint32, b mpt.Batch, cache *storage.MemCached
Index: index, Index: index,
Root: mpt.StateRoot(), Root: mpt.StateRoot(),
} }
err := s.addLocalStateRoot(cache, sr) s.addLocalStateRoot(cache, sr)
if err != nil { return &mpt, sr, nil
return nil, nil, err
}
return &mpt, sr, err
} }
// UpdateCurrentLocal updates local caches using provided state root. // UpdateCurrentLocal updates local caches using provided state root.

View file

@ -21,21 +21,19 @@ const (
prefixValidated = 0x03 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) key := makeStateRootKey(sr.Index)
if err := putStateRoot(store, key, sr); err != nil { putStateRoot(store, key, sr)
return err
}
data := make([]byte, 4) data := make([]byte, 4)
binary.LittleEndian.PutUint32(data, sr.Index) 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() w := io.NewBufBinWriter()
sr.EncodeBinary(w.BinWriter) sr.EncodeBinary(w.BinWriter)
return store.Put(key, w.Bytes()) store.Put(key, w.Bytes())
} }
func (s *Module) getStateRoot(key []byte) (*state.MPTRoot, error) { 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 { if len(local.Witness) != 0 {
return nil return nil
} }
if err := putStateRoot(s.Store, key, sr); err != nil { putStateRoot(s.Store, key, sr)
return err
}
data := make([]byte, 4) data := make([]byte, 4)
binary.LittleEndian.PutUint32(data, sr.Index) binary.LittleEndian.PutUint32(data, sr.Index)
if err := s.Store.Put([]byte{byte(storage.DataMPTAux), prefixValidated}, data); err != nil { s.Store.Put([]byte{byte(storage.DataMPTAux), prefixValidated}, data)
return err
}
s.validatedHeight.Store(sr.Index) s.validatedHeight.Store(sr.Index)
if !s.srInHead { if !s.srInHead {
updateStateHeightMetric(sr.Index) updateStateHeightMetric(sr.Index)

View file

@ -167,10 +167,7 @@ func (s *Module) Init(currChainHeight uint32) error {
} }
s.syncPoint = p s.syncPoint = p
err = s.dao.PutStateSyncPoint(p) s.dao.PutStateSyncPoint(p)
if err != nil {
return fmt.Errorf("failed to store state synchronisation point %d: %w", p, err)
}
s.syncStage = initialized s.syncStage = initialized
s.log.Info("try to sync state for the latest state synchronisation point", s.log.Info("try to sync state for the latest state synchronisation point",
zap.Uint32("point", p), zap.Uint32("point", p),
@ -339,10 +336,7 @@ func (s *Module) AddBlock(block *block.Block) error {
} }
writeBuf.Reset() writeBuf.Reset()
err := cache.PutStateSyncCurrentBlockHeight(block.Index) cache.PutStateSyncCurrentBlockHeight(block.Index)
if err != nil {
return fmt.Errorf("failed to store current block height: %w", err)
}
for _, tx := range block.Transactions { for _, tx := range block.Transactions {
if err := cache.StoreAsTransaction(tx, block.Index, nil, writeBuf); err != nil { 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() writeBuf.Reset()
} }
_, err = cache.Persist() _, err := cache.Persist()
if err != nil { if err != nil {
return fmt.Errorf("failed to persist results: %w", err) return fmt.Errorf("failed to persist results: %w", err)
} }

View file

@ -65,23 +65,21 @@ func (s *MemCachedStore) Get(key []byte) ([]byte, error) {
return s.ps.Get(key) return s.ps.Get(key)
} }
// Put puts new KV pair into the store. Never returns an error. // Put puts new KV pair into the store.
func (s *MemCachedStore) Put(key, value []byte) error { func (s *MemCachedStore) Put(key, value []byte) {
newKey := string(key) newKey := string(key)
vcopy := slice.Copy(value) vcopy := slice.Copy(value)
s.mut.Lock() s.mut.Lock()
put(s.chooseMap(key), newKey, vcopy) put(s.chooseMap(key), newKey, vcopy)
s.mut.Unlock() s.mut.Unlock()
return nil
} }
// Delete drops KV pair from the store. Never returns an error. // 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) newKey := string(key)
s.mut.Lock() s.mut.Lock()
put(s.chooseMap(key), newKey, nil) put(s.chooseMap(key), newKey, nil)
s.mut.Unlock() s.mut.Unlock()
return nil
} }
// GetBatch returns currently accumulated changeset. // GetBatch returns currently accumulated changeset.

View file

@ -18,26 +18,32 @@ func TestMemCachedPutGetDelete(t *testing.T) {
key := []byte("foo") key := []byte("foo")
value := []byte("bar") value := []byte("bar")
require.NoError(t, s.Put(key, value)) s.Put(key, value)
result, err := s.Get(key) result, err := s.Get(key)
assert.Nil(t, err) assert.Nil(t, err)
require.Equal(t, value, result) require.Equal(t, value, result)
err = s.Delete(key) s.Delete(key)
assert.Nil(t, err)
_, err = s.Get(key) _, err = s.Get(key)
assert.NotNil(t, err) assert.NotNil(t, err)
assert.Equal(t, err, ErrKeyNotFound) assert.Equal(t, err, ErrKeyNotFound)
// Double delete. // Double delete.
err = s.Delete(key) s.Delete(key)
assert.Nil(t, err)
_, err = s.Get(key)
assert.NotNil(t, err)
assert.Equal(t, err, ErrKeyNotFound)
// Nonexistent. // Nonexistent.
key = []byte("sparse") 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) { 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, nil, err)
assert.Equal(t, 0, c) assert.Equal(t, 0, c)
// persisting one key should result in one key in ps and nothing in ts // 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) checkBatch(t, ts, []KeyValueExists{{KeyValue: KeyValue{Key: []byte("key"), Value: []byte("value")}}}, nil)
c, err = ts.Persist() c, err = ts.Persist()
checkBatch(t, ts, nil, nil) checkBatch(t, ts, nil, nil)
@ -61,8 +67,8 @@ func testMemCachedStorePersist(t *testing.T, ps Store) {
assert.Equal(t, ErrKeyNotFound, err) assert.Equal(t, ErrKeyNotFound, err)
assert.Equal(t, []byte(nil), v) assert.Equal(t, []byte(nil), v)
// now we overwrite the previous `key` contents and also add `key2`, // now we overwrite the previous `key` contents and also add `key2`,
assert.NoError(t, ts.Put([]byte("key"), []byte("newvalue"))) ts.Put([]byte("key"), []byte("newvalue"))
assert.NoError(t, ts.Put([]byte("key2"), []byte("value2"))) ts.Put([]byte("key2"), []byte("value2"))
// this is to check that now key is written into the ps before we do // this is to check that now key is written into the ps before we do
// persist // persist
v, err = ps.Get([]byte("key2")) 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, nil, err)
assert.Equal(t, 0, c) assert.Equal(t, 0, c)
// test persisting deletions // test persisting deletions
err = ts.Delete([]byte("key")) ts.Delete([]byte("key"))
assert.Equal(t, nil, err)
checkBatch(t, ts, nil, []KeyValueExists{{KeyValue: KeyValue{Key: []byte("key")}, Exists: true}}) checkBatch(t, ts, nil, []KeyValueExists{{KeyValue: KeyValue{Key: []byte("key")}, Exists: true}})
c, err = ts.Persist() c, err = ts.Persist()
checkBatch(t, ts, nil, nil) checkBatch(t, ts, nil, nil)
@ -155,7 +160,7 @@ func TestCachedGetFromPersistent(t *testing.T) {
val, err := ts.Get(key) val, err := ts.Get(key)
assert.Nil(t, err) assert.Nil(t, err)
assert.Equal(t, value, val) assert.Equal(t, value, val)
assert.NoError(t, ts.Delete(key)) ts.Delete(key)
val, err = ts.Get(key) val, err = ts.Get(key)
assert.Equal(t, err, ErrKeyNotFound) assert.Equal(t, err, ErrKeyNotFound)
assert.Nil(t, val) assert.Nil(t, val)
@ -188,11 +193,11 @@ func TestCachedSeek(t *testing.T) {
} }
for _, v := range deletedKVs { for _, v := range deletedKVs {
require.NoError(t, ps.PutChangeSet(map[string][]byte{string(v.Key): v.Value}, nil)) 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 { for _, v := range updatedKVs {
require.NoError(t, ps.PutChangeSet(map[string][]byte{string(v.Key): v.Value}, nil)) 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) foundKVs := make(map[string][]byte)
ts.Seek(SeekRange{Prefix: goodPrefix}, func(k, v []byte) bool { 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++ { for i := 0; i < psElementsCount; i++ {
// lower KVs with matching prefix that should be found // 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 // 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 // deleted KVs with matching prefix that shouldn't be found
key := append(deletedPrefixGood, random.Bytes(10)...) key := append(deletedPrefixGood, random.Bytes(10)...)
require.NoError(t, ts.Put(key, []byte("deleted"))) ts.Put(key, []byte("deleted"))
if i < tsElementsCount { if i < tsElementsCount {
require.NoError(t, ts.Delete(key)) ts.Delete(key)
} }
// deleted KVs with non-matching prefix that shouldn't be found // deleted KVs with non-matching prefix that shouldn't be found
key = append(deletedPrefixBad, random.Bytes(10)...) key = append(deletedPrefixBad, random.Bytes(10)...)
require.NoError(t, ts.Put(key, []byte("deleted"))) ts.Put(key, []byte("deleted"))
if i < tsElementsCount { if i < tsElementsCount {
require.NoError(t, ts.Delete(key)) ts.Delete(key)
} }
// updated KVs with matching prefix that should be found // updated KVs with matching prefix that should be found
key = append(updatedPrefixGood, random.Bytes(10)...) key = append(updatedPrefixGood, random.Bytes(10)...)
require.NoError(t, ts.Put(key, []byte("stub"))) ts.Put(key, []byte("stub"))
if i < tsElementsCount { 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 // updated KVs with non-matching prefix that shouldn't be found
key = append(updatedPrefixBad, random.Bytes(10)...) key = append(updatedPrefixBad, random.Bytes(10)...)
require.NoError(t, ts.Put(key, []byte("stub"))) ts.Put(key, []byte("stub"))
if i < tsElementsCount { if i < tsElementsCount {
require.NoError(t, ts.Put(key, []byte("updated"))) ts.Put(key, []byte("updated"))
} }
} }
_, err := ts.PersistSync() _, err := ts.PersistSync()
@ -329,13 +334,13 @@ func TestMemCachedPersistFailing(t *testing.T) {
// cached Store // cached Store
ts := NewMemCachedStore(&bs) ts := NewMemCachedStore(&bs)
// Set a pair of keys. // Set a pair of keys.
require.NoError(t, ts.Put(t1, t1)) ts.Put(t1, t1)
require.NoError(t, ts.Put(t2, t2)) ts.Put(t2, t2)
// This will be called during Persist(). // This will be called during Persist().
bs.onPutBatch = func() { bs.onPutBatch = func() {
// Drop one, add one. // Drop one, add one.
require.NoError(t, ts.Put(b1, b1)) ts.Put(b1, b1)
require.NoError(t, ts.Delete(t1)) ts.Delete(t1)
} }
_, err := ts.Persist() _, err := ts.Persist()
require.Error(t, err) require.Error(t, err)
@ -381,11 +386,11 @@ func TestCachedSeekSorting(t *testing.T) {
} }
for _, v := range deletedKVs { for _, v := range deletedKVs {
require.NoError(t, ps.PutChangeSet(map[string][]byte{string(v.Key): v.Value}, nil)) 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 { for _, v := range updatedKVs {
require.NoError(t, ps.PutChangeSet(map[string][]byte{string(v.Key): v.Value}, nil)) 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 var foundKVs []KeyValue
ts.Seek(SeekRange{Prefix: goodPrefix}, func(k, v []byte) bool { ts.Seek(SeekRange{Prefix: goodPrefix}, func(k, v []byte) bool {

View file

@ -22,8 +22,8 @@ func BenchmarkMemorySeek(t *testing.B) {
) )
ts := NewMemCachedStore(ms) ts := NewMemCachedStore(ms)
for i := 0; i < count; i++ { for i := 0; i < count; i++ {
require.NoError(t, ts.Put(append(searchPrefix, random.Bytes(10)...), random.Bytes(10))) 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(badPrefix, random.Bytes(10)...), random.Bytes(10))
} }
_, err := ts.PersistSync() _, err := ts.PersistSync()
require.NoError(t, err) require.NoError(t, err)

View file

@ -39,7 +39,7 @@ func pushSeekDataSet(t *testing.T, s Store) []KeyValue {
} }
up := NewMemCachedStore(s) up := NewMemCachedStore(s)
for _, v := range kvs { for _, v := range kvs {
require.NoError(t, up.Put(v.Key, v.Value)) up.Put(v.Key, v.Value)
} }
_, err := up.PersistSync() _, err := up.PersistSync()
require.NoError(t, err) require.NoError(t, err)