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:
parent
be24bf6412
commit
9d2ef775cf
26 changed files with 228 additions and 380 deletions
|
@ -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.
|
||||
|
|
|
@ -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)
|
||||
})
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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)
|
||||
})
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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())
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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))
|
||||
})
|
||||
}
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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{}
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
Loading…
Reference in a new issue