mirror of
https://github.com/nspcc-dev/neo-go.git
synced 2024-11-26 19:42:23 +00:00
Merge pull request #2364 from nspcc-dev/botch
storage: speedups and simplifications
This commit is contained in:
commit
baa05bef43
38 changed files with 718 additions and 1078 deletions
|
@ -178,7 +178,7 @@ func TestAppCall(t *testing.T) {
|
|||
require.NoError(t, err)
|
||||
|
||||
ih := hash.Hash160(inner.Script)
|
||||
var contractGetter = func(_ dao.DAO, h util.Uint160) (*state.Contract, error) {
|
||||
var contractGetter = func(_ *dao.Simple, h util.Uint160) (*state.Contract, error) {
|
||||
if h.Equals(ih) {
|
||||
return &state.Contract{
|
||||
ContractBase: state.ContractBase{
|
||||
|
|
|
@ -20,6 +20,7 @@ import (
|
|||
"github.com/nspcc-dev/neo-go/pkg/core/interop"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/interop/contract"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/mempool"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/mpt"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/native"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/native/noderoles"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/state"
|
||||
|
@ -316,9 +317,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 +325,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)
|
||||
}
|
||||
|
@ -504,14 +500,10 @@ func (bc *Blockchain) jumpToStateInternal(p uint32, stage stateJumpStage) error
|
|||
|
||||
bc.log.Info("jumping to state sync point", zap.Uint32("state sync point", p))
|
||||
|
||||
writeBuf := io.NewBufBinWriter()
|
||||
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,55 +512,40 @@ 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:
|
||||
b := bc.dao.Store.Batch()
|
||||
cache := bc.dao.GetPrivate()
|
||||
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.
|
||||
b.Delete(k)
|
||||
cache.Store.Delete(k)
|
||||
return true
|
||||
})
|
||||
|
||||
err := bc.dao.Store.PutBatch(b)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to remove old storage items: %w", err)
|
||||
}
|
||||
|
||||
// After current state is updated, we need to remove outdated state-related data if so.
|
||||
// The only outdated data we might have is genesis-related data, so check it.
|
||||
if p-bc.config.MaxTraceableBlocks > 0 {
|
||||
cache := bc.dao.GetWrapped().(*dao.Simple)
|
||||
writeBuf.Reset()
|
||||
err := cache.DeleteBlock(bc.headerHashes[0], writeBuf)
|
||||
err := cache.DeleteBlock(bc.headerHashes[0])
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to remove outdated state data for the genesis block: %w", err)
|
||||
}
|
||||
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
|
||||
})
|
||||
}
|
||||
_, err = cache.Persist()
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to drop genesis block state: %w", err)
|
||||
}
|
||||
}
|
||||
err = bc.dao.Store.Put(jumpStageKey, []byte{byte(genesisStateRemoved)})
|
||||
cache.Store.Put(jumpStageKey, []byte{byte(genesisStateRemoved)})
|
||||
_, err := cache.Persist()
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to store state jump stage: %w", err)
|
||||
return fmt.Errorf("failed to persist old items removal: %w", err)
|
||||
}
|
||||
case genesisStateRemoved:
|
||||
// there's nothing to do after that, so just continue with common operations
|
||||
|
@ -581,11 +558,7 @@ func (bc *Blockchain) jumpToStateInternal(p uint32, stage stateJumpStage) error
|
|||
if err != nil {
|
||||
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)
|
||||
bc.topBlock.Store(block)
|
||||
atomic.StoreUint32(&bc.blockHeight, p)
|
||||
atomic.StoreUint32(&bc.persistedHeight, p)
|
||||
|
@ -594,12 +567,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 {
|
||||
|
@ -617,10 +588,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
|
||||
}
|
||||
|
||||
|
@ -933,7 +901,7 @@ func (bc *Blockchain) AddHeaders(headers ...*block.Header) error {
|
|||
func (bc *Blockchain) addHeaders(verify bool, headers ...*block.Header) error {
|
||||
var (
|
||||
start = time.Now()
|
||||
batch = bc.dao.Store.Batch()
|
||||
batch = bc.dao.GetPrivate()
|
||||
err error
|
||||
)
|
||||
|
||||
|
@ -982,7 +950,7 @@ func (bc *Blockchain) addHeaders(verify bool, headers ...*block.Header) error {
|
|||
}
|
||||
|
||||
key := storage.AppendPrefix(storage.DataExecutable, h.Hash().BytesBE())
|
||||
batch.Put(key, buf.Bytes())
|
||||
batch.Store.Put(key, buf.Bytes())
|
||||
buf.Reset()
|
||||
lastHeader = h
|
||||
}
|
||||
|
@ -995,13 +963,13 @@ func (bc *Blockchain) addHeaders(verify bool, headers ...*block.Header) error {
|
|||
}
|
||||
|
||||
key := storage.AppendPrefixInt(storage.IXHeaderHashList, int(bc.storedHeaderCount))
|
||||
batch.Put(key, buf.Bytes())
|
||||
batch.Store.Put(key, buf.Bytes())
|
||||
bc.storedHeaderCount += headerBatchCount
|
||||
}
|
||||
|
||||
batch.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 = bc.dao.Store.PutBatch(batch); err != nil {
|
||||
if _, err = batch.Persist(); err != nil {
|
||||
return err
|
||||
}
|
||||
bc.log.Debug("done processing headers",
|
||||
|
@ -1027,8 +995,8 @@ func (bc *Blockchain) GetStateSyncModule() *statesync.Module {
|
|||
// This is the only way to change Blockchain state.
|
||||
func (bc *Blockchain) storeBlock(block *block.Block, txpool *mempool.Pool) error {
|
||||
var (
|
||||
cache = bc.dao.GetWrapped()
|
||||
aerCache = bc.dao.GetWrapped()
|
||||
cache = bc.dao.GetPrivate()
|
||||
aerCache = bc.dao.GetPrivate()
|
||||
appExecResults = make([]*state.AppExecResult, 0, 2+len(block.Transactions))
|
||||
aerchan = make(chan *state.AppExecResult, len(block.Transactions)/8) // Tested 8 and 4 with no practical difference, but feel free to test more and tune.
|
||||
aerdone = make(chan error)
|
||||
|
@ -1036,17 +1004,12 @@ func (bc *Blockchain) storeBlock(block *block.Block, txpool *mempool.Pool) error
|
|||
go func() {
|
||||
var (
|
||||
kvcache = aerCache
|
||||
writeBuf = io.NewBufBinWriter()
|
||||
err error
|
||||
txCnt int
|
||||
baer1, baer2 *state.AppExecResult
|
||||
transCache = make(map[util.Uint160]transferData)
|
||||
)
|
||||
if err := kvcache.StoreAsCurrentBlock(block, writeBuf); err != nil {
|
||||
aerdone <- err
|
||||
return
|
||||
}
|
||||
writeBuf.Reset()
|
||||
kvcache.StoreAsCurrentBlock(block)
|
||||
if bc.config.RemoveUntraceableBlocks {
|
||||
var start, stop uint32
|
||||
if bc.config.P2PStateExchangeExtensions {
|
||||
|
@ -1064,13 +1027,12 @@ func (bc *Blockchain) storeBlock(block *block.Block, txpool *mempool.Pool) error
|
|||
stop = start + 1
|
||||
}
|
||||
for index := start; index < stop; index++ {
|
||||
err := kvcache.DeleteBlock(bc.headerHashes[index], writeBuf)
|
||||
err := kvcache.DeleteBlock(bc.headerHashes[index])
|
||||
if err != nil {
|
||||
bc.log.Warn("error while removing old block",
|
||||
zap.Uint32("index", index),
|
||||
zap.Error(err))
|
||||
}
|
||||
writeBuf.Reset()
|
||||
}
|
||||
}
|
||||
for aer := range aerchan {
|
||||
|
@ -1081,7 +1043,7 @@ func (bc *Blockchain) storeBlock(block *block.Block, txpool *mempool.Pool) error
|
|||
baer2 = aer
|
||||
}
|
||||
} else {
|
||||
err = kvcache.StoreAsTransaction(block.Transactions[txCnt], block.Index, aer, writeBuf)
|
||||
err = kvcache.StoreAsTransaction(block.Transactions[txCnt], block.Index, aer)
|
||||
txCnt++
|
||||
}
|
||||
if err != nil {
|
||||
|
@ -1093,17 +1055,15 @@ func (bc *Blockchain) storeBlock(block *block.Block, txpool *mempool.Pool) error
|
|||
bc.handleNotification(&aer.Execution.Events[j], kvcache, transCache, block, aer.Container)
|
||||
}
|
||||
}
|
||||
writeBuf.Reset()
|
||||
}
|
||||
if err != nil {
|
||||
aerdone <- err
|
||||
return
|
||||
}
|
||||
if err := kvcache.StoreAsBlock(block, baer1, baer2, writeBuf); err != nil {
|
||||
if err := kvcache.StoreAsBlock(block, baer1, baer2); err != nil {
|
||||
aerdone <- err
|
||||
return
|
||||
}
|
||||
writeBuf.Reset()
|
||||
for acc, trData := range transCache {
|
||||
err = kvcache.PutTokenTransferInfo(acc, &trData.Info)
|
||||
if err != nil {
|
||||
|
@ -1111,18 +1071,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)
|
||||
|
@ -1187,9 +1139,8 @@ func (bc *Blockchain) storeBlock(block *block.Block, txpool *mempool.Pool) error
|
|||
appExecResults = append(appExecResults, aer)
|
||||
aerchan <- aer
|
||||
close(aerchan)
|
||||
d := cache.(*dao.Simple)
|
||||
b := d.GetMPTBatch()
|
||||
mpt, sr, err := bc.stateRoot.AddMPTBatch(block.Index, b, d.Store)
|
||||
b := mpt.MapToMPTBatch(cache.Store.GetStorageChanges())
|
||||
mpt, sr, err := bc.stateRoot.AddMPTBatch(block.Index, b, cache.Store)
|
||||
if err != nil {
|
||||
// Release goroutines, don't care about errors, we already have one.
|
||||
<-aerdone
|
||||
|
@ -1213,7 +1164,7 @@ func (bc *Blockchain) storeBlock(block *block.Block, txpool *mempool.Pool) error
|
|||
}
|
||||
|
||||
if bc.config.SaveStorageBatch {
|
||||
bc.lastBatch = d.GetBatch()
|
||||
bc.lastBatch = cache.GetBatch()
|
||||
}
|
||||
// Every persist cycle we also compact our in-memory MPT. It's flushed
|
||||
// already in AddMPTBatch, so collapsing it is safe.
|
||||
|
@ -1313,7 +1264,7 @@ func (bc *Blockchain) IsExtensibleAllowed(u util.Uint160) bool {
|
|||
return n < len(us)
|
||||
}
|
||||
|
||||
func (bc *Blockchain) runPersist(script []byte, block *block.Block, cache dao.DAO, trig trigger.Type) (*state.AppExecResult, error) {
|
||||
func (bc *Blockchain) runPersist(script []byte, block *block.Block, cache *dao.Simple, trig trigger.Type) (*state.AppExecResult, error) {
|
||||
systemInterop := bc.newInteropContext(trig, cache, block, nil)
|
||||
v := systemInterop.SpawnVM()
|
||||
v.LoadScriptWithFlags(script, callflag.All)
|
||||
|
@ -1335,7 +1286,7 @@ func (bc *Blockchain) runPersist(script []byte, block *block.Block, cache dao.DA
|
|||
}, nil
|
||||
}
|
||||
|
||||
func (bc *Blockchain) handleNotification(note *state.NotificationEvent, d dao.DAO,
|
||||
func (bc *Blockchain) handleNotification(note *state.NotificationEvent, d *dao.Simple,
|
||||
transCache map[util.Uint160]transferData, b *block.Block, h util.Uint256) {
|
||||
if note.Name != "Transfer" {
|
||||
return
|
||||
|
@ -1378,7 +1329,7 @@ func parseUint160(itm stackitem.Item) (util.Uint160, error) {
|
|||
return util.Uint160DecodeBytesBE(bytes)
|
||||
}
|
||||
|
||||
func (bc *Blockchain) processTokenTransfer(cache dao.DAO, transCache map[util.Uint160]transferData,
|
||||
func (bc *Blockchain) processTokenTransfer(cache *dao.Simple, transCache map[util.Uint160]transferData,
|
||||
h util.Uint256, b *block.Block, sc util.Uint160, from util.Uint160, to util.Uint160,
|
||||
amount *big.Int, tokenID []byte) {
|
||||
var id int32
|
||||
|
@ -1432,7 +1383,7 @@ func (bc *Blockchain) processTokenTransfer(cache dao.DAO, transCache map[util.Ui
|
|||
}
|
||||
}
|
||||
|
||||
func appendTokenTransfer(cache dao.DAO, transCache map[util.Uint160]transferData, addr util.Uint160, transfer io.Serializable,
|
||||
func appendTokenTransfer(cache *dao.Simple, transCache map[util.Uint160]transferData, addr util.Uint160, transfer io.Serializable,
|
||||
token int32, bIndex uint32, bTimestamp uint64, isNEP11 bool) error {
|
||||
transferData, ok := transCache[addr]
|
||||
if !ok {
|
||||
|
@ -1480,10 +1431,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.
|
||||
|
@ -2198,7 +2146,7 @@ func (bc *Blockchain) GetEnrollments() ([]state.Validator, error) {
|
|||
|
||||
// GetTestVM returns an interop context with VM set up for a test run.
|
||||
func (bc *Blockchain) GetTestVM(t trigger.Type, tx *transaction.Transaction, b *block.Block) *interop.Context {
|
||||
d := bc.dao.GetWrapped().(*dao.Simple)
|
||||
d := bc.dao.GetPrivate()
|
||||
systemInterop := bc.newInteropContext(t, d, b, tx)
|
||||
vm := systemInterop.SpawnVM()
|
||||
vm.SetPriceGetter(systemInterop.GetPrice)
|
||||
|
@ -2368,7 +2316,7 @@ func hashAndIndexToBytes(h util.Uint256, index uint32) []byte {
|
|||
return buf.Bytes()
|
||||
}
|
||||
|
||||
func (bc *Blockchain) newInteropContext(trigger trigger.Type, d dao.DAO, block *block.Block, tx *transaction.Transaction) *interop.Context {
|
||||
func (bc *Blockchain) newInteropContext(trigger trigger.Type, d *dao.Simple, block *block.Block, tx *transaction.Transaction) *interop.Context {
|
||||
ic := interop.NewContext(trigger, bc, d, bc.contracts.Management.GetContract, bc.contracts.Contracts, block, tx, bc.log)
|
||||
ic.Functions = systemInterops
|
||||
switch {
|
||||
|
|
|
@ -804,7 +804,7 @@ func TestVerifyTx(t *testing.T) {
|
|||
},
|
||||
},
|
||||
}
|
||||
require.NoError(t, bc.dao.StoreAsTransaction(conflicting, bc.blockHeight, nil, nil))
|
||||
require.NoError(t, bc.dao.StoreAsTransaction(conflicting, bc.blockHeight, nil))
|
||||
require.True(t, errors.Is(bc.VerifyTx(tx), ErrHasConflicts))
|
||||
})
|
||||
t.Run("attribute on-chain conflict", func(t *testing.T) {
|
||||
|
@ -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()
|
||||
|
@ -1846,7 +1846,7 @@ func TestBlockchain_InitWithIncompleteStateJump(t *testing.T) {
|
|||
require.NoError(t, bcSpout.AddHeaders(&b.Header))
|
||||
|
||||
// put storage items with STTemp prefix
|
||||
batch := bcSpout.dao.Store.Batch()
|
||||
batch := storage.NewMemCachedStore(bcSpout.dao.Store)
|
||||
tempPrefix := storage.STTempStorage
|
||||
if bcSpout.dao.Version.StoragePrefix == tempPrefix {
|
||||
tempPrefix = storage.STStorage
|
||||
|
@ -1858,7 +1858,8 @@ func TestBlockchain_InitWithIncompleteStateJump(t *testing.T) {
|
|||
batch.Put(key, value)
|
||||
return true
|
||||
})
|
||||
require.NoError(t, bcSpout.dao.Store.PutBatch(batch))
|
||||
_, err := batch.Persist()
|
||||
require.NoError(t, err)
|
||||
|
||||
checkNewBlockchainErr := func(t *testing.T, cfg func(c *config.Config), store storage.Store, shouldFail bool) {
|
||||
unitTestNetCfg, err := config.Load("../../config", testchain.Network())
|
||||
|
@ -1878,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,18 +5,17 @@ import (
|
|||
"context"
|
||||
"encoding/binary"
|
||||
"errors"
|
||||
"fmt"
|
||||
iocore "io"
|
||||
"sort"
|
||||
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/block"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/mpt"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/state"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/storage"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/transaction"
|
||||
"github.com/nspcc-dev/neo-go/pkg/io"
|
||||
"github.com/nspcc-dev/neo-go/pkg/smartcontract/trigger"
|
||||
"github.com/nspcc-dev/neo-go/pkg/util"
|
||||
"github.com/nspcc-dev/neo-go/pkg/util/slice"
|
||||
)
|
||||
|
||||
// HasTransaction errors.
|
||||
|
@ -28,57 +27,21 @@ var (
|
|||
ErrHasConflicts = errors.New("transaction has conflicts")
|
||||
)
|
||||
|
||||
// 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
|
||||
GetAndDecode(entity io.Serializable, key []byte) error
|
||||
GetAppExecResults(hash util.Uint256, trig trigger.Type) ([]state.AppExecResult, error)
|
||||
GetBatch() *storage.MemBatch
|
||||
GetBlock(hash util.Uint256) (*block.Block, error)
|
||||
GetContractScriptHash(id int32) (util.Uint160, error)
|
||||
GetCurrentBlockHeight() (uint32, error)
|
||||
GetCurrentHeaderHeight() (i uint32, h util.Uint256, err error)
|
||||
GetHeaderHashes() ([]util.Uint256, error)
|
||||
GetTokenTransferInfo(acc util.Uint160) (*state.TokenTransferInfo, error)
|
||||
GetTokenTransferLog(acc util.Uint160, start uint64, index uint32, isNEP11 bool) (*state.TokenTransferLog, error)
|
||||
GetStateSyncPoint() (uint32, error)
|
||||
GetStateSyncCurrentBlockHeight() (uint32, error)
|
||||
GetStorageItem(id int32, key []byte) state.StorageItem
|
||||
GetStorageItems(id int32) ([]state.StorageItemWithKey, error)
|
||||
GetStorageItemsWithPrefix(id int32, prefix []byte) ([]state.StorageItemWithKey, error)
|
||||
GetTransaction(hash util.Uint256) (*transaction.Transaction, uint32, error)
|
||||
GetVersion() (Version, error)
|
||||
GetWrapped() DAO
|
||||
HasTransaction(hash util.Uint256) error
|
||||
Persist() (int, error)
|
||||
PersistSync() (int, error)
|
||||
PutContractID(id int32, hash util.Uint160) error
|
||||
PutCurrentHeader(hashAndIndex []byte) error
|
||||
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
|
||||
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
|
||||
StoreAsTransaction(tx *transaction.Transaction, index uint32, aer *state.AppExecResult, buf *io.BufBinWriter) error
|
||||
putTokenTransferInfo(acc util.Uint160, bs *state.TokenTransferInfo, buf *io.BufBinWriter) error
|
||||
}
|
||||
|
||||
// Simple is memCached wrapper around DB, simple DAO implementation.
|
||||
type Simple struct {
|
||||
Version Version
|
||||
Store *storage.MemCachedStore
|
||||
keyBuf []byte
|
||||
dataBuf *io.BufBinWriter
|
||||
}
|
||||
|
||||
// NewSimple creates new simple dao using provided backend store.
|
||||
func NewSimple(backend storage.Store, stateRootInHeader bool, p2pSigExtensions bool) *Simple {
|
||||
st := storage.NewMemCachedStore(backend)
|
||||
return newSimple(st, stateRootInHeader, p2pSigExtensions)
|
||||
}
|
||||
|
||||
func newSimple(st *storage.MemCachedStore, stateRootInHeader bool, p2pSigExtensions bool) *Simple {
|
||||
return &Simple{
|
||||
Version: Version{
|
||||
StoragePrefix: storage.STStorage,
|
||||
|
@ -96,12 +59,31 @@ func (dao *Simple) GetBatch() *storage.MemBatch {
|
|||
|
||||
// GetWrapped returns new DAO instance with another layer of wrapped
|
||||
// MemCachedStore around the current DAO Store.
|
||||
func (dao *Simple) GetWrapped() DAO {
|
||||
func (dao *Simple) GetWrapped() *Simple {
|
||||
d := NewSimple(dao.Store, dao.Version.StateRootInHeader, dao.Version.P2PSigExtensions)
|
||||
d.Version = dao.Version
|
||||
return d
|
||||
}
|
||||
|
||||
// GetPrivate returns new DAO instance with another layer of private
|
||||
// MemCachedStore around the current DAO Store.
|
||||
func (dao *Simple) GetPrivate() *Simple {
|
||||
st := storage.NewPrivateMemCachedStore(dao.Store)
|
||||
d := newSimple(st, dao.Version.StateRootInHeader, dao.Version.P2PSigExtensions)
|
||||
d.Version = dao.Version
|
||||
if dao.keyBuf != nil { // This one is private.
|
||||
d.keyBuf = dao.keyBuf // Thus we can reuse its buffer.
|
||||
} else {
|
||||
d.keyBuf = make([]byte, 0, 1+4+storage.MaxStorageKeyLen) // Prefix, uint32, key.
|
||||
}
|
||||
if dao.dataBuf != nil { // This one is private.
|
||||
d.dataBuf = dao.dataBuf // Thus we can reuse its buffer.
|
||||
} else {
|
||||
d.dataBuf = io.NewBufBinWriter()
|
||||
}
|
||||
return d
|
||||
}
|
||||
|
||||
// GetAndDecode performs get operation and decoding with serializable structures.
|
||||
func (dao *Simple) GetAndDecode(entity io.Serializable, key []byte) error {
|
||||
entityBytes, err := dao.Store.Get(key)
|
||||
|
@ -113,41 +95,37 @@ 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 {
|
||||
key := make([]byte, 5)
|
||||
func (dao *Simple) makeContractIDKey(id int32) []byte {
|
||||
key := dao.getKeyBuf(5)
|
||||
key[0] = byte(storage.STContractID)
|
||||
binary.LittleEndian.PutUint32(key[1:], uint32(id))
|
||||
return key
|
||||
}
|
||||
|
||||
// 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(dao.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(dao.makeContractIDKey(id), hash.BytesBE())
|
||||
}
|
||||
|
||||
// GetContractScriptHash retrieves contract's hash given its ID.
|
||||
func (dao *Simple) GetContractScriptHash(id int32) (util.Uint160, error) {
|
||||
var data = new(util.Uint160)
|
||||
if err := dao.GetAndDecode(data, makeContractIDKey(id)); err != nil {
|
||||
if err := dao.GetAndDecode(data, dao.makeContractIDKey(id)); err != nil {
|
||||
return *data, err
|
||||
}
|
||||
return *data, nil
|
||||
|
@ -155,9 +133,16 @@ func (dao *Simple) GetContractScriptHash(id int32) (util.Uint160, error) {
|
|||
|
||||
// -- start NEP-17 transfer info.
|
||||
|
||||
func (dao *Simple) makeTTIKey(acc util.Uint160) []byte {
|
||||
key := dao.getKeyBuf(1 + util.Uint160Size)
|
||||
key[0] = byte(storage.STTokenTransferInfo)
|
||||
copy(key[1:], acc.BytesBE())
|
||||
return key
|
||||
}
|
||||
|
||||
// GetTokenTransferInfo retrieves NEP-17 transfer info from the cache.
|
||||
func (dao *Simple) GetTokenTransferInfo(acc util.Uint160) (*state.TokenTransferInfo, error) {
|
||||
key := storage.AppendPrefix(storage.STTokenTransferInfo, acc.BytesBE())
|
||||
key := dao.makeTTIKey(acc)
|
||||
bs := state.NewTokenTransferInfo()
|
||||
err := dao.GetAndDecode(bs, key)
|
||||
if err != nil && err != storage.ErrKeyNotFound {
|
||||
|
@ -168,20 +153,19 @@ func (dao *Simple) GetTokenTransferInfo(acc util.Uint160) (*state.TokenTransferI
|
|||
|
||||
// PutTokenTransferInfo saves NEP-17 transfer info in the cache.
|
||||
func (dao *Simple) PutTokenTransferInfo(acc util.Uint160, bs *state.TokenTransferInfo) error {
|
||||
return dao.putTokenTransferInfo(acc, bs, io.NewBufBinWriter())
|
||||
return dao.putTokenTransferInfo(acc, bs, dao.getDataBuf())
|
||||
}
|
||||
|
||||
func (dao *Simple) putTokenTransferInfo(acc util.Uint160, bs *state.TokenTransferInfo, buf *io.BufBinWriter) error {
|
||||
key := storage.AppendPrefix(storage.STTokenTransferInfo, acc.BytesBE())
|
||||
return dao.putWithBuffer(bs, key, buf)
|
||||
return dao.putWithBuffer(bs, dao.makeTTIKey(acc), buf)
|
||||
}
|
||||
|
||||
// -- end NEP-17 transfer info.
|
||||
|
||||
// -- start transfer log.
|
||||
|
||||
func getTokenTransferLogKey(acc util.Uint160, newestTimestamp uint64, index uint32, isNEP11 bool) []byte {
|
||||
key := make([]byte, 1+util.Uint160Size+8+4)
|
||||
func (dao *Simple) getTokenTransferLogKey(acc util.Uint160, newestTimestamp uint64, index uint32, isNEP11 bool) []byte {
|
||||
key := dao.getKeyBuf(1 + util.Uint160Size + 8 + 4)
|
||||
if isNEP11 {
|
||||
key[0] = byte(storage.STNEP11Transfers)
|
||||
} else {
|
||||
|
@ -197,7 +181,7 @@ func getTokenTransferLogKey(acc util.Uint160, newestTimestamp uint64, index uint
|
|||
// the transfer with the newest timestamp up to the oldest transfer. It continues
|
||||
// iteration until false is returned from f. The last non-nil error is returned.
|
||||
func (dao *Simple) SeekNEP17TransferLog(acc util.Uint160, newestTimestamp uint64, f func(*state.NEP17Transfer) (bool, error)) error {
|
||||
key := getTokenTransferLogKey(acc, newestTimestamp, 0, false)
|
||||
key := dao.getTokenTransferLogKey(acc, newestTimestamp, 0, false)
|
||||
prefixLen := 1 + util.Uint160Size
|
||||
var seekErr error
|
||||
dao.Store.Seek(storage.SeekRange{
|
||||
|
@ -219,7 +203,7 @@ func (dao *Simple) SeekNEP17TransferLog(acc util.Uint160, newestTimestamp uint64
|
|||
// the transfer with the newest timestamp up to the oldest transfer. It continues
|
||||
// iteration until false is returned from f. The last non-nil error is returned.
|
||||
func (dao *Simple) SeekNEP11TransferLog(acc util.Uint160, newestTimestamp uint64, f func(*state.NEP11Transfer) (bool, error)) error {
|
||||
key := getTokenTransferLogKey(acc, newestTimestamp, 0, true)
|
||||
key := dao.getTokenTransferLogKey(acc, newestTimestamp, 0, true)
|
||||
prefixLen := 1 + util.Uint160Size
|
||||
var seekErr error
|
||||
dao.Store.Seek(storage.SeekRange{
|
||||
|
@ -239,7 +223,7 @@ func (dao *Simple) SeekNEP11TransferLog(acc util.Uint160, newestTimestamp uint64
|
|||
|
||||
// GetTokenTransferLog retrieves transfer log from the cache.
|
||||
func (dao *Simple) GetTokenTransferLog(acc util.Uint160, newestTimestamp uint64, index uint32, isNEP11 bool) (*state.TokenTransferLog, error) {
|
||||
key := getTokenTransferLogKey(acc, newestTimestamp, index, isNEP11)
|
||||
key := dao.getTokenTransferLogKey(acc, newestTimestamp, index, isNEP11)
|
||||
value, err := dao.Store.Get(key)
|
||||
if err != nil {
|
||||
if err == storage.ErrKeyNotFound {
|
||||
|
@ -251,19 +235,26 @@ 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 {
|
||||
key := getTokenTransferLogKey(acc, start, index, isNEP11)
|
||||
return dao.Store.Put(key, lg.Raw)
|
||||
func (dao *Simple) PutTokenTransferLog(acc util.Uint160, start uint64, index uint32, isNEP11 bool, lg *state.TokenTransferLog) {
|
||||
key := dao.getTokenTransferLogKey(acc, start, index, isNEP11)
|
||||
dao.Store.Put(key, lg.Raw)
|
||||
}
|
||||
|
||||
// -- end transfer log.
|
||||
|
||||
// -- start notification event.
|
||||
|
||||
func (dao *Simple) makeExecutableKey(hash util.Uint256) []byte {
|
||||
key := dao.getKeyBuf(1 + util.Uint256Size)
|
||||
key[0] = byte(storage.DataExecutable)
|
||||
copy(key[1:], hash.BytesBE())
|
||||
return key
|
||||
}
|
||||
|
||||
// GetAppExecResults gets application execution results with the specified trigger from the
|
||||
// given store.
|
||||
func (dao *Simple) GetAppExecResults(hash util.Uint256, trig trigger.Type) ([]state.AppExecResult, error) {
|
||||
key := storage.AppendPrefix(storage.DataExecutable, hash.BytesBE())
|
||||
key := dao.makeExecutableKey(hash)
|
||||
bs, err := dao.Store.Get(key)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -306,7 +297,7 @@ func (dao *Simple) GetAppExecResults(hash util.Uint256, trig trigger.Type) ([]st
|
|||
|
||||
// GetStorageItem returns StorageItem if it exists in the given store.
|
||||
func (dao *Simple) GetStorageItem(id int32, key []byte) state.StorageItem {
|
||||
b, err := dao.Store.Get(makeStorageItemKey(dao.Version.StoragePrefix, id, key))
|
||||
b, err := dao.Store.Get(dao.makeStorageItemKey(id, key))
|
||||
if err != nil {
|
||||
return nil
|
||||
}
|
||||
|
@ -315,16 +306,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 {
|
||||
stKey := makeStorageItemKey(dao.Version.StoragePrefix, id, key)
|
||||
return dao.Store.Put(stKey, si)
|
||||
func (dao *Simple) PutStorageItem(id int32, key []byte, si state.StorageItem) {
|
||||
stKey := dao.makeStorageItemKey(id, key)
|
||||
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 {
|
||||
stKey := makeStorageItemKey(dao.Version.StoragePrefix, id, key)
|
||||
return dao.Store.Delete(stKey)
|
||||
func (dao *Simple) DeleteStorageItem(id int32, key []byte) {
|
||||
stKey := dao.makeStorageItemKey(id, key)
|
||||
dao.Store.Delete(stKey)
|
||||
}
|
||||
|
||||
// GetStorageItems returns all storage items for a given id.
|
||||
|
@ -354,7 +345,7 @@ func (dao *Simple) GetStorageItemsWithPrefix(id int32, prefix []byte) ([]state.S
|
|||
// starting from the point specified). If key or value is to be used outside of f, they
|
||||
// may not be copied. Seek continues iterating until false is returned from f.
|
||||
func (dao *Simple) Seek(id int32, rng storage.SeekRange, f func(k, v []byte) bool) {
|
||||
rng.Prefix = makeStorageItemKey(dao.Version.StoragePrefix, id, rng.Prefix)
|
||||
rng.Prefix = slice.Copy(dao.makeStorageItemKey(id, rng.Prefix)) // f() can use dao too.
|
||||
dao.Store.Seek(rng, func(k, v []byte) bool {
|
||||
return f(k[len(rng.Prefix):], v)
|
||||
})
|
||||
|
@ -364,15 +355,15 @@ func (dao *Simple) Seek(id int32, rng storage.SeekRange, f func(k, v []byte) boo
|
|||
// starting from the point specified) to a channel and returns the channel.
|
||||
// Resulting keys and values may not be copied.
|
||||
func (dao *Simple) SeekAsync(ctx context.Context, id int32, rng storage.SeekRange) chan storage.KeyValue {
|
||||
rng.Prefix = makeStorageItemKey(dao.Version.StoragePrefix, id, rng.Prefix)
|
||||
rng.Prefix = slice.Copy(dao.makeStorageItemKey(id, rng.Prefix))
|
||||
return dao.Store.SeekAsync(ctx, rng, true)
|
||||
}
|
||||
|
||||
// makeStorageItemKey returns a key used to store StorageItem in the DB.
|
||||
func makeStorageItemKey(prefix storage.KeyPrefix, id int32, key []byte) []byte {
|
||||
func (dao *Simple) makeStorageItemKey(id int32, key []byte) []byte {
|
||||
// 1 for prefix + 4 for Uint32 + len(key) for key
|
||||
buf := make([]byte, 5+len(key))
|
||||
buf[0] = byte(prefix)
|
||||
buf := dao.getKeyBuf(5 + len(key))
|
||||
buf[0] = byte(dao.Version.StoragePrefix)
|
||||
binary.LittleEndian.PutUint32(buf[1:], uint32(id))
|
||||
copy(buf[5:], key)
|
||||
return buf
|
||||
|
@ -384,7 +375,7 @@ func makeStorageItemKey(prefix storage.KeyPrefix, id int32, key []byte) []byte {
|
|||
|
||||
// GetBlock returns Block by the given hash if it exists in the store.
|
||||
func (dao *Simple) GetBlock(hash util.Uint256) (*block.Block, error) {
|
||||
key := storage.AppendPrefix(storage.DataExecutable, hash.BytesBE())
|
||||
key := dao.makeExecutableKey(hash)
|
||||
b, err := dao.Store.Get(key)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -553,7 +544,7 @@ func (dao *Simple) GetHeaderHashes() ([]util.Uint256, error) {
|
|||
// GetTransaction returns Transaction and its height by the given hash
|
||||
// if it exists in the store. It does not return dummy transactions.
|
||||
func (dao *Simple) GetTransaction(hash util.Uint256) (*transaction.Transaction, uint32, error) {
|
||||
key := storage.AppendPrefix(storage.DataExecutable, hash.BytesBE())
|
||||
key := dao.makeExecutableKey(hash)
|
||||
b, err := dao.Store.Get(key)
|
||||
if err != nil {
|
||||
return nil, 0, err
|
||||
|
@ -582,28 +573,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 {
|
||||
buf := make([]byte, 4)
|
||||
func (dao *Simple) PutStateSyncPoint(p uint32) {
|
||||
buf := dao.getKeyBuf(4) // It's very small, no point in using BufBinWriter.
|
||||
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 {
|
||||
buf := make([]byte, 4)
|
||||
func (dao *Simple) PutStateSyncCurrentBlockHeight(h uint32) {
|
||||
buf := dao.getKeyBuf(4) // It's very small, no point in using BufBinWriter.
|
||||
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
|
||||
|
@ -623,7 +614,7 @@ func read2000Uint256Hashes(b []byte) ([]util.Uint256, error) {
|
|||
// Transaction hash. It returns an error in case if transaction is in chain
|
||||
// or in the list of conflicting transactions.
|
||||
func (dao *Simple) HasTransaction(hash util.Uint256) error {
|
||||
key := storage.AppendPrefix(storage.DataExecutable, hash.BytesBE())
|
||||
key := dao.makeExecutableKey(hash)
|
||||
bytes, err := dao.Store.Get(key)
|
||||
if err != nil {
|
||||
return nil
|
||||
|
@ -640,13 +631,11 @@ func (dao *Simple) HasTransaction(hash util.Uint256) error {
|
|||
|
||||
// StoreAsBlock stores given block as DataBlock. It can reuse given buffer for
|
||||
// the purpose of value serialization.
|
||||
func (dao *Simple) StoreAsBlock(block *block.Block, aer1 *state.AppExecResult, aer2 *state.AppExecResult, buf *io.BufBinWriter) error {
|
||||
func (dao *Simple) StoreAsBlock(block *block.Block, aer1 *state.AppExecResult, aer2 *state.AppExecResult) error {
|
||||
var (
|
||||
key = storage.AppendPrefix(storage.DataExecutable, block.Hash().BytesBE())
|
||||
key = dao.makeExecutableKey(block.Hash())
|
||||
buf = dao.getDataBuf()
|
||||
)
|
||||
if buf == nil {
|
||||
buf = io.NewBufBinWriter()
|
||||
}
|
||||
buf.WriteB(storage.ExecBlock)
|
||||
b, err := block.Trim()
|
||||
if err != nil {
|
||||
|
@ -662,15 +651,14 @@ 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.
|
||||
func (dao *Simple) DeleteBlock(h util.Uint256, w *io.BufBinWriter) error {
|
||||
batch := dao.Store.Batch()
|
||||
key := make([]byte, util.Uint256Size+1)
|
||||
key[0] = byte(storage.DataExecutable)
|
||||
copy(key[1:], h.BytesBE())
|
||||
// DeleteBlock removes block from dao. It's not atomic, so make sure you're
|
||||
// using private MemCached instance here.
|
||||
func (dao *Simple) DeleteBlock(h util.Uint256) error {
|
||||
key := dao.makeExecutableKey(h)
|
||||
bs, err := dao.Store.Get(key)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -685,53 +673,48 @@ func (dao *Simple) DeleteBlock(h util.Uint256, w *io.BufBinWriter) error {
|
|||
return err
|
||||
}
|
||||
|
||||
if w == nil {
|
||||
w = io.NewBufBinWriter()
|
||||
}
|
||||
w := dao.getDataBuf()
|
||||
w.WriteB(storage.ExecBlock)
|
||||
b.Header.EncodeBinary(w.BinWriter)
|
||||
w.BinWriter.WriteB(0)
|
||||
if w.Err != nil {
|
||||
return w.Err
|
||||
}
|
||||
batch.Put(key, w.Bytes())
|
||||
dao.Store.Put(key, w.Bytes())
|
||||
|
||||
for _, tx := range b.Transactions {
|
||||
copy(key[1:], tx.Hash().BytesBE())
|
||||
batch.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())
|
||||
batch.Delete(key)
|
||||
dao.Store.Delete(key)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return dao.Store.PutBatch(batch)
|
||||
return nil
|
||||
}
|
||||
|
||||
// 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 {
|
||||
if buf == nil {
|
||||
buf = io.NewBufBinWriter()
|
||||
}
|
||||
func (dao *Simple) StoreAsCurrentBlock(block *block.Block) {
|
||||
buf := dao.getDataBuf()
|
||||
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
|
||||
// given tx has conflicts with as DataTransaction with dummy version. It can reuse given
|
||||
// buffer for the purpose of value serialization.
|
||||
func (dao *Simple) StoreAsTransaction(tx *transaction.Transaction, index uint32, aer *state.AppExecResult, buf *io.BufBinWriter) error {
|
||||
key := storage.AppendPrefix(storage.DataExecutable, tx.Hash().BytesBE())
|
||||
if buf == nil {
|
||||
buf = io.NewBufBinWriter()
|
||||
}
|
||||
func (dao *Simple) StoreAsTransaction(tx *transaction.Transaction, index uint32, aer *state.AppExecResult) error {
|
||||
key := dao.makeExecutableKey(tx.Hash())
|
||||
buf := dao.getDataBuf()
|
||||
|
||||
buf.WriteB(storage.ExecTransaction)
|
||||
buf.WriteU32LE(index)
|
||||
tx.EncodeBinary(buf.BinWriter)
|
||||
|
@ -741,10 +724,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,15 +737,27 @@ 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
|
||||
}
|
||||
|
||||
func (dao *Simple) getKeyBuf(len int) []byte {
|
||||
if dao.keyBuf != nil { // Private DAO.
|
||||
return dao.keyBuf[:len] // Should have enough capacity.
|
||||
}
|
||||
return make([]byte, len)
|
||||
}
|
||||
|
||||
func (dao *Simple) getDataBuf() *io.BufBinWriter {
|
||||
if dao.dataBuf != nil {
|
||||
dao.dataBuf.Reset()
|
||||
return dao.dataBuf
|
||||
}
|
||||
return io.NewBufBinWriter()
|
||||
}
|
||||
|
||||
// Persist flushes all the changes made into the (supposedly) persistent
|
||||
// underlying store. It doesn't block accesses to DAO from other threads.
|
||||
func (dao *Simple) Persist() (int, error) {
|
||||
|
@ -778,12 +770,3 @@ func (dao *Simple) Persist() (int, error) {
|
|||
func (dao *Simple) PersistSync() (int, error) {
|
||||
return dao.Store.PersistSync()
|
||||
}
|
||||
|
||||
// GetMPTBatch storage changes to be applied to MPT.
|
||||
func (dao *Simple) GetMPTBatch() mpt.Batch {
|
||||
var b mpt.Batch
|
||||
dao.Store.MemoryStore.SeekAll([]byte{byte(dao.Version.StoragePrefix)}, func(k, v []byte) {
|
||||
b.Add(k[1:], v)
|
||||
})
|
||||
return b
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
@ -102,7 +98,7 @@ func TestPutGetBlock(t *testing.T) {
|
|||
Stack: []stackitem.Item{},
|
||||
},
|
||||
}
|
||||
err := dao.StoreAsBlock(b, appExecResult1, appExecResult2, nil)
|
||||
err := dao.StoreAsBlock(b, appExecResult1, appExecResult2)
|
||||
require.NoError(t, err)
|
||||
gotBlock, err := dao.GetBlock(hash)
|
||||
require.NoError(t, err)
|
||||
|
@ -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)
|
||||
height, err := dao.GetCurrentBlockHeight()
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, uint32(0), height)
|
||||
|
@ -191,7 +185,7 @@ func TestStoreAsTransaction(t *testing.T) {
|
|||
Stack: []stackitem.Item{},
|
||||
},
|
||||
}
|
||||
err := dao.StoreAsTransaction(tx, 0, aer, nil)
|
||||
err := dao.StoreAsTransaction(tx, 0, aer)
|
||||
require.NoError(t, err)
|
||||
err = dao.HasTransaction(hash)
|
||||
require.NotNil(t, err)
|
||||
|
@ -222,7 +216,7 @@ func TestStoreAsTransaction(t *testing.T) {
|
|||
Stack: []stackitem.Item{},
|
||||
},
|
||||
}
|
||||
err := dao.StoreAsTransaction(tx, 0, aer, nil)
|
||||
err := dao.StoreAsTransaction(tx, 0, aer)
|
||||
require.NoError(t, err)
|
||||
err = dao.HasTransaction(hash)
|
||||
require.True(t, errors.Is(err, ErrAlreadyExists))
|
||||
|
@ -271,7 +265,7 @@ func BenchmarkStoreAsTransaction(b *testing.B) {
|
|||
b.ResetTimer()
|
||||
b.ReportAllocs()
|
||||
for n := 0; n < b.N; n++ {
|
||||
err := dao.StoreAsTransaction(tx, 1, aer, nil)
|
||||
err := dao.StoreAsTransaction(tx, 1, aer)
|
||||
if err != nil {
|
||||
b.FailNow()
|
||||
}
|
||||
|
@ -281,18 +275,21 @@ func BenchmarkStoreAsTransaction(b *testing.B) {
|
|||
func TestMakeStorageItemKey(t *testing.T) {
|
||||
var id int32 = 5
|
||||
|
||||
dao := NewSimple(storage.NewMemoryStore(), true, false)
|
||||
|
||||
expected := []byte{byte(storage.STStorage), 0, 0, 0, 0, 1, 2, 3}
|
||||
binary.LittleEndian.PutUint32(expected[1:5], uint32(id))
|
||||
actual := makeStorageItemKey(storage.STStorage, id, []byte{1, 2, 3})
|
||||
actual := dao.makeStorageItemKey(id, []byte{1, 2, 3})
|
||||
require.Equal(t, expected, actual)
|
||||
|
||||
expected = expected[0:5]
|
||||
actual = makeStorageItemKey(storage.STStorage, id, nil)
|
||||
actual = dao.makeStorageItemKey(id, nil)
|
||||
require.Equal(t, expected, actual)
|
||||
|
||||
expected = []byte{byte(storage.STTempStorage), 0, 0, 0, 0, 1, 2, 3}
|
||||
binary.LittleEndian.PutUint32(expected[1:5], uint32(id))
|
||||
actual = makeStorageItemKey(storage.STTempStorage, id, []byte{1, 2, 3})
|
||||
dao.Version.StoragePrefix = storage.STTempStorage
|
||||
actual = dao.makeStorageItemKey(id, []byte{1, 2, 3})
|
||||
require.Equal(t, expected, actual)
|
||||
}
|
||||
|
||||
|
@ -305,7 +302,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 +317,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)
|
||||
|
|
|
@ -54,24 +54,24 @@ type Context struct {
|
|||
Block *block.Block
|
||||
NonceData [16]byte
|
||||
Tx *transaction.Transaction
|
||||
DAO dao.DAO
|
||||
DAO *dao.Simple
|
||||
Notifications []state.NotificationEvent
|
||||
Log *zap.Logger
|
||||
VM *vm.VM
|
||||
Functions []Function
|
||||
Invocations map[util.Uint160]int
|
||||
cancelFuncs []context.CancelFunc
|
||||
getContract func(dao.DAO, util.Uint160) (*state.Contract, error)
|
||||
getContract func(*dao.Simple, util.Uint160) (*state.Contract, error)
|
||||
baseExecFee int64
|
||||
signers []transaction.Signer
|
||||
}
|
||||
|
||||
// NewContext returns new interop context.
|
||||
func NewContext(trigger trigger.Type, bc Ledger, d dao.DAO,
|
||||
getContract func(dao.DAO, util.Uint160) (*state.Contract, error), natives []Contract,
|
||||
func NewContext(trigger trigger.Type, bc Ledger, d *dao.Simple,
|
||||
getContract func(*dao.Simple, util.Uint160) (*state.Contract, error), natives []Contract,
|
||||
block *block.Block, tx *transaction.Transaction, log *zap.Logger) *Context {
|
||||
baseExecFee := int64(DefaultBaseExecFee)
|
||||
dao := d.GetWrapped()
|
||||
dao := d.GetPrivate()
|
||||
|
||||
if bc != nil && (block == nil || block.Index != 0) {
|
||||
baseExecFee = bc.GetBaseExecFee()
|
||||
|
|
|
@ -18,7 +18,7 @@ import (
|
|||
)
|
||||
|
||||
type policyChecker interface {
|
||||
IsBlockedInternal(dao.DAO, util.Uint160) bool
|
||||
IsBlockedInternal(*dao.Simple, util.Uint160) bool
|
||||
}
|
||||
|
||||
// LoadToken calls method specified by token id.
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -16,23 +16,19 @@ type keyValue struct {
|
|||
value []byte
|
||||
}
|
||||
|
||||
// Add adds key-value pair to batch.
|
||||
// If there is an item with the specified key, it is replaced.
|
||||
func (b *Batch) Add(key []byte, value []byte) {
|
||||
path := toNibbles(key)
|
||||
i := sort.Search(len(b.kv), func(i int) bool {
|
||||
return bytes.Compare(path, b.kv[i].key) <= 0
|
||||
})
|
||||
if i == len(b.kv) {
|
||||
b.kv = append(b.kv, keyValue{path, value})
|
||||
} else if bytes.Equal(b.kv[i].key, path) {
|
||||
b.kv[i].value = value
|
||||
} else {
|
||||
b.kv = append(b.kv, keyValue{})
|
||||
copy(b.kv[i+1:], b.kv[i:])
|
||||
b.kv[i].key = path
|
||||
b.kv[i].value = value
|
||||
// MapToMPTBatch makes a Batch from unordered set of storage changes.
|
||||
func MapToMPTBatch(m map[string][]byte) Batch {
|
||||
var b Batch
|
||||
|
||||
b.kv = make([]keyValue, 0, len(m))
|
||||
|
||||
for k, v := range m {
|
||||
b.kv = append(b.kv, keyValue{strToNibbles(k), v}) // Strip storage prefix.
|
||||
}
|
||||
sort.Slice(b.kv, func(i, j int) bool {
|
||||
return bytes.Compare(b.kv[i].key, b.kv[j].key) < 0
|
||||
})
|
||||
return b
|
||||
}
|
||||
|
||||
// PutBatch puts batch to trie.
|
||||
|
|
|
@ -10,12 +10,13 @@ import (
|
|||
)
|
||||
|
||||
func TestBatchAdd(t *testing.T) {
|
||||
b := new(Batch)
|
||||
b.Add([]byte{1}, []byte{2})
|
||||
b.Add([]byte{2, 16}, []byte{3})
|
||||
b.Add([]byte{2, 0}, []byte{4})
|
||||
b.Add([]byte{0, 1}, []byte{5})
|
||||
b.Add([]byte{2, 0}, []byte{6})
|
||||
b := MapToMPTBatch(map[string][]byte{
|
||||
"a\x01": {2},
|
||||
"a\x02\x10": {3},
|
||||
"a\x00\x01": {5},
|
||||
"a\x02\x00": {6},
|
||||
})
|
||||
|
||||
expected := []keyValue{
|
||||
{[]byte{0, 0, 0, 1}, []byte{5}},
|
||||
{[]byte{0, 1}, []byte{2}},
|
||||
|
@ -28,7 +29,7 @@ func TestBatchAdd(t *testing.T) {
|
|||
type pairs = [][2][]byte
|
||||
|
||||
func testIncompletePut(t *testing.T, ps pairs, n int, tr1, tr2 *Trie) {
|
||||
var b Batch
|
||||
var m = make(map[string][]byte)
|
||||
for i, p := range ps {
|
||||
if i < n {
|
||||
if p[1] == nil {
|
||||
|
@ -43,9 +44,10 @@ func testIncompletePut(t *testing.T, ps pairs, n int, tr1, tr2 *Trie) {
|
|||
require.Error(t, tr1.Put(p[0], p[1]), "item %d", i)
|
||||
}
|
||||
}
|
||||
b.Add(p[0], p[1])
|
||||
m["a"+string(p[0])] = p[1]
|
||||
}
|
||||
|
||||
b := MapToMPTBatch(m)
|
||||
num, err := tr2.PutBatch(b)
|
||||
if n == len(ps) {
|
||||
require.NoError(t, err)
|
||||
|
@ -275,8 +277,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)
|
||||
})
|
||||
}
|
||||
|
@ -308,8 +310,10 @@ func TestTrie_PutBatchEmpty(t *testing.T) {
|
|||
// For the sake of coverage.
|
||||
func TestTrie_InvalidNodeType(t *testing.T) {
|
||||
tr := NewTrie(EmptyNode{}, ModeAll, newTestStore())
|
||||
var b Batch
|
||||
b.Add([]byte{1}, []byte("value"))
|
||||
var b = Batch{kv: []keyValue{{
|
||||
key: []byte{0, 1},
|
||||
value: []byte("value"),
|
||||
}}}
|
||||
tr.root = Node(nil)
|
||||
require.Panics(t, func() { _, _ = tr.PutBatch(b) })
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -43,6 +43,17 @@ func toNibbles(path []byte) []byte {
|
|||
return result
|
||||
}
|
||||
|
||||
// strToNibbles mangles path by splitting every byte into 2 containing low- and high- 4-byte part,
|
||||
// ignoring the first byte (prefix).
|
||||
func strToNibbles(path string) []byte {
|
||||
result := make([]byte, (len(path)-1)*2)
|
||||
for i := 0; i < len(path)-1; i++ {
|
||||
result[i*2] = path[i+1] >> 4
|
||||
result[i*2+1] = path[i+1] & 0x0F
|
||||
}
|
||||
return result
|
||||
}
|
||||
|
||||
// fromNibbles performs operation opposite to toNibbles and does no path validity checks.
|
||||
func fromNibbles(path []byte) []byte {
|
||||
result := make([]byte, len(path)/2)
|
||||
|
|
|
@ -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())
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -180,7 +180,7 @@ func (s *Designate) hashFromNodes(r noderoles.Role, nodes keys.PublicKeys) util.
|
|||
return hash.Hash160(script)
|
||||
}
|
||||
|
||||
func (s *Designate) updateCachedRoleData(v *atomic.Value, d dao.DAO, r noderoles.Role) error {
|
||||
func (s *Designate) updateCachedRoleData(v *atomic.Value, d *dao.Simple, r noderoles.Role) error {
|
||||
nodeKeys, height, err := s.GetDesignatedByRole(d, r, math.MaxUint32)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -226,7 +226,7 @@ func (s *Designate) getCachedRoleData(r noderoles.Role) *roleData {
|
|||
}
|
||||
|
||||
// GetLastDesignatedHash returns last designated hash of a given role.
|
||||
func (s *Designate) GetLastDesignatedHash(d dao.DAO, r noderoles.Role) (util.Uint160, error) {
|
||||
func (s *Designate) GetLastDesignatedHash(d *dao.Simple, r noderoles.Role) (util.Uint160, error) {
|
||||
if !s.isValidRole(r) {
|
||||
return util.Uint160{}, ErrInvalidRole
|
||||
}
|
||||
|
@ -244,7 +244,7 @@ func (s *Designate) GetLastDesignatedHash(d dao.DAO, r noderoles.Role) (util.Uin
|
|||
}
|
||||
|
||||
// GetDesignatedByRole returns nodes for role r.
|
||||
func (s *Designate) GetDesignatedByRole(d dao.DAO, r noderoles.Role, index uint32) (keys.PublicKeys, uint32, error) {
|
||||
func (s *Designate) GetDesignatedByRole(d *dao.Simple, r noderoles.Role, index uint32) (keys.PublicKeys, uint32, error) {
|
||||
if !s.isValidRole(r) {
|
||||
return nil, 0, ErrInvalidRole
|
||||
}
|
||||
|
|
|
@ -179,7 +179,7 @@ func getBlockHashFromItem(bc interop.Ledger, item stackitem.Item) util.Uint256 {
|
|||
|
||||
// getTransactionAndHeight returns transaction and its height if it's present
|
||||
// on the chain. It panics if anything goes wrong.
|
||||
func getTransactionAndHeight(d dao.DAO, item stackitem.Item) (*transaction.Transaction, uint32, error) {
|
||||
func getTransactionAndHeight(d *dao.Simple, item stackitem.Item) (*transaction.Transaction, uint32, error) {
|
||||
hashbytes, err := item.TryBytes()
|
||||
if err != nil {
|
||||
panic(err)
|
||||
|
|
|
@ -145,7 +145,7 @@ func (m *Management) getContract(ic *interop.Context, args []stackitem.Item) sta
|
|||
}
|
||||
|
||||
// GetContract returns contract with given hash from given DAO.
|
||||
func (m *Management) GetContract(d dao.DAO, hash util.Uint160) (*state.Contract, error) {
|
||||
func (m *Management) GetContract(d *dao.Simple, hash util.Uint160) (*state.Contract, error) {
|
||||
m.mtx.RLock()
|
||||
cs, ok := m.contracts[hash]
|
||||
m.mtx.RUnlock()
|
||||
|
@ -157,7 +157,7 @@ func (m *Management) GetContract(d dao.DAO, hash util.Uint160) (*state.Contract,
|
|||
return m.getContractFromDAO(d, hash)
|
||||
}
|
||||
|
||||
func (m *Management) getContractFromDAO(d dao.DAO, hash util.Uint160) (*state.Contract, error) {
|
||||
func (m *Management) getContractFromDAO(d *dao.Simple, hash util.Uint160) (*state.Contract, error) {
|
||||
contract := new(state.Contract)
|
||||
key := MakeContractKey(hash)
|
||||
err := getConvertibleFromDAO(m.ID, d, key, contract)
|
||||
|
@ -269,7 +269,7 @@ func (m *Management) markUpdated(h util.Uint160) {
|
|||
|
||||
// Deploy creates contract's hash/ID and saves new contract into the given DAO.
|
||||
// It doesn't run _deploy method and doesn't emit notification.
|
||||
func (m *Management) Deploy(d dao.DAO, sender util.Uint160, neff *nef.File, manif *manifest.Manifest) (*state.Contract, error) {
|
||||
func (m *Management) Deploy(d *dao.Simple, sender util.Uint160, neff *nef.File, manif *manifest.Manifest) (*state.Contract, error) {
|
||||
h := state.CreateContractHash(sender, neff.Checksum, manif.Name)
|
||||
key := MakeContractKey(h)
|
||||
si := d.GetStorageItem(m.ID, key)
|
||||
|
@ -329,7 +329,7 @@ func (m *Management) updateWithData(ic *interop.Context, args []stackitem.Item)
|
|||
|
||||
// Update updates contract's script and/or manifest in the given DAO.
|
||||
// It doesn't run _deploy method and doesn't emit notification.
|
||||
func (m *Management) Update(d dao.DAO, hash util.Uint160, neff *nef.File, manif *manifest.Manifest) (*state.Contract, error) {
|
||||
func (m *Management) Update(d *dao.Simple, hash util.Uint160, neff *nef.File, manif *manifest.Manifest) (*state.Contract, error) {
|
||||
var contract state.Contract
|
||||
|
||||
oldcontract, err := m.GetContract(d, hash)
|
||||
|
@ -380,29 +380,20 @@ func (m *Management) destroy(ic *interop.Context, sis []stackitem.Item) stackite
|
|||
}
|
||||
|
||||
// Destroy drops given contract from DAO along with its storage. It doesn't emit notification.
|
||||
func (m *Management) Destroy(d dao.DAO, hash util.Uint160) error {
|
||||
func (m *Management) Destroy(d *dao.Simple, hash util.Uint160) error {
|
||||
contract, err := m.GetContract(d, hash)
|
||||
if err != nil {
|
||||
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
|
||||
|
@ -413,7 +404,7 @@ func (m *Management) getMinimumDeploymentFee(ic *interop.Context, args []stackit
|
|||
}
|
||||
|
||||
// GetMinimumDeploymentFee returns the minimum required fee for contract deploy.
|
||||
func (m *Management) GetMinimumDeploymentFee(dao dao.DAO) int64 {
|
||||
func (m *Management) GetMinimumDeploymentFee(dao *dao.Simple) int64 {
|
||||
return getIntWithKey(m.ID, dao, keyMinimumDeploymentFee)
|
||||
}
|
||||
|
||||
|
@ -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{}
|
||||
}
|
||||
|
||||
|
@ -498,7 +486,7 @@ func (m *Management) OnPersist(ic *interop.Context) error {
|
|||
// InitializeCache initializes contract cache with the proper values from storage.
|
||||
// Cache initialisation should be done apart from Initialize because Initialize is
|
||||
// called only when deploying native contracts.
|
||||
func (m *Management) InitializeCache(d dao.DAO) error {
|
||||
func (m *Management) InitializeCache(d *dao.Simple) error {
|
||||
m.mtx.Lock()
|
||||
defer m.mtx.Unlock()
|
||||
|
||||
|
@ -564,14 +552,13 @@ 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.
|
||||
func (m *Management) PutContractState(d dao.DAO, cs *state.Contract) error {
|
||||
func (m *Management) PutContractState(d *dao.Simple, cs *state.Contract) error {
|
||||
key := MakeContractKey(cs.Hash)
|
||||
if err := putConvertibleToDAO(m.ID, d, key, cs); err != nil {
|
||||
return err
|
||||
|
@ -580,10 +567,11 @@ 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) {
|
||||
func (m *Management) getNextContractID(d *dao.Simple) (int32, error) {
|
||||
si := d.GetStorageItem(m.ID, keyNextAvailableID)
|
||||
if si == nil {
|
||||
return 0, errors.New("nextAvailableID is not initialized")
|
||||
|
@ -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))
|
||||
})
|
||||
}
|
||||
|
|
|
@ -131,7 +131,7 @@ func (g *GAS) PostPersist(ic *interop.Context) error {
|
|||
}
|
||||
|
||||
// BalanceOf returns native GAS token balance for the acc.
|
||||
func (g *GAS) BalanceOf(d dao.DAO, acc util.Uint160) *big.Int {
|
||||
func (g *GAS) BalanceOf(d *dao.Simple, acc util.Uint160) *big.Int {
|
||||
return g.balanceOfInternal(d, acc)
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
@ -252,7 +241,7 @@ func (n *NEO) Initialize(ic *interop.Context) error {
|
|||
// InitializeCache initializes all NEO cache with the proper values from storage.
|
||||
// Cache initialisation should be done apart from Initialize because Initialize is
|
||||
// called only when deploying native contracts.
|
||||
func (n *NEO) InitializeCache(bc interop.Ledger, d dao.DAO) error {
|
||||
func (n *NEO) InitializeCache(bc interop.Ledger, d *dao.Simple) error {
|
||||
err := n.initConfigCache(bc)
|
||||
if err != nil {
|
||||
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))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -402,7 +391,7 @@ func (n *NEO) PostPersist(ic *interop.Context) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (n *NEO) getGASPerVote(d dao.DAO, key []byte, indexes []uint32) []big.Int {
|
||||
func (n *NEO) getGASPerVote(d *dao.Simple, key []byte, indexes []uint32) []big.Int {
|
||||
sort.Slice(indexes, func(i, j int) bool {
|
||||
return indexes[i] < indexes[j]
|
||||
})
|
||||
|
@ -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
|
||||
|
@ -509,7 +495,7 @@ func (n *NEO) getGASPerBlock(ic *interop.Context, _ []stackitem.Item) stackitem.
|
|||
return stackitem.NewBigInteger(gas)
|
||||
}
|
||||
|
||||
func (n *NEO) getSortedGASRecordFromDAO(d dao.DAO) (gasRecord, error) {
|
||||
func (n *NEO) getSortedGASRecordFromDAO(d *dao.Simple) (gasRecord, error) {
|
||||
grArr, err := d.GetStorageItemsWithPrefix(n.ID, []byte{prefixGASPerBlock})
|
||||
if err != nil {
|
||||
return gasRecord{}, fmt.Errorf("failed to get gas records from storage: %w", err)
|
||||
|
@ -528,7 +514,7 @@ func (n *NEO) getSortedGASRecordFromDAO(d dao.DAO) (gasRecord, error) {
|
|||
}
|
||||
|
||||
// GetGASPerBlock returns gas generated for block with provided index.
|
||||
func (n *NEO) GetGASPerBlock(d dao.DAO, index uint32) *big.Int {
|
||||
func (n *NEO) GetGASPerBlock(d *dao.Simple, index uint32) *big.Int {
|
||||
var (
|
||||
gr gasRecord
|
||||
err error
|
||||
|
@ -581,14 +567,15 @@ 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 {
|
||||
return stackitem.NewBigInteger(big.NewInt(n.getRegisterPriceInternal(ic.DAO)))
|
||||
}
|
||||
|
||||
func (n *NEO) getRegisterPriceInternal(d dao.DAO) int64 {
|
||||
func (n *NEO) getRegisterPriceInternal(d *dao.Simple) int64 {
|
||||
if !n.registerPriceChanged.Load().(bool) {
|
||||
return n.registerPrice.Load().(int64)
|
||||
}
|
||||
|
@ -604,21 +591,16 @@ 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{}
|
||||
}
|
||||
|
||||
func (n *NEO) dropCandidateIfZero(d dao.DAO, pub *keys.PublicKey, c *candidate) (bool, error) {
|
||||
func (n *NEO) dropCandidateIfZero(d *dao.Simple, pub *keys.PublicKey, c *candidate) (bool, error) {
|
||||
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))
|
||||
|
||||
|
@ -650,7 +630,7 @@ func makeVoterKey(pub []byte, prealloc ...[]byte) []byte {
|
|||
|
||||
// CalculateBonus calculates amount of gas generated for holding value NEO from start to end block
|
||||
// and having voted for active committee member.
|
||||
func (n *NEO) CalculateBonus(d dao.DAO, acc util.Uint160, end uint32) (*big.Int, error) {
|
||||
func (n *NEO) CalculateBonus(d *dao.Simple, acc util.Uint160, end uint32) (*big.Int, error) {
|
||||
key := makeAccountKey(acc)
|
||||
si := d.GetStorageItem(n.ID, key)
|
||||
if si == nil {
|
||||
|
@ -663,7 +643,7 @@ func (n *NEO) CalculateBonus(d dao.DAO, acc util.Uint160, end uint32) (*big.Int,
|
|||
return n.calculateBonus(d, st.VoteTo, &st.Balance, st.BalanceHeight, end)
|
||||
}
|
||||
|
||||
func (n *NEO) calculateBonus(d dao.DAO, vote *keys.PublicKey, value *big.Int, start, end uint32) (*big.Int, error) {
|
||||
func (n *NEO) calculateBonus(d *dao.Simple, vote *keys.PublicKey, value *big.Int, start, end uint32) (*big.Int, error) {
|
||||
r, err := n.CalculateNEOHolderReward(d, value, start, end)
|
||||
if err != nil || vote == nil {
|
||||
return r, err
|
||||
|
@ -679,7 +659,7 @@ func (n *NEO) calculateBonus(d dao.DAO, vote *keys.PublicKey, value *big.Int, st
|
|||
}
|
||||
|
||||
// CalculateNEOHolderReward return GAS reward for holding `value` of NEO from start to end block.
|
||||
func (n *NEO) CalculateNEOHolderReward(d dao.DAO, value *big.Int, start, end uint32) (*big.Int, error) {
|
||||
func (n *NEO) CalculateNEOHolderReward(d *dao.Simple, value *big.Int, start, end uint32) (*big.Int, error) {
|
||||
if value.Sign() == 0 || start >= end {
|
||||
return big.NewInt(0), nil
|
||||
} else if value.Sign() < 0 {
|
||||
|
@ -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,12 +818,13 @@ 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).
|
||||
// typ specifies if this modify is occurring during transfer or vote (with old or new validator).
|
||||
func (n *NEO) ModifyAccountVotes(acc *state.NEOBalance, d dao.DAO, value *big.Int, isNewVote bool) error {
|
||||
func (n *NEO) ModifyAccountVotes(acc *state.NEOBalance, d *dao.Simple, value *big.Int, isNewVote bool) error {
|
||||
n.votesChanged.Store(true)
|
||||
if acc.VoteTo != nil {
|
||||
key := makeValidatorKey(acc.VoteTo)
|
||||
|
@ -871,7 +846,7 @@ func (n *NEO) ModifyAccountVotes(acc *state.NEOBalance, d dao.DAO, value *big.In
|
|||
return nil
|
||||
}
|
||||
|
||||
func (n *NEO) getCandidates(d dao.DAO, sortByKey bool) ([]keyWithVotes, error) {
|
||||
func (n *NEO) getCandidates(d *dao.Simple, sortByKey bool) ([]keyWithVotes, error) {
|
||||
siArr, err := d.GetStorageItemsWithPrefix(n.ID, []byte{prefixCandidate})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -911,7 +886,7 @@ func (n *NEO) getCandidates(d dao.DAO, sortByKey bool) ([]keyWithVotes, error) {
|
|||
|
||||
// GetCandidates returns current registered validators list with keys
|
||||
// and votes.
|
||||
func (n *NEO) GetCandidates(d dao.DAO) ([]state.Validator, error) {
|
||||
func (n *NEO) GetCandidates(d *dao.Simple) ([]state.Validator, error) {
|
||||
kvs, err := n.getCandidates(d, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -957,7 +932,7 @@ func (n *NEO) getAccountState(ic *interop.Context, args []stackitem.Item) stacki
|
|||
}
|
||||
|
||||
// ComputeNextBlockValidators returns an actual list of current validators.
|
||||
func (n *NEO) ComputeNextBlockValidators(bc interop.Ledger, d dao.DAO) (keys.PublicKeys, error) {
|
||||
func (n *NEO) ComputeNextBlockValidators(bc interop.Ledger, d *dao.Simple) (keys.PublicKeys, error) {
|
||||
numOfCNs := n.cfg.GetNumOfCNs(bc.BlockHeight() + 1)
|
||||
if vals := n.validators.Load().(keys.PublicKeys); vals != nil && numOfCNs == len(vals) {
|
||||
return vals.Copy(), nil
|
||||
|
@ -978,7 +953,7 @@ func (n *NEO) getCommittee(ic *interop.Context, _ []stackitem.Item) stackitem.It
|
|||
return pubsToArray(pubs)
|
||||
}
|
||||
|
||||
func (n *NEO) modifyVoterTurnout(d dao.DAO, amount *big.Int) error {
|
||||
func (n *NEO) modifyVoterTurnout(d *dao.Simple, amount *big.Int) error {
|
||||
key := []byte{prefixVotersCount}
|
||||
si := d.GetStorageItem(n.ID, key)
|
||||
if si == nil {
|
||||
|
@ -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.
|
||||
|
@ -1015,7 +991,7 @@ func toKeysWithVotes(pubs keys.PublicKeys) keysWithVotes {
|
|||
}
|
||||
|
||||
// computeCommitteeMembers returns public keys of nodes in committee.
|
||||
func (n *NEO) computeCommitteeMembers(bc interop.Ledger, d dao.DAO) (keys.PublicKeys, keysWithVotes, error) {
|
||||
func (n *NEO) computeCommitteeMembers(bc interop.Ledger, d *dao.Simple) (keys.PublicKeys, keysWithVotes, error) {
|
||||
key := []byte{prefixVotersCount}
|
||||
si := d.GetStorageItem(n.ID, key)
|
||||
if si == nil {
|
||||
|
@ -1071,7 +1047,7 @@ func (n *NEO) GetNextBlockValidatorsInternal() keys.PublicKeys {
|
|||
}
|
||||
|
||||
// BalanceOf returns native NEO token balance for the acc.
|
||||
func (n *NEO) BalanceOf(d dao.DAO, acc util.Uint160) (*big.Int, uint32) {
|
||||
func (n *NEO) BalanceOf(d *dao.Simple, acc util.Uint160) (*big.Int, uint32) {
|
||||
key := makeAccountKey(acc)
|
||||
si := d.GetStorageItem(n.ID, key)
|
||||
if si == nil {
|
||||
|
@ -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.Simple, 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))
|
||||
}
|
||||
|
|
|
@ -99,7 +99,7 @@ func (c *nep17TokenNative) TotalSupply(ic *interop.Context, _ []stackitem.Item)
|
|||
return stackitem.NewBigInteger(supply)
|
||||
}
|
||||
|
||||
func (c *nep17TokenNative) getTotalSupply(d dao.DAO) (state.StorageItem, *big.Int) {
|
||||
func (c *nep17TokenNative) getTotalSupply(d *dao.Simple) (state.StorageItem, *big.Int) {
|
||||
si := d.GetStorageItem(c.ID, totalSupplyKey)
|
||||
if si == nil {
|
||||
si = []byte{}
|
||||
|
@ -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.Simple, 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.
|
||||
|
@ -237,7 +237,7 @@ func (c *nep17TokenNative) balanceOf(ic *interop.Context, args []stackitem.Item)
|
|||
return stackitem.NewBigInteger(c.balanceOfInternal(ic.DAO, h))
|
||||
}
|
||||
|
||||
func (c *nep17TokenNative) balanceOfInternal(d dao.DAO, h util.Uint160) *big.Int {
|
||||
func (c *nep17TokenNative) balanceOfInternal(d *dao.Simple, h util.Uint160) *big.Int {
|
||||
key := makeAccountKey(h)
|
||||
si := d.GetStorageItem(c.ID, key)
|
||||
if si == nil {
|
||||
|
@ -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)
|
||||
}
|
||||
|
||||
|
@ -302,7 +293,7 @@ func (n *Notary) balanceOf(ic *interop.Context, args []stackitem.Item) stackitem
|
|||
}
|
||||
|
||||
// BalanceOf is an internal representation of `balanceOf` Notary method.
|
||||
func (n *Notary) BalanceOf(dao dao.DAO, acc util.Uint160) *big.Int {
|
||||
func (n *Notary) BalanceOf(dao *dao.Simple, acc util.Uint160) *big.Int {
|
||||
deposit := n.GetDepositFor(dao, acc)
|
||||
if deposit == nil {
|
||||
return big.NewInt(0)
|
||||
|
@ -317,7 +308,7 @@ func (n *Notary) expirationOf(ic *interop.Context, args []stackitem.Item) stacki
|
|||
}
|
||||
|
||||
// ExpirationOf is an internal representation of `expirationOf` Notary method.
|
||||
func (n *Notary) ExpirationOf(dao dao.DAO, acc util.Uint160) uint32 {
|
||||
func (n *Notary) ExpirationOf(dao *dao.Simple, acc util.Uint160) uint32 {
|
||||
deposit := n.GetDepositFor(dao, acc)
|
||||
if deposit == nil {
|
||||
return 0
|
||||
|
@ -369,7 +360,7 @@ func (n *Notary) verify(ic *interop.Context, args []stackitem.Item) stackitem.It
|
|||
}
|
||||
|
||||
// GetNotaryNodes returns public keys of notary nodes.
|
||||
func (n *Notary) GetNotaryNodes(d dao.DAO) (keys.PublicKeys, error) {
|
||||
func (n *Notary) GetNotaryNodes(d *dao.Simple) (keys.PublicKeys, error) {
|
||||
nodes, _, err := n.Desig.GetDesignatedByRole(d, noderoles.P2PNotary, math.MaxUint32)
|
||||
return nodes, err
|
||||
}
|
||||
|
@ -380,7 +371,7 @@ func (n *Notary) getMaxNotValidBeforeDelta(ic *interop.Context, _ []stackitem.It
|
|||
}
|
||||
|
||||
// GetMaxNotValidBeforeDelta is an internal representation of Notary getMaxNotValidBeforeDelta method.
|
||||
func (n *Notary) GetMaxNotValidBeforeDelta(dao dao.DAO) uint32 {
|
||||
func (n *Notary) GetMaxNotValidBeforeDelta(dao *dao.Simple) uint32 {
|
||||
n.lock.RLock()
|
||||
defer n.lock.RUnlock()
|
||||
if n.isValid {
|
||||
|
@ -402,17 +393,14 @@ 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{}
|
||||
}
|
||||
|
||||
// GetDepositFor returns state.Deposit for the account specified. It returns nil in case if
|
||||
// deposit is not found in storage and panics in case of any other error.
|
||||
func (n *Notary) GetDepositFor(dao dao.DAO, acc util.Uint160) *state.Deposit {
|
||||
func (n *Notary) GetDepositFor(dao *dao.Simple, acc util.Uint160) *state.Deposit {
|
||||
key := append([]byte{prefixDeposit}, acc.BytesBE()...)
|
||||
deposit := new(state.Deposit)
|
||||
err := getConvertibleFromDAO(n.ID, dao, key, deposit)
|
||||
|
@ -426,15 +414,15 @@ func (n *Notary) GetDepositFor(dao dao.DAO, acc util.Uint160) *state.Deposit {
|
|||
}
|
||||
|
||||
// putDepositFor puts deposit on the balance of the specified account in the storage.
|
||||
func (n *Notary) putDepositFor(dao dao.DAO, deposit *state.Deposit, acc util.Uint160) error {
|
||||
func (n *Notary) putDepositFor(dao *dao.Simple, deposit *state.Deposit, acc util.Uint160) error {
|
||||
key := append([]byte{prefixDeposit}, acc.BytesBE()...)
|
||||
return putConvertibleToDAO(n.ID, dao, key, deposit)
|
||||
}
|
||||
|
||||
// removeDepositFor removes deposit from the storage.
|
||||
func (n *Notary) removeDepositFor(dao dao.DAO, acc util.Uint160) error {
|
||||
func (n *Notary) removeDepositFor(dao *dao.Simple, 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())
|
||||
|
@ -395,7 +387,7 @@ func (o *Oracle) RequestInternal(ic *interop.Context, url string, filter *string
|
|||
}
|
||||
|
||||
// PutRequestInternal puts oracle request with the specified id to d.
|
||||
func (o *Oracle) PutRequestInternal(id uint64, req *state.OracleRequest, d dao.DAO) error {
|
||||
func (o *Oracle) PutRequestInternal(id uint64, req *state.OracleRequest, d *dao.Simple) error {
|
||||
reqKey := makeRequestKey(id)
|
||||
if err := putConvertibleToDAO(o.ID, d, reqKey, req); err != nil {
|
||||
return err
|
||||
|
@ -416,25 +408,25 @@ func (o *Oracle) PutRequestInternal(id uint64, req *state.OracleRequest, d dao.D
|
|||
}
|
||||
|
||||
// GetScriptHash returns script hash or oracle nodes.
|
||||
func (o *Oracle) GetScriptHash(d dao.DAO) (util.Uint160, error) {
|
||||
func (o *Oracle) GetScriptHash(d *dao.Simple) (util.Uint160, error) {
|
||||
return o.Desig.GetLastDesignatedHash(d, noderoles.Oracle)
|
||||
}
|
||||
|
||||
// GetOracleNodes returns public keys of oracle nodes.
|
||||
func (o *Oracle) GetOracleNodes(d dao.DAO) (keys.PublicKeys, error) {
|
||||
func (o *Oracle) GetOracleNodes(d *dao.Simple) (keys.PublicKeys, error) {
|
||||
nodes, _, err := o.Desig.GetDesignatedByRole(d, noderoles.Oracle, math.MaxUint32)
|
||||
return nodes, err
|
||||
}
|
||||
|
||||
// GetRequestInternal returns request by ID and key under which it is stored.
|
||||
func (o *Oracle) GetRequestInternal(d dao.DAO, id uint64) (*state.OracleRequest, error) {
|
||||
func (o *Oracle) GetRequestInternal(d *dao.Simple, id uint64) (*state.OracleRequest, error) {
|
||||
key := makeRequestKey(id)
|
||||
req := new(state.OracleRequest)
|
||||
return req, o.getConvertibleFromDAO(d, key, req)
|
||||
}
|
||||
|
||||
// GetIDListInternal returns request by ID and key under which it is stored.
|
||||
func (o *Oracle) GetIDListInternal(d dao.DAO, url string) (*IDList, error) {
|
||||
func (o *Oracle) GetIDListInternal(d *dao.Simple, url string) (*IDList, error) {
|
||||
key := makeIDListKey(url)
|
||||
idList := new(IDList)
|
||||
return idList, o.getConvertibleFromDAO(d, key, idList)
|
||||
|
@ -448,7 +440,7 @@ func (o *Oracle) getPrice(ic *interop.Context, _ []stackitem.Item) stackitem.Ite
|
|||
return stackitem.NewBigInteger(big.NewInt(o.getPriceInternal(ic.DAO)))
|
||||
}
|
||||
|
||||
func (o *Oracle) getPriceInternal(d dao.DAO) int64 {
|
||||
func (o *Oracle) getPriceInternal(d *dao.Simple) int64 {
|
||||
if !o.requestPriceChanged.Load().(bool) {
|
||||
return o.requestPrice.Load().(int64)
|
||||
}
|
||||
|
@ -463,14 +455,12 @@ 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{}
|
||||
}
|
||||
|
||||
func (o *Oracle) getOriginalTxID(d dao.DAO, tx *transaction.Transaction) util.Uint256 {
|
||||
func (o *Oracle) getOriginalTxID(d *dao.Simple, tx *transaction.Transaction) util.Uint256 {
|
||||
for i := range tx.Attributes {
|
||||
if tx.Attributes[i].Type == transaction.OracleResponseT {
|
||||
id := tx.Attributes[i].Value.(*transaction.OracleResponse).ID
|
||||
|
@ -482,7 +472,7 @@ func (o *Oracle) getOriginalTxID(d dao.DAO, tx *transaction.Transaction) util.Ui
|
|||
}
|
||||
|
||||
// getRequests returns all requests which have not been finished yet.
|
||||
func (o *Oracle) getRequests(d dao.DAO) (map[uint64]*state.OracleRequest, error) {
|
||||
func (o *Oracle) getRequests(d *dao.Simple) (map[uint64]*state.OracleRequest, error) {
|
||||
arr, err := d.GetStorageItemsWithPrefix(o.ID, prefixRequest)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -514,12 +504,12 @@ func makeIDListKey(url string) []byte {
|
|||
return append(prefixIDList, hash.Hash160([]byte(url)).BytesBE()...)
|
||||
}
|
||||
|
||||
func (o *Oracle) getConvertibleFromDAO(d dao.DAO, key []byte, item stackitem.Convertible) error {
|
||||
func (o *Oracle) getConvertibleFromDAO(d *dao.Simple, key []byte, item stackitem.Convertible) error {
|
||||
return getConvertibleFromDAO(o.ID, d, key, item)
|
||||
}
|
||||
|
||||
// updateCache updates cached Oracle values if they've been changed.
|
||||
func (o *Oracle) updateCache(d dao.DAO) error {
|
||||
func (o *Oracle) updateCache(d *dao.Simple) error {
|
||||
orc, _ := o.Module.Load().(services.Oracle)
|
||||
if orc == nil {
|
||||
return nil
|
||||
|
|
|
@ -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
|
||||
|
@ -188,7 +182,7 @@ func (p *Policy) getFeePerByte(ic *interop.Context, _ []stackitem.Item) stackite
|
|||
}
|
||||
|
||||
// GetFeePerByteInternal returns required transaction's fee per byte.
|
||||
func (p *Policy) GetFeePerByteInternal(dao dao.DAO) int64 {
|
||||
func (p *Policy) GetFeePerByteInternal(dao *dao.Simple) int64 {
|
||||
p.lock.RLock()
|
||||
defer p.lock.RUnlock()
|
||||
if p.isValid {
|
||||
|
@ -198,7 +192,7 @@ func (p *Policy) GetFeePerByteInternal(dao dao.DAO) int64 {
|
|||
}
|
||||
|
||||
// GetMaxVerificationGas returns maximum gas allowed to be burned during verificaion.
|
||||
func (p *Policy) GetMaxVerificationGas(_ dao.DAO) int64 {
|
||||
func (p *Policy) GetMaxVerificationGas(_ *dao.Simple) int64 {
|
||||
if p.isValid {
|
||||
return p.maxVerificationGas
|
||||
}
|
||||
|
@ -210,7 +204,7 @@ func (p *Policy) getExecFeeFactor(ic *interop.Context, _ []stackitem.Item) stack
|
|||
}
|
||||
|
||||
// GetExecFeeFactorInternal returns current execution fee factor.
|
||||
func (p *Policy) GetExecFeeFactorInternal(d dao.DAO) int64 {
|
||||
func (p *Policy) GetExecFeeFactorInternal(d *dao.Simple) int64 {
|
||||
p.lock.RLock()
|
||||
defer p.lock.RUnlock()
|
||||
if p.isValid {
|
||||
|
@ -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{}
|
||||
}
|
||||
|
@ -244,7 +235,7 @@ func (p *Policy) isBlocked(ic *interop.Context, args []stackitem.Item) stackitem
|
|||
}
|
||||
|
||||
// IsBlockedInternal checks whether provided account is blocked.
|
||||
func (p *Policy) IsBlockedInternal(dao dao.DAO, hash util.Uint160) bool {
|
||||
func (p *Policy) IsBlockedInternal(dao *dao.Simple, hash util.Uint160) bool {
|
||||
p.lock.RLock()
|
||||
defer p.lock.RUnlock()
|
||||
if p.isValid {
|
||||
|
@ -266,7 +257,7 @@ func (p *Policy) getStoragePrice(ic *interop.Context, _ []stackitem.Item) stacki
|
|||
}
|
||||
|
||||
// GetStoragePriceInternal returns current execution fee factor.
|
||||
func (p *Policy) GetStoragePriceInternal(d dao.DAO) int64 {
|
||||
func (p *Policy) GetStoragePriceInternal(d *dao.Simple) int64 {
|
||||
p.lock.RLock()
|
||||
defer p.lock.RUnlock()
|
||||
if p.isValid {
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -362,7 +341,7 @@ func (p *Policy) unblockAccount(ic *interop.Context, args []stackitem.Item) stac
|
|||
// CheckPolicy checks whether transaction conforms to current policy restrictions
|
||||
// like not being signed by blocked account or not exceeding block-level system
|
||||
// fee limit.
|
||||
func (p *Policy) CheckPolicy(d dao.DAO, tx *transaction.Transaction) error {
|
||||
func (p *Policy) CheckPolicy(d *dao.Simple, tx *transaction.Transaction) error {
|
||||
for _, signer := range tx.Signers {
|
||||
if p.IsBlockedInternal(d, signer.Account) {
|
||||
return fmt.Errorf("account %s is blocked", signer.Account.StringLE())
|
||||
|
|
|
@ -15,7 +15,7 @@ import (
|
|||
var intOne = big.NewInt(1)
|
||||
var intTwo = big.NewInt(2)
|
||||
|
||||
func getConvertibleFromDAO(id int32, d dao.DAO, key []byte, conv stackitem.Convertible) error {
|
||||
func getConvertibleFromDAO(id int32, d *dao.Simple, key []byte, conv stackitem.Convertible) error {
|
||||
si := d.GetStorageItem(id, key)
|
||||
if si == nil {
|
||||
return storage.ErrKeyNotFound
|
||||
|
@ -23,19 +23,20 @@ func getConvertibleFromDAO(id int32, d dao.DAO, key []byte, conv stackitem.Conve
|
|||
return stackitem.DeserializeConvertible(si, conv)
|
||||
}
|
||||
|
||||
func putConvertibleToDAO(id int32, d dao.DAO, key []byte, conv stackitem.Convertible) error {
|
||||
func putConvertibleToDAO(id int32, d *dao.Simple, key []byte, conv stackitem.Convertible) error {
|
||||
data, err := stackitem.SerializeConvertible(conv)
|
||||
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.Simple, key []byte, value int64) {
|
||||
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.Simple, key []byte) int64 {
|
||||
si := dao.GetStorageItem(id, key)
|
||||
if si == nil {
|
||||
panic(fmt.Errorf("item with id = %d and key = %s is not initialized", id, hex.EncodeToString(key)))
|
||||
|
|
|
@ -146,13 +146,13 @@ func (s *Module) CleanStorage() error {
|
|||
if s.localHeight.Load() != 0 {
|
||||
return fmt.Errorf("can't clean MPT data for non-genesis block: expected local stateroot height 0, got %d", s.localHeight.Load())
|
||||
}
|
||||
b := s.Store.Batch()
|
||||
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)
|
||||
return true
|
||||
})
|
||||
err := s.Store.PutBatch(b)
|
||||
_, err := b.Persist()
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to remove outdated MPT-reated items: %w", err)
|
||||
}
|
||||
|
@ -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),
|
||||
|
@ -332,26 +329,20 @@ func (s *Module) AddBlock(block *block.Block) error {
|
|||
return errors.New("invalid block: MerkleRoot mismatch")
|
||||
}
|
||||
}
|
||||
cache := s.dao.GetWrapped()
|
||||
writeBuf := io.NewBufBinWriter()
|
||||
if err := cache.StoreAsBlock(block, nil, nil, writeBuf); err != nil {
|
||||
cache := s.dao.GetPrivate()
|
||||
if err := cache.StoreAsBlock(block, nil, nil); err != nil {
|
||||
return err
|
||||
}
|
||||
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 {
|
||||
if err := cache.StoreAsTransaction(tx, block.Index, nil); err != nil {
|
||||
return err
|
||||
}
|
||||
writeBuf.Reset()
|
||||
}
|
||||
|
||||
_, err = cache.Persist()
|
||||
_, err := cache.Persist()
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to persist results: %w", err)
|
||||
}
|
||||
|
|
|
@ -47,15 +47,6 @@ func NewBoltDBStore(cfg BoltDBOptions) (*BoltDBStore, error) {
|
|||
return &BoltDBStore{db: db}, nil
|
||||
}
|
||||
|
||||
// Put implements the Store interface.
|
||||
func (s *BoltDBStore) Put(key, value []byte) error {
|
||||
return s.db.Update(func(tx *bbolt.Tx) error {
|
||||
b := tx.Bucket(Bucket)
|
||||
err := b.Put(key, value)
|
||||
return err
|
||||
})
|
||||
}
|
||||
|
||||
// Get implements the Store interface.
|
||||
func (s *BoltDBStore) Get(key []byte) (val []byte, err error) {
|
||||
err = s.db.View(func(tx *bbolt.Tx) error {
|
||||
|
@ -73,34 +64,22 @@ func (s *BoltDBStore) Get(key []byte) (val []byte, err error) {
|
|||
return
|
||||
}
|
||||
|
||||
// Delete implements the Store interface.
|
||||
func (s *BoltDBStore) Delete(key []byte) error {
|
||||
return s.db.Update(func(tx *bbolt.Tx) error {
|
||||
b := tx.Bucket(Bucket)
|
||||
return b.Delete(key)
|
||||
})
|
||||
}
|
||||
|
||||
// PutBatch implements the Store interface.
|
||||
func (s *BoltDBStore) PutBatch(batch Batch) error {
|
||||
memBatch := batch.(*MemoryBatch)
|
||||
return s.PutChangeSet(memBatch.mem)
|
||||
}
|
||||
|
||||
// PutChangeSet implements the Store interface.
|
||||
func (s *BoltDBStore) PutChangeSet(puts map[string][]byte) error {
|
||||
func (s *BoltDBStore) PutChangeSet(puts map[string][]byte, stores map[string][]byte) error {
|
||||
var err error
|
||||
|
||||
return s.db.Update(func(tx *bbolt.Tx) error {
|
||||
b := tx.Bucket(Bucket)
|
||||
for k, v := range puts {
|
||||
if v != nil {
|
||||
err = b.Put([]byte(k), v)
|
||||
} else {
|
||||
err = b.Delete([]byte(k))
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
for _, m := range []map[string][]byte{puts, stores} {
|
||||
for k, v := range m {
|
||||
if v != nil {
|
||||
err = b.Put([]byte(k), v)
|
||||
} else {
|
||||
err = b.Delete([]byte(k))
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
|
@ -166,12 +145,6 @@ func boltSeek(txopener func(func(*bbolt.Tx) error) error, rng SeekRange, f func(
|
|||
})
|
||||
}
|
||||
|
||||
// Batch implements the Batch interface and returns a boltdb
|
||||
// compatible Batch.
|
||||
func (s *BoltDBStore) Batch() Batch {
|
||||
return newMemoryBatch()
|
||||
}
|
||||
|
||||
// Close releases all db resources.
|
||||
func (s *BoltDBStore) Close() error {
|
||||
return s.db.Close()
|
||||
|
|
|
@ -36,11 +36,6 @@ func NewLevelDBStore(cfg LevelDBOptions) (*LevelDBStore, error) {
|
|||
}, nil
|
||||
}
|
||||
|
||||
// Put implements the Store interface.
|
||||
func (s *LevelDBStore) Put(key, value []byte) error {
|
||||
return s.db.Put(key, value, nil)
|
||||
}
|
||||
|
||||
// Get implements the Store interface.
|
||||
func (s *LevelDBStore) Get(key []byte) ([]byte, error) {
|
||||
value, err := s.db.Get(key, nil)
|
||||
|
@ -50,32 +45,23 @@ func (s *LevelDBStore) Get(key []byte) ([]byte, error) {
|
|||
return value, err
|
||||
}
|
||||
|
||||
// Delete implements the Store interface.
|
||||
func (s *LevelDBStore) Delete(key []byte) error {
|
||||
return s.db.Delete(key, nil)
|
||||
}
|
||||
|
||||
// PutBatch implements the Store interface.
|
||||
func (s *LevelDBStore) PutBatch(batch Batch) error {
|
||||
lvldbBatch := batch.(*leveldb.Batch)
|
||||
return s.db.Write(lvldbBatch, nil)
|
||||
}
|
||||
|
||||
// PutChangeSet implements the Store interface.
|
||||
func (s *LevelDBStore) PutChangeSet(puts map[string][]byte) error {
|
||||
func (s *LevelDBStore) PutChangeSet(puts map[string][]byte, stores map[string][]byte) error {
|
||||
tx, err := s.db.OpenTransaction()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for k := range puts {
|
||||
if puts[k] != nil {
|
||||
err = tx.Put([]byte(k), puts[k], nil)
|
||||
} else {
|
||||
err = tx.Delete([]byte(k), nil)
|
||||
}
|
||||
if err != nil {
|
||||
tx.Discard()
|
||||
return err
|
||||
for _, m := range []map[string][]byte{puts, stores} {
|
||||
for k := range m {
|
||||
if m[k] != nil {
|
||||
err = tx.Put([]byte(k), m[k], nil)
|
||||
} else {
|
||||
err = tx.Delete([]byte(k), nil)
|
||||
}
|
||||
if err != nil {
|
||||
tx.Discard()
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
return tx.Commit()
|
||||
|
@ -131,12 +117,6 @@ func (s *LevelDBStore) seek(iter iterator.Iterator, backwards bool, f func(k, v
|
|||
iter.Release()
|
||||
}
|
||||
|
||||
// Batch implements the Batch interface and returns a leveldb
|
||||
// compatible Batch.
|
||||
func (s *LevelDBStore) Batch() Batch {
|
||||
return new(leveldb.Batch)
|
||||
}
|
||||
|
||||
// Close implements the Store interface.
|
||||
func (s *LevelDBStore) Close() error {
|
||||
return s.db.Close()
|
||||
|
|
|
@ -15,6 +15,7 @@ import (
|
|||
type MemCachedStore struct {
|
||||
MemoryStore
|
||||
|
||||
private bool
|
||||
// plock protects Persist from double entrance.
|
||||
plock sync.Mutex
|
||||
// Persistent Store.
|
||||
|
@ -51,11 +52,50 @@ func NewMemCachedStore(lower Store) *MemCachedStore {
|
|||
}
|
||||
}
|
||||
|
||||
// NewPrivateMemCachedStore creates a new private (unlocked) MemCachedStore object.
|
||||
// Private cached stores are closed after Persist.
|
||||
func NewPrivateMemCachedStore(lower Store) *MemCachedStore {
|
||||
return &MemCachedStore{
|
||||
MemoryStore: *NewMemoryStore(),
|
||||
private: true,
|
||||
ps: lower,
|
||||
}
|
||||
}
|
||||
|
||||
// lock write-locks non-private store.
|
||||
func (s *MemCachedStore) lock() {
|
||||
if !s.private {
|
||||
s.mut.Lock()
|
||||
}
|
||||
}
|
||||
|
||||
// unlock unlocks non-private store.
|
||||
func (s *MemCachedStore) unlock() {
|
||||
if !s.private {
|
||||
s.mut.Unlock()
|
||||
}
|
||||
}
|
||||
|
||||
// rlock read-locks non-private store.
|
||||
func (s *MemCachedStore) rlock() {
|
||||
if !s.private {
|
||||
s.mut.RLock()
|
||||
}
|
||||
}
|
||||
|
||||
// runlock drops read lock for non-private stores.
|
||||
func (s *MemCachedStore) runlock() {
|
||||
if !s.private {
|
||||
s.mut.RUnlock()
|
||||
}
|
||||
}
|
||||
|
||||
// Get implements the Store interface.
|
||||
func (s *MemCachedStore) Get(key []byte) ([]byte, error) {
|
||||
s.mut.RLock()
|
||||
defer s.mut.RUnlock()
|
||||
if val, ok := s.mem[string(key)]; ok {
|
||||
s.rlock()
|
||||
defer s.runlock()
|
||||
m := s.chooseMap(key)
|
||||
if val, ok := m[string(key)]; ok {
|
||||
if val == nil {
|
||||
return nil, ErrKeyNotFound
|
||||
}
|
||||
|
@ -64,32 +104,67 @@ func (s *MemCachedStore) Get(key []byte) ([]byte, error) {
|
|||
return s.ps.Get(key)
|
||||
}
|
||||
|
||||
// Put puts new KV pair into the store.
|
||||
func (s *MemCachedStore) Put(key, value []byte) {
|
||||
newKey := string(key)
|
||||
vcopy := slice.Copy(value)
|
||||
s.lock()
|
||||
put(s.chooseMap(key), newKey, vcopy)
|
||||
s.unlock()
|
||||
}
|
||||
|
||||
// Delete drops KV pair from the store. Never returns an error.
|
||||
func (s *MemCachedStore) Delete(key []byte) {
|
||||
newKey := string(key)
|
||||
s.lock()
|
||||
put(s.chooseMap(key), newKey, nil)
|
||||
s.unlock()
|
||||
}
|
||||
|
||||
// GetBatch returns currently accumulated changeset.
|
||||
func (s *MemCachedStore) GetBatch() *MemBatch {
|
||||
s.mut.RLock()
|
||||
defer s.mut.RUnlock()
|
||||
|
||||
s.rlock()
|
||||
defer s.runlock()
|
||||
var b MemBatch
|
||||
|
||||
b.Put = make([]KeyValueExists, 0, len(s.mem))
|
||||
b.Put = make([]KeyValueExists, 0, len(s.mem)+len(s.stor))
|
||||
b.Deleted = make([]KeyValueExists, 0)
|
||||
for k, v := range s.mem {
|
||||
key := []byte(k)
|
||||
_, err := s.ps.Get(key)
|
||||
if v == nil {
|
||||
b.Deleted = append(b.Deleted, KeyValueExists{KeyValue: KeyValue{Key: key}, Exists: err == nil})
|
||||
} else {
|
||||
b.Put = append(b.Put, KeyValueExists{KeyValue: KeyValue{Key: key, Value: v}, Exists: err == nil})
|
||||
for _, m := range []map[string][]byte{s.mem, s.stor} {
|
||||
for k, v := range m {
|
||||
key := []byte(k)
|
||||
_, err := s.ps.Get(key)
|
||||
if v == nil {
|
||||
b.Deleted = append(b.Deleted, KeyValueExists{KeyValue: KeyValue{Key: key}, Exists: err == nil})
|
||||
} else {
|
||||
b.Put = append(b.Put, KeyValueExists{KeyValue: KeyValue{Key: key, Value: v}, Exists: err == nil})
|
||||
}
|
||||
}
|
||||
}
|
||||
return &b
|
||||
}
|
||||
|
||||
// PutChangeSet implements the Store interface. Never returns an error.
|
||||
func (s *MemCachedStore) PutChangeSet(puts map[string][]byte, stores map[string][]byte) error {
|
||||
s.lock()
|
||||
s.MemoryStore.putChangeSet(puts, stores)
|
||||
s.unlock()
|
||||
return nil
|
||||
}
|
||||
|
||||
// Seek implements the Store interface.
|
||||
func (s *MemCachedStore) Seek(rng SeekRange, f func(k, v []byte) bool) {
|
||||
s.seek(context.Background(), rng, false, f)
|
||||
}
|
||||
|
||||
// GetStorageChanges returns all current storage changes. It can only be done for private
|
||||
// MemCachedStore.
|
||||
func (s *MemCachedStore) GetStorageChanges() map[string][]byte {
|
||||
if !s.private {
|
||||
panic("GetStorageChanges called on shared MemCachedStore")
|
||||
}
|
||||
return s.stor
|
||||
}
|
||||
|
||||
// SeekAsync returns non-buffered channel with matching KeyValue pairs. Key and
|
||||
// value slices may not be copied and may be modified. SeekAsync can guarantee
|
||||
// that key-value items are sorted by key in ascending way.
|
||||
|
@ -130,8 +205,9 @@ func (s *MemCachedStore) seek(ctx context.Context, rng SeekRange, cutPrefix bool
|
|||
return strings.HasPrefix(key, sPrefix) && (lStart == 0 || strings.Compare(key[lPrefix:], sStart) <= 0)
|
||||
}
|
||||
}
|
||||
s.mut.RLock()
|
||||
for k, v := range s.MemoryStore.mem {
|
||||
s.rlock()
|
||||
m := s.MemoryStore.chooseMap(rng.Prefix)
|
||||
for k, v := range m {
|
||||
if isKeyOK(k) {
|
||||
memRes = append(memRes, KeyValueExists{
|
||||
KeyValue: KeyValue{
|
||||
|
@ -143,8 +219,7 @@ func (s *MemCachedStore) seek(ctx context.Context, rng SeekRange, cutPrefix bool
|
|||
}
|
||||
}
|
||||
ps := s.ps
|
||||
s.mut.RUnlock()
|
||||
|
||||
s.runlock()
|
||||
less := func(k1, k2 []byte) bool {
|
||||
res := bytes.Compare(k1, k2)
|
||||
return res != 0 && rng.Backwards == (res > 0)
|
||||
|
@ -255,11 +330,25 @@ func (s *MemCachedStore) persist(isSync bool) (int, error) {
|
|||
var err error
|
||||
var keys int
|
||||
|
||||
if s.private {
|
||||
keys = len(s.mem) + len(s.stor)
|
||||
if keys == 0 {
|
||||
return 0, nil
|
||||
}
|
||||
err = s.ps.PutChangeSet(s.mem, s.stor)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
s.mem = nil
|
||||
s.stor = nil
|
||||
return keys, nil
|
||||
}
|
||||
|
||||
s.plock.Lock()
|
||||
defer s.plock.Unlock()
|
||||
s.mut.Lock()
|
||||
|
||||
keys = len(s.mem)
|
||||
keys = len(s.mem) + len(s.stor)
|
||||
if keys == 0 {
|
||||
s.mut.Unlock()
|
||||
return 0, nil
|
||||
|
@ -269,14 +358,15 @@ func (s *MemCachedStore) persist(isSync bool) (int, error) {
|
|||
// starts using fresh new maps. This tempstore is only known here and
|
||||
// nothing ever changes it, therefore accesses to it (reads) can go
|
||||
// unprotected while writes are handled by s proper.
|
||||
var tempstore = &MemCachedStore{MemoryStore: MemoryStore{mem: s.mem}, ps: s.ps}
|
||||
var tempstore = &MemCachedStore{MemoryStore: MemoryStore{mem: s.mem, stor: s.stor}, ps: s.ps}
|
||||
s.ps = tempstore
|
||||
s.mem = make(map[string][]byte, len(s.mem))
|
||||
s.stor = make(map[string][]byte, len(s.stor))
|
||||
if !isSync {
|
||||
s.mut.Unlock()
|
||||
}
|
||||
|
||||
err = tempstore.ps.PutChangeSet(tempstore.mem)
|
||||
err = tempstore.ps.PutChangeSet(tempstore.mem, tempstore.stor)
|
||||
|
||||
if !isSync {
|
||||
s.mut.Lock()
|
||||
|
@ -290,10 +380,14 @@ func (s *MemCachedStore) persist(isSync bool) (int, error) {
|
|||
// We're toast. We'll try to still keep proper state, but OOM
|
||||
// killer will get to us eventually.
|
||||
for k := range s.mem {
|
||||
tempstore.put(k, s.mem[k])
|
||||
put(tempstore.mem, k, s.mem[k])
|
||||
}
|
||||
for k := range s.stor {
|
||||
put(tempstore.stor, k, s.stor[k])
|
||||
}
|
||||
s.ps = tempstore.ps
|
||||
s.mem = tempstore.mem
|
||||
s.stor = tempstore.stor
|
||||
}
|
||||
s.mut.Unlock()
|
||||
return keys, err
|
||||
|
|
|
@ -12,6 +12,40 @@ import (
|
|||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func TestMemCachedPutGetDelete(t *testing.T) {
|
||||
ps := NewMemoryStore()
|
||||
s := NewMemCachedStore(ps)
|
||||
key := []byte("foo")
|
||||
value := []byte("bar")
|
||||
|
||||
s.Put(key, value)
|
||||
|
||||
result, err := s.Get(key)
|
||||
assert.Nil(t, err)
|
||||
require.Equal(t, value, result)
|
||||
|
||||
s.Delete(key)
|
||||
|
||||
_, err = s.Get(key)
|
||||
assert.NotNil(t, err)
|
||||
assert.Equal(t, err, ErrKeyNotFound)
|
||||
|
||||
// Double delete.
|
||||
s.Delete(key)
|
||||
|
||||
_, err = s.Get(key)
|
||||
assert.NotNil(t, err)
|
||||
assert.Equal(t, err, ErrKeyNotFound)
|
||||
|
||||
// Nonexistent.
|
||||
key = []byte("sparse")
|
||||
s.Delete(key)
|
||||
|
||||
_, err = s.Get(key)
|
||||
assert.NotNil(t, err)
|
||||
assert.Equal(t, err, ErrKeyNotFound)
|
||||
}
|
||||
|
||||
func testMemCachedStorePersist(t *testing.T, ps Store) {
|
||||
// cached Store
|
||||
ts := NewMemCachedStore(ps)
|
||||
|
@ -20,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)
|
||||
|
@ -33,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"))
|
||||
|
@ -68,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)
|
||||
|
@ -123,11 +156,11 @@ func TestCachedGetFromPersistent(t *testing.T) {
|
|||
ps := NewMemoryStore()
|
||||
ts := NewMemCachedStore(ps)
|
||||
|
||||
assert.NoError(t, ps.Put(key, value))
|
||||
assert.NoError(t, ps.PutChangeSet(map[string][]byte{string(key): value}, nil))
|
||||
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)
|
||||
|
@ -156,15 +189,15 @@ func TestCachedSeek(t *testing.T) {
|
|||
ts = NewMemCachedStore(ps)
|
||||
)
|
||||
for _, v := range lowerKVs {
|
||||
require.NoError(t, ps.Put(v.Key, v.Value))
|
||||
require.NoError(t, ps.PutChangeSet(map[string][]byte{string(v.Key): v.Value}, nil))
|
||||
}
|
||||
for _, v := range deletedKVs {
|
||||
require.NoError(t, ps.Put(v.Key, v.Value))
|
||||
require.NoError(t, ts.Delete(v.Key))
|
||||
require.NoError(t, ps.PutChangeSet(map[string][]byte{string(v.Key): v.Value}, nil))
|
||||
ts.Delete(v.Key)
|
||||
}
|
||||
for _, v := range updatedKVs {
|
||||
require.NoError(t, ps.Put(v.Key, []byte("stub")))
|
||||
require.NoError(t, ts.Put(v.Key, v.Value))
|
||||
require.NoError(t, ps.PutChangeSet(map[string][]byte{string(v.Key): v.Value}, nil))
|
||||
ts.Put(v.Key, v.Value)
|
||||
}
|
||||
foundKVs := make(map[string][]byte)
|
||||
ts.Seek(SeekRange{Prefix: goodPrefix}, func(k, v []byte) bool {
|
||||
|
@ -199,36 +232,38 @@ 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, ps.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, ps.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, ps.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, ps.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, ps.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, ps.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()
|
||||
require.NoError(t, err)
|
||||
|
||||
t.ReportAllocs()
|
||||
t.ResetTimer()
|
||||
|
@ -263,18 +298,10 @@ func newMemCachedStoreForTesting(t testing.TB) Store {
|
|||
return NewMemCachedStore(NewMemoryStore())
|
||||
}
|
||||
|
||||
type BadBatch struct{}
|
||||
|
||||
func (b BadBatch) Delete(k []byte) {}
|
||||
func (b BadBatch) Put(k, v []byte) {}
|
||||
|
||||
type BadStore struct {
|
||||
onPutBatch func()
|
||||
}
|
||||
|
||||
func (b *BadStore) Batch() Batch {
|
||||
return BadBatch{}
|
||||
}
|
||||
func (b *BadStore) Delete(k []byte) error {
|
||||
return nil
|
||||
}
|
||||
|
@ -284,10 +311,7 @@ func (b *BadStore) Get([]byte) ([]byte, error) {
|
|||
func (b *BadStore) Put(k, v []byte) error {
|
||||
return nil
|
||||
}
|
||||
func (b *BadStore) PutBatch(Batch) error {
|
||||
return nil
|
||||
}
|
||||
func (b *BadStore) PutChangeSet(_ map[string][]byte) error {
|
||||
func (b *BadStore) PutChangeSet(_ map[string][]byte, _ map[string][]byte) error {
|
||||
b.onPutBatch()
|
||||
return ErrKeyNotFound
|
||||
}
|
||||
|
@ -310,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)
|
||||
|
@ -331,6 +355,31 @@ func TestMemCachedPersistFailing(t *testing.T) {
|
|||
require.Equal(t, b1, res)
|
||||
}
|
||||
|
||||
func TestPrivateMemCachedPersistFailing(t *testing.T) {
|
||||
var (
|
||||
bs BadStore
|
||||
t1 = []byte("t1")
|
||||
t2 = []byte("t2")
|
||||
)
|
||||
// cached Store
|
||||
ts := NewPrivateMemCachedStore(&bs)
|
||||
// Set a pair of keys.
|
||||
ts.Put(t1, t1)
|
||||
ts.Put(t2, t2)
|
||||
// This will be called during Persist().
|
||||
bs.onPutBatch = func() {}
|
||||
|
||||
_, err := ts.Persist()
|
||||
require.Error(t, err)
|
||||
// PutBatch() failed in Persist, but we still should have proper state.
|
||||
res, err := ts.Get(t1)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, t1, res)
|
||||
res, err = ts.Get(t2)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, t2, res)
|
||||
}
|
||||
|
||||
func TestCachedSeekSorting(t *testing.T) {
|
||||
var (
|
||||
// Given this prefix...
|
||||
|
@ -354,29 +403,31 @@ func TestCachedSeekSorting(t *testing.T) {
|
|||
{[]byte{1, 3, 2}, []byte("wop")},
|
||||
{[]byte{1, 3, 4}, []byte("zaq")},
|
||||
}
|
||||
ps = NewMemoryStore()
|
||||
ts = NewMemCachedStore(ps)
|
||||
)
|
||||
for _, v := range lowerKVs {
|
||||
require.NoError(t, ps.Put(v.Key, v.Value))
|
||||
for _, newCached := range []func(Store) *MemCachedStore{NewMemCachedStore, NewPrivateMemCachedStore} {
|
||||
ps := NewMemoryStore()
|
||||
ts := newCached(ps)
|
||||
for _, v := range lowerKVs {
|
||||
require.NoError(t, ps.PutChangeSet(map[string][]byte{string(v.Key): v.Value}, nil))
|
||||
}
|
||||
for _, v := range deletedKVs {
|
||||
require.NoError(t, ps.PutChangeSet(map[string][]byte{string(v.Key): v.Value}, nil))
|
||||
ts.Delete(v.Key)
|
||||
}
|
||||
for _, v := range updatedKVs {
|
||||
require.NoError(t, ps.PutChangeSet(map[string][]byte{string(v.Key): v.Value}, nil))
|
||||
ts.Put(v.Key, v.Value)
|
||||
}
|
||||
var foundKVs []KeyValue
|
||||
ts.Seek(SeekRange{Prefix: goodPrefix}, func(k, v []byte) bool {
|
||||
foundKVs = append(foundKVs, KeyValue{Key: slice.Copy(k), Value: slice.Copy(v)})
|
||||
return true
|
||||
})
|
||||
assert.Equal(t, len(foundKVs), len(lowerKVs)+len(updatedKVs))
|
||||
expected := append(lowerKVs, updatedKVs...)
|
||||
sort.Slice(expected, func(i, j int) bool {
|
||||
return bytes.Compare(expected[i].Key, expected[j].Key) < 0
|
||||
})
|
||||
require.Equal(t, expected, foundKVs)
|
||||
}
|
||||
for _, v := range deletedKVs {
|
||||
require.NoError(t, ps.Put(v.Key, v.Value))
|
||||
require.NoError(t, ts.Delete(v.Key))
|
||||
}
|
||||
for _, v := range updatedKVs {
|
||||
require.NoError(t, ps.Put(v.Key, []byte("stub")))
|
||||
require.NoError(t, ts.Put(v.Key, v.Value))
|
||||
}
|
||||
var foundKVs []KeyValue
|
||||
ts.Seek(SeekRange{Prefix: goodPrefix}, func(k, v []byte) bool {
|
||||
foundKVs = append(foundKVs, KeyValue{Key: slice.Copy(k), Value: slice.Copy(v)})
|
||||
return true
|
||||
})
|
||||
assert.Equal(t, len(foundKVs), len(lowerKVs)+len(updatedKVs))
|
||||
expected := append(lowerKVs, updatedKVs...)
|
||||
sort.Slice(expected, func(i, j int) bool {
|
||||
return bytes.Compare(expected[i].Key, expected[j].Key) < 0
|
||||
})
|
||||
require.Equal(t, expected, foundKVs)
|
||||
}
|
||||
|
|
|
@ -5,36 +5,21 @@ import (
|
|||
"sort"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"github.com/nspcc-dev/neo-go/pkg/util/slice"
|
||||
)
|
||||
|
||||
// MemoryStore is an in-memory implementation of a Store, mainly
|
||||
// used for testing. Do not use MemoryStore in production.
|
||||
type MemoryStore struct {
|
||||
mut sync.RWMutex
|
||||
mem map[string][]byte
|
||||
}
|
||||
|
||||
// MemoryBatch is an in-memory batch compatible with MemoryStore.
|
||||
type MemoryBatch struct {
|
||||
MemoryStore
|
||||
}
|
||||
|
||||
// Put implements the Batch interface.
|
||||
func (b *MemoryBatch) Put(k, v []byte) {
|
||||
b.MemoryStore.put(string(k), slice.Copy(v))
|
||||
}
|
||||
|
||||
// Delete implements Batch interface.
|
||||
func (b *MemoryBatch) Delete(k []byte) {
|
||||
b.MemoryStore.drop(string(k))
|
||||
mut sync.RWMutex
|
||||
mem map[string][]byte
|
||||
stor map[string][]byte
|
||||
}
|
||||
|
||||
// NewMemoryStore creates a new MemoryStore object.
|
||||
func NewMemoryStore() *MemoryStore {
|
||||
return &MemoryStore{
|
||||
mem: make(map[string][]byte),
|
||||
mem: make(map[string][]byte),
|
||||
stor: make(map[string][]byte),
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -42,59 +27,45 @@ func NewMemoryStore() *MemoryStore {
|
|||
func (s *MemoryStore) Get(key []byte) ([]byte, error) {
|
||||
s.mut.RLock()
|
||||
defer s.mut.RUnlock()
|
||||
if val, ok := s.mem[string(key)]; ok && val != nil {
|
||||
m := s.chooseMap(key)
|
||||
if val, ok := m[string(key)]; ok && val != nil {
|
||||
return val, nil
|
||||
}
|
||||
return nil, ErrKeyNotFound
|
||||
}
|
||||
|
||||
func (s *MemoryStore) chooseMap(key []byte) map[string][]byte {
|
||||
switch KeyPrefix(key[0]) {
|
||||
case STStorage, STTempStorage:
|
||||
return s.stor
|
||||
default:
|
||||
return s.mem
|
||||
}
|
||||
}
|
||||
|
||||
// put puts a key-value pair into the store, it's supposed to be called
|
||||
// with mutex locked.
|
||||
func (s *MemoryStore) put(key string, value []byte) {
|
||||
s.mem[key] = value
|
||||
}
|
||||
|
||||
// Put implements the Store interface. Never returns an error.
|
||||
func (s *MemoryStore) Put(key, value []byte) error {
|
||||
newKey := string(key)
|
||||
vcopy := slice.Copy(value)
|
||||
s.mut.Lock()
|
||||
s.put(newKey, vcopy)
|
||||
s.mut.Unlock()
|
||||
return nil
|
||||
}
|
||||
|
||||
// drop deletes a key-value pair from the store, it's supposed to be called
|
||||
// with mutex locked.
|
||||
func (s *MemoryStore) drop(key string) {
|
||||
s.mem[key] = nil
|
||||
}
|
||||
|
||||
// Delete implements Store interface. Never returns an error.
|
||||
func (s *MemoryStore) Delete(key []byte) error {
|
||||
newKey := string(key)
|
||||
s.mut.Lock()
|
||||
s.drop(newKey)
|
||||
s.mut.Unlock()
|
||||
return nil
|
||||
}
|
||||
|
||||
// PutBatch implements the Store interface. Never returns an error.
|
||||
func (s *MemoryStore) PutBatch(batch Batch) error {
|
||||
b := batch.(*MemoryBatch)
|
||||
return s.PutChangeSet(b.mem)
|
||||
func put(m map[string][]byte, key string, value []byte) {
|
||||
m[key] = value
|
||||
}
|
||||
|
||||
// PutChangeSet implements the Store interface. Never returns an error.
|
||||
func (s *MemoryStore) PutChangeSet(puts map[string][]byte) error {
|
||||
func (s *MemoryStore) PutChangeSet(puts map[string][]byte, stores map[string][]byte) error {
|
||||
s.mut.Lock()
|
||||
for k := range puts {
|
||||
s.put(k, puts[k])
|
||||
}
|
||||
s.putChangeSet(puts, stores)
|
||||
s.mut.Unlock()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *MemoryStore) putChangeSet(puts map[string][]byte, stores map[string][]byte) {
|
||||
for k := range puts {
|
||||
put(s.mem, k, puts[k])
|
||||
}
|
||||
for k := range stores {
|
||||
put(s.stor, k, stores[k])
|
||||
}
|
||||
}
|
||||
|
||||
// Seek implements the Store interface.
|
||||
func (s *MemoryStore) Seek(rng SeekRange, f func(k, v []byte) bool) {
|
||||
s.mut.RLock()
|
||||
|
@ -109,7 +80,7 @@ func (s *MemoryStore) SeekGC(rng SeekRange, keep func(k, v []byte) bool) error {
|
|||
// sensitive to the order of KV pairs.
|
||||
s.seek(rng, func(k, v []byte) bool {
|
||||
if !keep(k, v) {
|
||||
s.drop(string(k))
|
||||
delete(s.chooseMap(k), string(k))
|
||||
}
|
||||
return true
|
||||
})
|
||||
|
@ -117,18 +88,6 @@ func (s *MemoryStore) SeekGC(rng SeekRange, keep func(k, v []byte) bool) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
// SeekAll is like seek but also iterates over deleted items.
|
||||
func (s *MemoryStore) SeekAll(key []byte, f func(k, v []byte)) {
|
||||
s.mut.RLock()
|
||||
defer s.mut.RUnlock()
|
||||
sk := string(key)
|
||||
for k, v := range s.mem {
|
||||
if strings.HasPrefix(k, sk) {
|
||||
f([]byte(k), v)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// seek is an internal unlocked implementation of Seek. `start` denotes whether
|
||||
// seeking starting from the provided prefix should be performed. Backwards
|
||||
// seeking from some point is supported with corresponding SeekRange field set.
|
||||
|
@ -152,7 +111,8 @@ func (s *MemoryStore) seek(rng SeekRange, f func(k, v []byte) bool) {
|
|||
return res != 0 && rng.Backwards == (res > 0)
|
||||
}
|
||||
|
||||
for k, v := range s.mem {
|
||||
m := s.chooseMap(rng.Prefix)
|
||||
for k, v := range m {
|
||||
if v != nil && isKeyOK(k) {
|
||||
memList = append(memList, KeyValue{
|
||||
Key: []byte(k),
|
||||
|
@ -170,21 +130,12 @@ func (s *MemoryStore) seek(rng SeekRange, f func(k, v []byte) bool) {
|
|||
}
|
||||
}
|
||||
|
||||
// Batch implements the Batch interface and returns a compatible Batch.
|
||||
func (s *MemoryStore) Batch() Batch {
|
||||
return newMemoryBatch()
|
||||
}
|
||||
|
||||
// newMemoryBatch returns new memory batch.
|
||||
func newMemoryBatch() *MemoryBatch {
|
||||
return &MemoryBatch{MemoryStore: *NewMemoryStore()}
|
||||
}
|
||||
|
||||
// Close implements Store interface and clears up memory. Never returns an
|
||||
// error.
|
||||
func (s *MemoryStore) Close() error {
|
||||
s.mut.Lock()
|
||||
s.mem = nil
|
||||
s.stor = nil
|
||||
s.mut.Unlock()
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -20,10 +20,13 @@ func BenchmarkMemorySeek(t *testing.B) {
|
|||
searchPrefix = []byte{1}
|
||||
badPrefix = []byte{2}
|
||||
)
|
||||
ts := NewMemCachedStore(ms)
|
||||
for i := 0; i < count; i++ {
|
||||
require.NoError(t, ms.Put(append(searchPrefix, random.Bytes(10)...), random.Bytes(10)))
|
||||
require.NoError(t, ms.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)
|
||||
|
||||
t.ReportAllocs()
|
||||
t.ResetTimer()
|
||||
|
|
|
@ -61,8 +61,7 @@ type Operation struct {
|
|||
// SeekRange represents options for Store.Seek operation.
|
||||
type SeekRange struct {
|
||||
// Prefix denotes the Seek's lookup key.
|
||||
// Empty Prefix means seeking through all keys in the DB starting from
|
||||
// the Start if specified.
|
||||
// Empty Prefix is not supported.
|
||||
Prefix []byte
|
||||
// Start denotes value appended to the Prefix to start Seek from.
|
||||
// Seeking starting from some key includes this key to the result;
|
||||
|
@ -83,16 +82,13 @@ type SeekRange struct {
|
|||
var ErrKeyNotFound = errors.New("key not found")
|
||||
|
||||
type (
|
||||
// Store is anything that can persist and retrieve the blockchain.
|
||||
// information.
|
||||
// Store is the underlying KV backend for the blockchain data, it's
|
||||
// not intended to be used directly, you wrap it with some memory cache
|
||||
// layer most of the time.
|
||||
Store interface {
|
||||
Batch() Batch
|
||||
Delete(k []byte) error
|
||||
Get([]byte) ([]byte, error)
|
||||
Put(k, v []byte) error
|
||||
PutBatch(Batch) error
|
||||
// PutChangeSet allows to push prepared changeset to the Store.
|
||||
PutChangeSet(puts map[string][]byte) error
|
||||
PutChangeSet(puts map[string][]byte, stor map[string][]byte) error
|
||||
// Seek can guarantee that provided key (k) and value (v) are the only valid until the next call to f.
|
||||
// Seek continues iteration until false is returned from f.
|
||||
// Key and value slices should not be modified.
|
||||
|
@ -106,15 +102,6 @@ type (
|
|||
Close() error
|
||||
}
|
||||
|
||||
// Batch represents an abstraction on top of batch operations.
|
||||
// Each Store implementation is responsible of casting a Batch
|
||||
// to its appropriate type. Batches can only be used in a single
|
||||
// thread.
|
||||
Batch interface {
|
||||
Delete(k []byte)
|
||||
Put(k, v []byte)
|
||||
}
|
||||
|
||||
// KeyPrefix is a constant byte added as a prefix for each key
|
||||
// stored.
|
||||
KeyPrefix uint8
|
||||
|
|
|
@ -19,17 +19,6 @@ type dbSetup struct {
|
|||
|
||||
type dbTestFunction func(*testing.T, Store)
|
||||
|
||||
func testStorePutAndGet(t *testing.T, s Store) {
|
||||
key := []byte("foo")
|
||||
value := []byte("bar")
|
||||
|
||||
require.NoError(t, s.Put(key, value))
|
||||
|
||||
result, err := s.Get(key)
|
||||
assert.Nil(t, err)
|
||||
require.Equal(t, value, result)
|
||||
}
|
||||
|
||||
func testStoreGetNonExistent(t *testing.T, s Store) {
|
||||
key := []byte("sparse")
|
||||
|
||||
|
@ -37,28 +26,7 @@ func testStoreGetNonExistent(t *testing.T, s Store) {
|
|||
assert.Equal(t, err, ErrKeyNotFound)
|
||||
}
|
||||
|
||||
func testStorePutBatch(t *testing.T, s Store) {
|
||||
var (
|
||||
key = []byte("foo")
|
||||
value = []byte("bar")
|
||||
batch = s.Batch()
|
||||
)
|
||||
// Test that key and value are copied when batching.
|
||||
keycopy := slice.Copy(key)
|
||||
valuecopy := slice.Copy(value)
|
||||
|
||||
batch.Put(keycopy, valuecopy)
|
||||
copy(valuecopy, key)
|
||||
copy(keycopy, value)
|
||||
|
||||
require.NoError(t, s.PutBatch(batch))
|
||||
newVal, err := s.Get(key)
|
||||
assert.Nil(t, err)
|
||||
require.Equal(t, value, newVal)
|
||||
assert.Equal(t, value, newVal)
|
||||
}
|
||||
|
||||
func testStoreSeek(t *testing.T, s Store) {
|
||||
func pushSeekDataSet(t *testing.T, s Store) []KeyValue {
|
||||
// Use the same set of kvs to test Seek with different prefix/start values.
|
||||
kvs := []KeyValue{
|
||||
{[]byte("10"), []byte("bar")},
|
||||
|
@ -69,10 +37,17 @@ func testStoreSeek(t *testing.T, s Store) {
|
|||
{[]byte("30"), []byte("bare")},
|
||||
{[]byte("31"), []byte("barf")},
|
||||
}
|
||||
up := NewMemCachedStore(s)
|
||||
for _, v := range kvs {
|
||||
require.NoError(t, s.Put(v.Key, v.Value))
|
||||
up.Put(v.Key, v.Value)
|
||||
}
|
||||
_, err := up.PersistSync()
|
||||
require.NoError(t, err)
|
||||
return kvs
|
||||
}
|
||||
|
||||
func testStoreSeek(t *testing.T, s Store) {
|
||||
kvs := pushSeekDataSet(t, s)
|
||||
check := func(t *testing.T, goodprefix, start []byte, goodkvs []KeyValue, backwards bool, cont func(k, v []byte) bool) {
|
||||
// Seek result expected to be sorted in an ascending (for forwards seeking) or descending (for backwards seeking) way.
|
||||
cmpFunc := func(i, j int) bool {
|
||||
|
@ -228,215 +203,10 @@ func testStoreSeek(t *testing.T, s Store) {
|
|||
})
|
||||
})
|
||||
})
|
||||
|
||||
t.Run("empty prefix, non-empty start", func(t *testing.T) {
|
||||
t.Run("forwards", func(t *testing.T) {
|
||||
t.Run("good", func(t *testing.T) {
|
||||
goodprefix := []byte{}
|
||||
start := []byte("21")
|
||||
goodkvs := []KeyValue{
|
||||
kvs[3], // key = "21"
|
||||
kvs[4], // key = "22"
|
||||
kvs[5], // key = "30"
|
||||
kvs[6], // key = "31"
|
||||
}
|
||||
check(t, goodprefix, start, goodkvs, false, nil)
|
||||
})
|
||||
t.Run("no matching items", func(t *testing.T) {
|
||||
goodprefix := []byte{}
|
||||
start := []byte("32") // start is more than all keys.
|
||||
check(t, goodprefix, start, []KeyValue{}, false, nil)
|
||||
})
|
||||
t.Run("early stop", func(t *testing.T) {
|
||||
goodprefix := []byte{}
|
||||
start := []byte("21")
|
||||
goodkvs := []KeyValue{
|
||||
kvs[3], // key = "21"
|
||||
kvs[4], // key = "22"
|
||||
kvs[5], // key = "30"
|
||||
}
|
||||
check(t, goodprefix, start, goodkvs, false, func(k, v []byte) bool {
|
||||
return string(k) < "30"
|
||||
})
|
||||
})
|
||||
})
|
||||
t.Run("backwards", func(t *testing.T) {
|
||||
t.Run("good", func(t *testing.T) {
|
||||
goodprefix := []byte{}
|
||||
start := []byte("21")
|
||||
goodkvs := []KeyValue{
|
||||
kvs[3], // key = "21"
|
||||
kvs[2], // key = "20"
|
||||
kvs[1], // key = "11"
|
||||
kvs[0], // key = "10"
|
||||
}
|
||||
check(t, goodprefix, start, goodkvs, true, nil)
|
||||
})
|
||||
t.Run("no matching items", func(t *testing.T) {
|
||||
goodprefix := []byte{}
|
||||
start := []byte("0") // start is less than all keys.
|
||||
check(t, goodprefix, start, []KeyValue{}, true, nil)
|
||||
})
|
||||
t.Run("early stop", func(t *testing.T) {
|
||||
goodprefix := []byte{}
|
||||
start := []byte("21")
|
||||
goodkvs := []KeyValue{
|
||||
kvs[3], // key = "21"
|
||||
kvs[2], // key = "20"
|
||||
kvs[1], // key = "11"
|
||||
}
|
||||
check(t, goodprefix, start, goodkvs, true, func(k, v []byte) bool {
|
||||
return string(k) > "11"
|
||||
})
|
||||
})
|
||||
})
|
||||
})
|
||||
|
||||
t.Run("empty prefix, empty start", func(t *testing.T) {
|
||||
goodprefix := []byte{}
|
||||
start := []byte{}
|
||||
goodkvs := make([]KeyValue, len(kvs))
|
||||
copy(goodkvs, kvs)
|
||||
t.Run("forwards", func(t *testing.T) {
|
||||
t.Run("good", func(t *testing.T) {
|
||||
check(t, goodprefix, start, goodkvs, false, nil)
|
||||
})
|
||||
t.Run("early stop", func(t *testing.T) {
|
||||
goodkvs := []KeyValue{
|
||||
kvs[0], // key = "10"
|
||||
kvs[1], // key = "11"
|
||||
kvs[2], // key = "20"
|
||||
kvs[3], // key = "21"
|
||||
}
|
||||
check(t, goodprefix, start, goodkvs, false, func(k, v []byte) bool {
|
||||
return string(k) < "21"
|
||||
})
|
||||
})
|
||||
})
|
||||
t.Run("backwards", func(t *testing.T) {
|
||||
t.Run("good", func(t *testing.T) {
|
||||
check(t, goodprefix, start, goodkvs, true, nil)
|
||||
})
|
||||
t.Run("early stop", func(t *testing.T) {
|
||||
goodkvs := []KeyValue{
|
||||
kvs[6], // key = "31"
|
||||
kvs[5], // key = "30"
|
||||
kvs[4], // key = "22"
|
||||
kvs[3], // key = "21"
|
||||
}
|
||||
check(t, goodprefix, start, goodkvs, true, func(k, v []byte) bool {
|
||||
return string(k) > "21"
|
||||
})
|
||||
})
|
||||
})
|
||||
})
|
||||
}
|
||||
|
||||
func testStoreDeleteNonExistent(t *testing.T, s Store) {
|
||||
key := []byte("sparse")
|
||||
|
||||
assert.NoError(t, s.Delete(key))
|
||||
}
|
||||
|
||||
func testStorePutAndDelete(t *testing.T, s Store) {
|
||||
key := []byte("foo")
|
||||
value := []byte("bar")
|
||||
|
||||
require.NoError(t, s.Put(key, value))
|
||||
|
||||
err := s.Delete(key)
|
||||
assert.Nil(t, err)
|
||||
|
||||
_, err = s.Get(key)
|
||||
assert.NotNil(t, err)
|
||||
assert.Equal(t, err, ErrKeyNotFound)
|
||||
|
||||
// Double delete.
|
||||
err = s.Delete(key)
|
||||
assert.Nil(t, err)
|
||||
}
|
||||
|
||||
func testStorePutBatchWithDelete(t *testing.T, s Store) {
|
||||
var (
|
||||
toBeStored = map[string][]byte{
|
||||
"foo": []byte("bar"),
|
||||
"bar": []byte("baz"),
|
||||
}
|
||||
deletedInBatch = map[string][]byte{
|
||||
"edc": []byte("rfv"),
|
||||
"tgb": []byte("yhn"),
|
||||
}
|
||||
readdedToBatch = map[string][]byte{
|
||||
"yhn": []byte("ujm"),
|
||||
}
|
||||
toBeDeleted = map[string][]byte{
|
||||
"qaz": []byte("wsx"),
|
||||
"qwe": []byte("123"),
|
||||
}
|
||||
toStay = map[string][]byte{
|
||||
"key": []byte("val"),
|
||||
"faa": []byte("bra"),
|
||||
}
|
||||
)
|
||||
for k, v := range toBeDeleted {
|
||||
require.NoError(t, s.Put([]byte(k), v))
|
||||
}
|
||||
for k, v := range toStay {
|
||||
require.NoError(t, s.Put([]byte(k), v))
|
||||
}
|
||||
batch := s.Batch()
|
||||
for k, v := range toBeStored {
|
||||
batch.Put([]byte(k), v)
|
||||
}
|
||||
for k := range toBeDeleted {
|
||||
batch.Delete([]byte(k))
|
||||
}
|
||||
for k, v := range readdedToBatch {
|
||||
batch.Put([]byte(k), v)
|
||||
}
|
||||
for k, v := range deletedInBatch {
|
||||
batch.Put([]byte(k), v)
|
||||
}
|
||||
for k := range deletedInBatch {
|
||||
batch.Delete([]byte(k))
|
||||
}
|
||||
for k := range readdedToBatch {
|
||||
batch.Delete([]byte(k))
|
||||
}
|
||||
for k, v := range readdedToBatch {
|
||||
batch.Put([]byte(k), v)
|
||||
}
|
||||
require.NoError(t, s.PutBatch(batch))
|
||||
toBe := []map[string][]byte{toStay, toBeStored, readdedToBatch}
|
||||
notToBe := []map[string][]byte{deletedInBatch, toBeDeleted}
|
||||
for _, kvs := range toBe {
|
||||
for k, v := range kvs {
|
||||
value, err := s.Get([]byte(k))
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, value, v)
|
||||
}
|
||||
}
|
||||
for _, kvs := range notToBe {
|
||||
for k, v := range kvs {
|
||||
_, err := s.Get([]byte(k))
|
||||
assert.Equal(t, ErrKeyNotFound, err, "%s:%s", k, v)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func testStoreSeekGC(t *testing.T, s Store) {
|
||||
kvs := []KeyValue{
|
||||
{[]byte("10"), []byte("bar")},
|
||||
{[]byte("11"), []byte("bara")},
|
||||
{[]byte("20"), []byte("barb")},
|
||||
{[]byte("21"), []byte("barc")},
|
||||
{[]byte("22"), []byte("bard")},
|
||||
{[]byte("30"), []byte("bare")},
|
||||
{[]byte("31"), []byte("barf")},
|
||||
}
|
||||
for _, v := range kvs {
|
||||
require.NoError(t, s.Put(v.Key, v.Value))
|
||||
}
|
||||
kvs := pushSeekDataSet(t, s)
|
||||
err := s.SeekGC(SeekRange{Prefix: []byte("1")}, func(k, v []byte) bool {
|
||||
return true
|
||||
})
|
||||
|
@ -466,10 +236,8 @@ func TestAllDBs(t *testing.T) {
|
|||
{"MemCached", newMemCachedStoreForTesting},
|
||||
{"Memory", newMemoryStoreForTesting},
|
||||
}
|
||||
var tests = []dbTestFunction{testStorePutAndGet,
|
||||
testStoreGetNonExistent, testStorePutBatch, testStoreSeek,
|
||||
testStoreDeleteNonExistent, testStorePutAndDelete,
|
||||
testStorePutBatchWithDelete, testStoreSeekGC}
|
||||
var tests = []dbTestFunction{testStoreGetNonExistent, testStoreSeek,
|
||||
testStoreSeekGC}
|
||||
for _, db := range DBs {
|
||||
for _, test := range tests {
|
||||
s := db.create(t)
|
||||
|
|
Loading…
Reference in a new issue