Merge pull request #832 from nspcc-dev/refactoring/core

core: split into several packages
This commit is contained in:
Roman Khimov 2020-04-08 09:13:47 +03:00 committed by GitHub
commit 60b795f3ac
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
13 changed files with 255 additions and 252 deletions

View file

@ -11,6 +11,7 @@ import (
"github.com/nspcc-dev/neo-go/pkg/config" "github.com/nspcc-dev/neo-go/pkg/config"
"github.com/nspcc-dev/neo-go/pkg/core/block" "github.com/nspcc-dev/neo-go/pkg/core/block"
"github.com/nspcc-dev/neo-go/pkg/core/dao"
"github.com/nspcc-dev/neo-go/pkg/core/mempool" "github.com/nspcc-dev/neo-go/pkg/core/mempool"
"github.com/nspcc-dev/neo-go/pkg/core/state" "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/storage"
@ -80,7 +81,7 @@ type Blockchain struct {
lock sync.RWMutex lock sync.RWMutex
// Data access object for CRUD operations around storage. // Data access object for CRUD operations around storage.
dao *simpleDao dao *dao.Simple
// Current index/height of the highest block. // Current index/height of the highest block.
// Read access should always be called by BlockHeight(). // Read access should always be called by BlockHeight().
@ -154,7 +155,7 @@ func NewBlockchain(s storage.Store, cfg config.ProtocolConfiguration, log *zap.L
} }
bc := &Blockchain{ bc := &Blockchain{
config: cfg, config: cfg,
dao: newSimpleDao(s), dao: dao.NewSimple(s),
headersOp: make(chan headersOpFunc), headersOp: make(chan headersOpFunc),
headersOpDone: make(chan struct{}), headersOpDone: make(chan struct{}),
stopCh: make(chan struct{}), stopCh: make(chan struct{}),
@ -271,7 +272,7 @@ func (bc *Blockchain) Run() {
if err := bc.persist(); err != nil { if err := bc.persist(); err != nil {
bc.log.Warn("failed to persist", zap.Error(err)) bc.log.Warn("failed to persist", zap.Error(err))
} }
if err := bc.dao.store.Close(); err != nil { if err := bc.dao.Store.Close(); err != nil {
bc.log.Warn("failed to close db", zap.Error(err)) bc.log.Warn("failed to close db", zap.Error(err))
} }
close(bc.runToExitCh) close(bc.runToExitCh)
@ -348,7 +349,7 @@ func (bc *Blockchain) AddHeaders(headers ...*block.Header) error {
func (bc *Blockchain) addHeaders(verify bool, headers ...*block.Header) (err error) { func (bc *Blockchain) addHeaders(verify bool, headers ...*block.Header) (err error) {
var ( var (
start = time.Now() start = time.Now()
batch = bc.dao.store.Batch() batch = bc.dao.Store.Batch()
) )
if len(headers) > 0 { if len(headers) > 0 {
@ -402,7 +403,7 @@ func (bc *Blockchain) addHeaders(verify bool, headers ...*block.Header) (err err
if oldlen != headerList.Len() { if oldlen != headerList.Len() {
updateHeaderHeightMetric(headerList.Len() - 1) updateHeaderHeightMetric(headerList.Len() - 1)
if err = bc.dao.store.PutBatch(batch); err != nil { if err = bc.dao.Store.PutBatch(batch); err != nil {
return return
} }
bc.log.Debug("done processing headers", bc.log.Debug("done processing headers",
@ -457,7 +458,7 @@ func (bc *Blockchain) getSystemFeeAmount(h util.Uint256) uint32 {
// is happening here, quite allot as you can see :). If things are wired together // is happening here, quite allot as you can see :). If things are wired together
// and all tests are in place, we can make a more optimized and cleaner implementation. // and all tests are in place, we can make a more optimized and cleaner implementation.
func (bc *Blockchain) storeBlock(block *block.Block) error { func (bc *Blockchain) storeBlock(block *block.Block) error {
cache := newCachedDao(bc.dao) cache := dao.NewCached(bc.dao)
fee := bc.getSystemFeeAmount(block.PrevHash) fee := bc.getSystemFeeAmount(block.PrevHash)
for _, tx := range block.Transactions { for _, tx := range block.Transactions {
fee += uint32(bc.SystemFee(tx).IntegralValue()) fee += uint32(bc.SystemFee(tx).IntegralValue())
@ -656,7 +657,7 @@ func (bc *Blockchain) storeBlock(block *block.Block) error {
} }
case *transaction.InvocationTX: case *transaction.InvocationTX:
systemInterop := bc.newInteropContext(trigger.Application, cache, block, tx) systemInterop := bc.newInteropContext(trigger.Application, cache, block, tx)
v := bc.spawnVMWithInterops(systemInterop) v := systemInterop.SpawnVM()
v.SetCheckedHash(tx.VerificationHash().BytesBE()) v.SetCheckedHash(tx.VerificationHash().BytesBE())
v.LoadScript(t.Script) v.LoadScript(t.Script)
v.SetPriceGetter(getPrice) v.SetPriceGetter(getPrice)
@ -713,7 +714,7 @@ func (bc *Blockchain) storeBlock(block *block.Block) error {
} }
err = cache.PutAppExecResult(aer) err = cache.PutAppExecResult(aer)
if err != nil { if err != nil {
return errors.Wrap(err, "failed to store notifications") return errors.Wrap(err, "failed to Store notifications")
} }
} }
} }
@ -721,7 +722,7 @@ func (bc *Blockchain) storeBlock(block *block.Block) error {
defer bc.lock.Unlock() defer bc.lock.Unlock()
if bc.config.SaveStorageBatch { if bc.config.SaveStorageBatch {
bc.lastBatch = cache.dao.GetBatch() bc.lastBatch = cache.DAO.GetBatch()
} }
_, err := cache.Persist() _, err := cache.Persist()
@ -742,7 +743,7 @@ func parseUint160(addr []byte) util.Uint160 {
return util.Uint160{} return util.Uint160{}
} }
func (bc *Blockchain) processNEP5Transfer(cache *cachedDao, tx *transaction.Transaction, b *block.Block, sc util.Uint160, from, to []byte, amount int64) { func (bc *Blockchain) processNEP5Transfer(cache *dao.Cached, tx *transaction.Transaction, b *block.Block, sc util.Uint160, from, to []byte, amount int64) {
toAddr := parseUint160(to) toAddr := parseUint160(to)
fromAddr := parseUint160(from) fromAddr := parseUint160(from)
transfer := &state.NEP5Transfer{ transfer := &state.NEP5Transfer{
@ -831,7 +832,7 @@ func (bc *Blockchain) LastBatch() *storage.MemBatch {
} }
// processOutputs processes transaction outputs. // processOutputs processes transaction outputs.
func processOutputs(tx *transaction.Transaction, dao *cachedDao) error { func processOutputs(tx *transaction.Transaction, dao *dao.Cached) error {
for index, output := range tx.Outputs { for index, output := range tx.Outputs {
account, err := dao.GetAccountStateOrNew(output.ScriptHash) account, err := dao.GetAccountStateOrNew(output.ScriptHash)
if err != nil { if err != nil {
@ -852,14 +853,14 @@ func processOutputs(tx *transaction.Transaction, dao *cachedDao) error {
return nil return nil
} }
func processTXWithValidatorsAdd(output *transaction.Output, account *state.Account, dao *cachedDao) error { func processTXWithValidatorsAdd(output *transaction.Output, account *state.Account, dao *dao.Cached) error {
if output.AssetID.Equals(GoverningTokenID()) && len(account.Votes) > 0 { if output.AssetID.Equals(GoverningTokenID()) && len(account.Votes) > 0 {
return modAccountVotes(account, dao, output.Amount) return modAccountVotes(account, dao, output.Amount)
} }
return nil return nil
} }
func processTXWithValidatorsSubtract(output *transaction.Output, account *state.Account, dao *cachedDao) error { func processTXWithValidatorsSubtract(output *transaction.Output, account *state.Account, dao *dao.Cached) error {
if output.AssetID.Equals(GoverningTokenID()) && len(account.Votes) > 0 { if output.AssetID.Equals(GoverningTokenID()) && len(account.Votes) > 0 {
return modAccountVotes(account, dao, -output.Amount) return modAccountVotes(account, dao, -output.Amount)
} }
@ -867,7 +868,7 @@ func processTXWithValidatorsSubtract(output *transaction.Output, account *state.
} }
// modAccountVotes adds given value to given account voted validators. // modAccountVotes adds given value to given account voted validators.
func modAccountVotes(account *state.Account, dao *cachedDao, value util.Fixed8) error { func modAccountVotes(account *state.Account, dao *dao.Cached, value util.Fixed8) error {
for _, vote := range account.Votes { for _, vote := range account.Votes {
validator, err := dao.GetValidatorStateOrNew(vote) validator, err := dao.GetValidatorStateOrNew(vote)
if err != nil { if err != nil {
@ -898,7 +899,7 @@ func modAccountVotes(account *state.Account, dao *cachedDao, value util.Fixed8)
return nil return nil
} }
func processValidatorStateDescriptor(descriptor *transaction.StateDescriptor, dao *cachedDao) error { func processValidatorStateDescriptor(descriptor *transaction.StateDescriptor, dao *dao.Cached) error {
publicKey := &keys.PublicKey{} publicKey := &keys.PublicKey{}
err := publicKey.DecodeBytes(descriptor.Key) err := publicKey.DecodeBytes(descriptor.Key)
if err != nil { if err != nil {
@ -918,7 +919,7 @@ func processValidatorStateDescriptor(descriptor *transaction.StateDescriptor, da
return nil return nil
} }
func processAccountStateDescriptor(descriptor *transaction.StateDescriptor, dao *cachedDao) error { func processAccountStateDescriptor(descriptor *transaction.StateDescriptor, dao *dao.Cached) error {
hash, err := util.Uint160DecodeBytesBE(descriptor.Key) hash, err := util.Uint160DecodeBytesBE(descriptor.Key)
if err != nil { if err != nil {
return err return err
@ -971,7 +972,7 @@ func processAccountStateDescriptor(descriptor *transaction.StateDescriptor, dao
return nil return nil
} }
// persist flushes current in-memory store contents to the persistent storage. // persist flushes current in-memory Store contents to the persistent storage.
func (bc *Blockchain) persist() error { func (bc *Blockchain) persist() error {
var ( var (
start = time.Now() start = time.Now()
@ -1725,7 +1726,7 @@ func (bc *Blockchain) GetStandByValidators() (keys.PublicKeys, error) {
// GetValidators returns validators. // GetValidators returns validators.
// Golang implementation of GetValidators method in C# (https://github.com/neo-project/neo/blob/c64748ecbac3baeb8045b16af0d518398a6ced24/neo/Persistence/Snapshot.cs#L182) // Golang implementation of GetValidators method in C# (https://github.com/neo-project/neo/blob/c64748ecbac3baeb8045b16af0d518398a6ced24/neo/Persistence/Snapshot.cs#L182)
func (bc *Blockchain) GetValidators(txes ...*transaction.Transaction) ([]*keys.PublicKey, error) { func (bc *Blockchain) GetValidators(txes ...*transaction.Transaction) ([]*keys.PublicKey, error) {
cache := newCachedDao(bc.dao) cache := dao.NewCached(bc.dao)
if len(txes) > 0 { if len(txes) > 0 {
for _, tx := range txes { for _, tx := range txes {
// iterate through outputs // iterate through outputs
@ -1873,7 +1874,7 @@ func (bc *Blockchain) GetEnrollments() ([]*state.Validator, error) {
return result, nil return result, nil
} }
func processStateTX(dao *cachedDao, tx *transaction.StateTX) error { func processStateTX(dao *dao.Cached, tx *transaction.StateTX) error {
for _, desc := range tx.Descriptors { for _, desc := range tx.Descriptors {
switch desc.Type { switch desc.Type {
case transaction.Account: case transaction.Account:
@ -1889,7 +1890,7 @@ func processStateTX(dao *cachedDao, tx *transaction.StateTX) error {
return nil return nil
} }
func processEnrollmentTX(dao *cachedDao, tx *transaction.EnrollmentTX) error { func processEnrollmentTX(dao *dao.Cached, tx *transaction.EnrollmentTX) error {
validatorState, err := dao.GetValidatorStateOrNew(&tx.PublicKey) validatorState, err := dao.GetValidatorStateOrNew(&tx.PublicKey)
if err != nil { if err != nil {
return err return err
@ -1998,27 +1999,10 @@ func (bc *Blockchain) GetScriptHashesForVerifying(t *transaction.Transaction) ([
} }
// spawnVMWithInterops returns a VM with script getter and interop functions set
// up for current blockchain.
func (bc *Blockchain) spawnVMWithInterops(interopCtx *interopContext) *vm.VM {
vm := vm.New()
vm.SetScriptGetter(func(hash util.Uint160) ([]byte, bool) {
cs, err := interopCtx.dao.GetContractState(hash)
if err != nil {
return nil, false
}
hasDynamicInvoke := (cs.Properties & smartcontract.HasDynamicInvoke) != 0
return cs.Script, hasDynamicInvoke
})
vm.RegisterInteropGetter(interopCtx.getSystemInterop)
vm.RegisterInteropGetter(interopCtx.getNeoInterop)
return vm
}
// GetTestVM returns a VM and a Store setup for a test run of some sort of code. // GetTestVM returns a VM and a Store setup for a test run of some sort of code.
func (bc *Blockchain) GetTestVM() *vm.VM { func (bc *Blockchain) GetTestVM() *vm.VM {
systemInterop := bc.newInteropContext(trigger.Application, bc.dao, nil, nil) systemInterop := bc.newInteropContext(trigger.Application, bc.dao, nil, nil)
vm := bc.spawnVMWithInterops(systemInterop) vm := systemInterop.SpawnVM()
vm.SetPriceGetter(getPrice) vm.SetPriceGetter(getPrice)
return vm return vm
} }
@ -2046,7 +2030,7 @@ func (bc *Blockchain) verifyHashAgainstScript(hash util.Uint160, witness *transa
return err return err
} }
vm := bc.spawnVMWithInterops(interopCtx) vm := interopCtx.SpawnVM()
vm.SetCheckedHash(checkedHash.BytesBE()) vm.SetCheckedHash(checkedHash.BytesBE())
vm.LoadScript(verification) vm.LoadScript(verification)
vm.LoadScript(witness.InvocationScript) vm.LoadScript(witness.InvocationScript)
@ -2140,6 +2124,6 @@ func (bc *Blockchain) secondsPerBlock() int {
return bc.config.SecondsPerBlock return bc.config.SecondsPerBlock
} }
func (bc *Blockchain) newInteropContext(trigger trigger.Type, d dao, block *block.Block, tx *transaction.Transaction) *interopContext { func (bc *Blockchain) newInteropContext(trigger trigger.Type, d dao.DAO, block *block.Block, tx *transaction.Transaction) *interopContext {
return newInteropContext(trigger, bc, d, block, tx, bc.log) return newInteropContext(trigger, bc, d, block, tx, bc.log)
} }

View file

@ -67,7 +67,7 @@ func TestAddBlock(t *testing.T) {
for _, block := range blocks { for _, block := range blocks {
key := storage.AppendPrefix(storage.DataBlock, block.Hash().BytesLE()) key := storage.AppendPrefix(storage.DataBlock, block.Hash().BytesLE())
_, err := bc.dao.store.Get(key) _, err := bc.dao.Store.Get(key)
require.NoErrorf(t, err, "block %s not persisted", block.Hash()) require.NoErrorf(t, err, "block %s not persisted", block.Hash())
} }
@ -224,7 +224,7 @@ func TestClose(t *testing.T) {
// It's a hack, but we use internal knowledge of MemoryStore // It's a hack, but we use internal knowledge of MemoryStore
// implementation which makes it completely unusable (up to panicing) // implementation which makes it completely unusable (up to panicing)
// after Close(). // after Close().
_ = bc.dao.store.Put([]byte{0}, []byte{1}) _ = bc.dao.Store.Put([]byte{0}, []byte{1})
// This should never be executed. // This should never be executed.
assert.Nil(t, t) assert.Nil(t, t)

View file

@ -1,4 +1,4 @@
package core package dao
import ( import (
"errors" "errors"
@ -8,11 +8,11 @@ import (
"github.com/nspcc-dev/neo-go/pkg/util" "github.com/nspcc-dev/neo-go/pkg/util"
) )
// cachedDao is a data access object that mimics dao, but has a write cache // Cached is a data access object that mimics DAO, but has a write cache
// for accounts and read cache for contracts. These are the most frequently used // for accounts and read cache for contracts. These are the most frequently used
// objects in the storeBlock(). // objects in the storeBlock().
type cachedDao struct { type Cached struct {
dao DAO
accounts map[util.Uint160]*state.Account accounts map[util.Uint160]*state.Account
contracts map[util.Uint160]*state.Contract contracts map[util.Uint160]*state.Contract
unspents map[util.Uint256]*state.UnspentCoin unspents map[util.Uint256]*state.UnspentCoin
@ -20,45 +20,45 @@ type cachedDao struct {
transfers map[util.Uint160]map[uint32]*state.NEP5TransferLog transfers map[util.Uint160]map[uint32]*state.NEP5TransferLog
} }
// newCachedDao returns new cachedDao wrapping around given backing store. // NewCached returns new Cached wrapping around given backing store.
func newCachedDao(d dao) *cachedDao { func NewCached(d DAO) *Cached {
accs := make(map[util.Uint160]*state.Account) accs := make(map[util.Uint160]*state.Account)
ctrs := make(map[util.Uint160]*state.Contract) ctrs := make(map[util.Uint160]*state.Contract)
unspents := make(map[util.Uint256]*state.UnspentCoin) unspents := make(map[util.Uint256]*state.UnspentCoin)
balances := make(map[util.Uint160]*state.NEP5Balances) balances := make(map[util.Uint160]*state.NEP5Balances)
transfers := make(map[util.Uint160]map[uint32]*state.NEP5TransferLog) transfers := make(map[util.Uint160]map[uint32]*state.NEP5TransferLog)
return &cachedDao{d.GetWrapped(), accs, ctrs, unspents, balances, transfers} return &Cached{d.GetWrapped(), accs, ctrs, unspents, balances, transfers}
} }
// GetAccountStateOrNew retrieves Account from cache or underlying Store // GetAccountStateOrNew retrieves Account from cache or underlying store
// or creates a new one if it doesn't exist. // or creates a new one if it doesn't exist.
func (cd *cachedDao) GetAccountStateOrNew(hash util.Uint160) (*state.Account, error) { func (cd *Cached) GetAccountStateOrNew(hash util.Uint160) (*state.Account, error) {
if cd.accounts[hash] != nil { if cd.accounts[hash] != nil {
return cd.accounts[hash], nil return cd.accounts[hash], nil
} }
return cd.dao.GetAccountStateOrNew(hash) return cd.DAO.GetAccountStateOrNew(hash)
} }
// GetAccountState retrieves Account from cache or underlying Store. // GetAccountState retrieves Account from cache or underlying store.
func (cd *cachedDao) GetAccountState(hash util.Uint160) (*state.Account, error) { func (cd *Cached) GetAccountState(hash util.Uint160) (*state.Account, error) {
if cd.accounts[hash] != nil { if cd.accounts[hash] != nil {
return cd.accounts[hash], nil return cd.accounts[hash], nil
} }
return cd.dao.GetAccountState(hash) return cd.DAO.GetAccountState(hash)
} }
// PutAccountState saves given Account in the cache. // PutAccountState saves given Account in the cache.
func (cd *cachedDao) PutAccountState(as *state.Account) error { func (cd *Cached) PutAccountState(as *state.Account) error {
cd.accounts[as.ScriptHash] = as cd.accounts[as.ScriptHash] = as
return nil return nil
} }
// GetContractState returns contract state from cache or underlying Store. // GetContractState returns contract state from cache or underlying store.
func (cd *cachedDao) GetContractState(hash util.Uint160) (*state.Contract, error) { func (cd *Cached) GetContractState(hash util.Uint160) (*state.Contract, error) {
if cd.contracts[hash] != nil { if cd.contracts[hash] != nil {
return cd.contracts[hash], nil return cd.contracts[hash], nil
} }
cs, err := cd.dao.GetContractState(hash) cs, err := cd.DAO.GetContractState(hash)
if err == nil { if err == nil {
cd.contracts[hash] = cs cd.contracts[hash] = cs
} }
@ -66,56 +66,56 @@ func (cd *cachedDao) GetContractState(hash util.Uint160) (*state.Contract, error
} }
// PutContractState puts given contract state into the given store. // PutContractState puts given contract state into the given store.
func (cd *cachedDao) PutContractState(cs *state.Contract) error { func (cd *Cached) PutContractState(cs *state.Contract) error {
cd.contracts[cs.ScriptHash()] = cs cd.contracts[cs.ScriptHash()] = cs
return cd.dao.PutContractState(cs) return cd.DAO.PutContractState(cs)
} }
// DeleteContractState deletes given contract state in cache and backing Store. // DeleteContractState deletes given contract state in cache and backing store.
func (cd *cachedDao) DeleteContractState(hash util.Uint160) error { func (cd *Cached) DeleteContractState(hash util.Uint160) error {
cd.contracts[hash] = nil cd.contracts[hash] = nil
return cd.dao.DeleteContractState(hash) return cd.DAO.DeleteContractState(hash)
} }
// GetUnspentCoinState retrieves UnspentCoin from cache or underlying Store. // GetUnspentCoinState retrieves UnspentCoin from cache or underlying store.
func (cd *cachedDao) GetUnspentCoinState(hash util.Uint256) (*state.UnspentCoin, error) { func (cd *Cached) GetUnspentCoinState(hash util.Uint256) (*state.UnspentCoin, error) {
if cd.unspents[hash] != nil { if cd.unspents[hash] != nil {
return cd.unspents[hash], nil return cd.unspents[hash], nil
} }
return cd.dao.GetUnspentCoinState(hash) return cd.DAO.GetUnspentCoinState(hash)
} }
// PutUnspentCoinState saves given UnspentCoin in the cache. // PutUnspentCoinState saves given UnspentCoin in the cache.
func (cd *cachedDao) PutUnspentCoinState(hash util.Uint256, ucs *state.UnspentCoin) error { func (cd *Cached) PutUnspentCoinState(hash util.Uint256, ucs *state.UnspentCoin) error {
cd.unspents[hash] = ucs cd.unspents[hash] = ucs
return nil return nil
} }
// GetNEP5Balances retrieves NEP5Balances for the acc. // GetNEP5Balances retrieves NEP5Balances for the acc.
func (cd *cachedDao) GetNEP5Balances(acc util.Uint160) (*state.NEP5Balances, error) { func (cd *Cached) GetNEP5Balances(acc util.Uint160) (*state.NEP5Balances, error) {
if bs := cd.balances[acc]; bs != nil { if bs := cd.balances[acc]; bs != nil {
return bs, nil return bs, nil
} }
return cd.dao.GetNEP5Balances(acc) return cd.DAO.GetNEP5Balances(acc)
} }
// PutNEP5Balances saves NEP5Balances for the acc. // PutNEP5Balances saves NEP5Balances for the acc.
func (cd *cachedDao) PutNEP5Balances(acc util.Uint160, bs *state.NEP5Balances) error { func (cd *Cached) PutNEP5Balances(acc util.Uint160, bs *state.NEP5Balances) error {
cd.balances[acc] = bs cd.balances[acc] = bs
return nil return nil
} }
// GetNEP5TransferLog retrieves NEP5TransferLog for the acc. // GetNEP5TransferLog retrieves NEP5TransferLog for the acc.
func (cd *cachedDao) GetNEP5TransferLog(acc util.Uint160, index uint32) (*state.NEP5TransferLog, error) { func (cd *Cached) GetNEP5TransferLog(acc util.Uint160, index uint32) (*state.NEP5TransferLog, error) {
ts := cd.transfers[acc] ts := cd.transfers[acc]
if ts != nil && ts[index] != nil { if ts != nil && ts[index] != nil {
return ts[index], nil return ts[index], nil
} }
return cd.dao.GetNEP5TransferLog(acc, index) return cd.DAO.GetNEP5TransferLog(acc, index)
} }
// PutNEP5TransferLog saves NEP5TransferLog for the acc. // PutNEP5TransferLog saves NEP5TransferLog for the acc.
func (cd *cachedDao) PutNEP5TransferLog(acc util.Uint160, index uint32, bs *state.NEP5TransferLog) error { func (cd *Cached) PutNEP5TransferLog(acc util.Uint160, index uint32, bs *state.NEP5TransferLog) error {
ts := cd.transfers[acc] ts := cd.transfers[acc]
if ts == nil { if ts == nil {
ts = make(map[uint32]*state.NEP5TransferLog, 2) ts = make(map[uint32]*state.NEP5TransferLog, 2)
@ -126,7 +126,7 @@ func (cd *cachedDao) PutNEP5TransferLog(acc util.Uint160, index uint32, bs *stat
} }
// AppendNEP5Transfer appends new transfer to a transfer event log. // AppendNEP5Transfer appends new transfer to a transfer event log.
func (cd *cachedDao) AppendNEP5Transfer(acc util.Uint160, index uint32, tr *state.NEP5Transfer) (bool, error) { func (cd *Cached) AppendNEP5Transfer(acc util.Uint160, index uint32, tr *state.NEP5Transfer) (bool, error) {
lg, err := cd.GetNEP5TransferLog(acc, index) lg, err := cd.GetNEP5TransferLog(acc, index)
if err != nil { if err != nil {
return false, err return false, err
@ -139,20 +139,20 @@ func (cd *cachedDao) AppendNEP5Transfer(acc util.Uint160, index uint32, tr *stat
// Persist flushes all the changes made into the (supposedly) persistent // Persist flushes all the changes made into the (supposedly) persistent
// underlying store. // underlying store.
func (cd *cachedDao) Persist() (int, error) { func (cd *Cached) Persist() (int, error) {
lowerCache, ok := cd.dao.(*cachedDao) lowerCache, ok := cd.DAO.(*Cached)
// If the lower dao is cachedDao, we only need to flush the MemCached DB. // If the lower DAO is Cached, we only need to flush the MemCached DB.
// This actually breaks dao interface incapsulation, but for our current // This actually breaks DAO interface incapsulation, but for our current
// usage scenario it should be good enough if cd doesn't modify object // usage scenario it should be good enough if cd doesn't modify object
// caches (accounts/contracts/etc) in any way. // caches (accounts/contracts/etc) in any way.
if ok { if ok {
var simpleCache *simpleDao var simpleCache *Simple
for simpleCache == nil { for simpleCache == nil {
simpleCache, ok = lowerCache.dao.(*simpleDao) simpleCache, ok = lowerCache.DAO.(*Simple)
if !ok { if !ok {
lowerCache, ok = cd.dao.(*cachedDao) lowerCache, ok = cd.DAO.(*Cached)
if !ok { if !ok {
return 0, errors.New("unsupported lower dao") return 0, errors.New("unsupported lower DAO")
} }
} }
} }
@ -161,21 +161,21 @@ func (cd *cachedDao) Persist() (int, error) {
buf := io.NewBufBinWriter() buf := io.NewBufBinWriter()
for sc := range cd.accounts { for sc := range cd.accounts {
err := cd.dao.putAccountState(cd.accounts[sc], buf) err := cd.DAO.putAccountState(cd.accounts[sc], buf)
if err != nil { if err != nil {
return 0, err return 0, err
} }
buf.Reset() buf.Reset()
} }
for hash := range cd.unspents { for hash := range cd.unspents {
err := cd.dao.putUnspentCoinState(hash, cd.unspents[hash], buf) err := cd.DAO.putUnspentCoinState(hash, cd.unspents[hash], buf)
if err != nil { if err != nil {
return 0, err return 0, err
} }
buf.Reset() buf.Reset()
} }
for acc, bs := range cd.balances { for acc, bs := range cd.balances {
err := cd.dao.putNEP5Balances(acc, bs, buf) err := cd.DAO.putNEP5Balances(acc, bs, buf)
if err != nil { if err != nil {
return 0, err return 0, err
} }
@ -183,17 +183,18 @@ func (cd *cachedDao) Persist() (int, error) {
} }
for acc, ts := range cd.transfers { for acc, ts := range cd.transfers {
for ind, lg := range ts { for ind, lg := range ts {
err := cd.dao.PutNEP5TransferLog(acc, ind, lg) err := cd.DAO.PutNEP5TransferLog(acc, ind, lg)
if err != nil { if err != nil {
return 0, err return 0, err
} }
} }
} }
return cd.dao.Persist() return cd.DAO.Persist()
} }
func (cd *cachedDao) GetWrapped() dao { // GetWrapped implements DAO interface.
return &cachedDao{cd.dao.GetWrapped(), func (cd *Cached) GetWrapped() DAO {
return &Cached{cd.DAO.GetWrapped(),
cd.accounts, cd.accounts,
cd.contracts, cd.contracts,
cd.unspents, cd.unspents,

View file

@ -1,4 +1,4 @@
package core package dao
import ( import (
"testing" "testing"
@ -15,9 +15,9 @@ import (
func TestCachedDaoAccounts(t *testing.T) { func TestCachedDaoAccounts(t *testing.T) {
store := storage.NewMemoryStore() store := storage.NewMemoryStore()
// Persistent DAO to check for backing storage. // Persistent DAO to check for backing storage.
pdao := newSimpleDao(store) pdao := NewSimple(store)
// Cached DAO. // Cached DAO.
cdao := newCachedDao(pdao) cdao := NewCached(pdao)
hash := random.Uint160() hash := random.Uint160()
_, err := cdao.GetAccountState(hash) _, err := cdao.GetAccountState(hash)
@ -51,8 +51,8 @@ func TestCachedDaoAccounts(t *testing.T) {
func TestCachedDaoContracts(t *testing.T) { func TestCachedDaoContracts(t *testing.T) {
store := storage.NewMemoryStore() store := storage.NewMemoryStore()
pdao := newSimpleDao(store) pdao := NewSimple(store)
dao := newCachedDao(pdao) dao := NewCached(pdao)
script := []byte{0xde, 0xad, 0xbe, 0xef} script := []byte{0xde, 0xad, 0xbe, 0xef}
sh := hash.Hash160(script) sh := hash.Hash160(script)
@ -71,7 +71,7 @@ func TestCachedDaoContracts(t *testing.T) {
_, err = dao.Persist() _, err = dao.Persist()
require.Nil(t, err) require.Nil(t, err)
dao2 := newCachedDao(pdao) dao2 := NewCached(pdao)
cs2, err = dao2.GetContractState(sh) cs2, err = dao2.GetContractState(sh)
require.Nil(t, err) require.Nil(t, err)
require.Equal(t, cs, cs2) require.Equal(t, cs, cs2)
@ -87,21 +87,21 @@ func TestCachedDaoContracts(t *testing.T) {
func TestCachedCachedDao(t *testing.T) { func TestCachedCachedDao(t *testing.T) {
store := storage.NewMemoryStore() store := storage.NewMemoryStore()
// Persistent DAO to check for backing storage. // Persistent DAO to check for backing storage.
pdao := newSimpleDao(store) pdao := NewSimple(store)
assert.NotEqual(t, store, pdao.store) assert.NotEqual(t, store, pdao.Store)
// Cached DAO. // Cached DAO.
cdao := newCachedDao(pdao) cdao := NewCached(pdao)
cdaoDao := cdao.dao.(*simpleDao) cdaoDao := cdao.DAO.(*Simple)
assert.NotEqual(t, store, cdaoDao.store) assert.NotEqual(t, store, cdaoDao.Store)
assert.NotEqual(t, pdao.store, cdaoDao.store) assert.NotEqual(t, pdao.Store, cdaoDao.Store)
// Cached cached DAO. // Cached cached DAO.
ccdao := newCachedDao(cdao) ccdao := NewCached(cdao)
ccdaoDao := ccdao.dao.(*cachedDao) ccdaoDao := ccdao.DAO.(*Cached)
intDao := ccdaoDao.dao.(*simpleDao) intDao := ccdaoDao.DAO.(*Simple)
assert.NotEqual(t, store, intDao.store) assert.NotEqual(t, store, intDao.Store)
assert.NotEqual(t, pdao.store, intDao.store) assert.NotEqual(t, pdao.Store, intDao.Store)
assert.NotEqual(t, cdaoDao.store, intDao.store) assert.NotEqual(t, cdaoDao.Store, intDao.Store)
hash := random.Uint160() hash := random.Uint160()
key := []byte("qwerty") key := []byte("qwerty")

View file

@ -1,4 +1,4 @@
package core package dao
import ( import (
"bytes" "bytes"
@ -15,8 +15,8 @@ import (
"github.com/nspcc-dev/neo-go/pkg/util" "github.com/nspcc-dev/neo-go/pkg/util"
) )
// dao is a data access object. // DAO is a data access object.
type dao interface { type DAO interface {
AppendNEP5Transfer(acc util.Uint160, index uint32, tr *state.NEP5Transfer) (bool, error) AppendNEP5Transfer(acc util.Uint160, index uint32, tr *state.NEP5Transfer) (bool, error)
DeleteContractState(hash util.Uint160) error DeleteContractState(hash util.Uint160) error
DeleteStorageItem(scripthash util.Uint160, key []byte) error DeleteStorageItem(scripthash util.Uint160, key []byte) error
@ -43,7 +43,7 @@ type dao interface {
GetValidators() []*state.Validator GetValidators() []*state.Validator
GetValidatorsCount() (*state.ValidatorsCount, error) GetValidatorsCount() (*state.ValidatorsCount, error)
GetVersion() (string, error) GetVersion() (string, error)
GetWrapped() dao GetWrapped() DAO
HasTransaction(hash util.Uint256) bool HasTransaction(hash util.Uint256) bool
IsDoubleClaim(claim *transaction.ClaimTX) bool IsDoubleClaim(claim *transaction.ClaimTX) bool
IsDoubleSpend(tx *transaction.Transaction) bool IsDoubleSpend(tx *transaction.Transaction) bool
@ -68,29 +68,30 @@ type dao interface {
putUnspentCoinState(hash util.Uint256, ucs *state.UnspentCoin, buf *io.BufBinWriter) error putUnspentCoinState(hash util.Uint256, ucs *state.UnspentCoin, buf *io.BufBinWriter) error
} }
// simpleDao is memCached wrapper around DB, simple dao implementation. // Simple is memCached wrapper around DB, simple DAO implementation.
type simpleDao struct { type Simple struct {
store *storage.MemCachedStore Store *storage.MemCachedStore
} }
func newSimpleDao(backend storage.Store) *simpleDao { // NewSimple creates new simple dao using provided backend store.
return &simpleDao{store: storage.NewMemCachedStore(backend)} func NewSimple(backend storage.Store) *Simple {
return &Simple{Store: storage.NewMemCachedStore(backend)}
} }
// GetBatch returns currently accumulated DB changeset. // GetBatch returns currently accumulated DB changeset.
func (dao *simpleDao) GetBatch() *storage.MemBatch { func (dao *Simple) GetBatch() *storage.MemBatch {
return dao.store.GetBatch() return dao.Store.GetBatch()
} }
// GetWrapped returns new dao instance with another layer of wrapped // GetWrapped returns new DAO instance with another layer of wrapped
// MemCachedStore around the current dao store. // MemCachedStore around the current DAO Store.
func (dao *simpleDao) GetWrapped() dao { func (dao *Simple) GetWrapped() DAO {
return newSimpleDao(dao.store) return NewSimple(dao.Store)
} }
// GetAndDecode performs get operation and decoding with serializable structures. // GetAndDecode performs get operation and decoding with serializable structures.
func (dao *simpleDao) GetAndDecode(entity io.Serializable, key []byte) error { func (dao *Simple) GetAndDecode(entity io.Serializable, key []byte) error {
entityBytes, err := dao.store.Get(key) entityBytes, err := dao.Store.Get(key)
if err != nil { if err != nil {
return err return err
} }
@ -100,24 +101,24 @@ func (dao *simpleDao) GetAndDecode(entity io.Serializable, key []byte) error {
} }
// Put performs put operation with serializable structures. // Put performs put operation with serializable structures.
func (dao *simpleDao) Put(entity io.Serializable, key []byte) error { func (dao *Simple) Put(entity io.Serializable, key []byte) error {
return dao.putWithBuffer(entity, key, io.NewBufBinWriter()) return dao.putWithBuffer(entity, key, io.NewBufBinWriter())
} }
// putWithBuffer performs put operation using buf as a pre-allocated buffer for serialization. // putWithBuffer performs put operation using buf as a pre-allocated buffer for serialization.
func (dao *simpleDao) putWithBuffer(entity io.Serializable, key []byte, buf *io.BufBinWriter) error { func (dao *Simple) putWithBuffer(entity io.Serializable, key []byte, buf *io.BufBinWriter) error {
entity.EncodeBinary(buf.BinWriter) entity.EncodeBinary(buf.BinWriter)
if buf.Err != nil { if buf.Err != nil {
return buf.Err return buf.Err
} }
return dao.store.Put(key, buf.Bytes()) return dao.Store.Put(key, buf.Bytes())
} }
// -- start accounts. // -- start accounts.
// GetAccountStateOrNew retrieves Account from temporary or persistent Store // GetAccountStateOrNew retrieves Account from temporary or persistent Store
// or creates a new one if it doesn't exist and persists it. // or creates a new one if it doesn't exist and persists it.
func (dao *simpleDao) GetAccountStateOrNew(hash util.Uint160) (*state.Account, error) { func (dao *Simple) GetAccountStateOrNew(hash util.Uint160) (*state.Account, error) {
account, err := dao.GetAccountState(hash) account, err := dao.GetAccountState(hash)
if err != nil { if err != nil {
if err != storage.ErrKeyNotFound { if err != storage.ErrKeyNotFound {
@ -130,7 +131,7 @@ func (dao *simpleDao) GetAccountStateOrNew(hash util.Uint160) (*state.Account, e
// GetAccountState returns Account from the given Store if it's // GetAccountState returns Account from the given Store if it's
// present there. Returns nil otherwise. // present there. Returns nil otherwise.
func (dao *simpleDao) GetAccountState(hash util.Uint160) (*state.Account, error) { func (dao *Simple) GetAccountState(hash util.Uint160) (*state.Account, error) {
account := &state.Account{} account := &state.Account{}
key := storage.AppendPrefix(storage.STAccount, hash.BytesBE()) key := storage.AppendPrefix(storage.STAccount, hash.BytesBE())
err := dao.GetAndDecode(account, key) err := dao.GetAndDecode(account, key)
@ -140,11 +141,12 @@ func (dao *simpleDao) GetAccountState(hash util.Uint160) (*state.Account, error)
return account, err return account, err
} }
func (dao *simpleDao) PutAccountState(as *state.Account) error { // PutAccountState saves given Account in given store.
func (dao *Simple) PutAccountState(as *state.Account) error {
return dao.putAccountState(as, io.NewBufBinWriter()) return dao.putAccountState(as, io.NewBufBinWriter())
} }
func (dao *simpleDao) putAccountState(as *state.Account, buf *io.BufBinWriter) error { func (dao *Simple) putAccountState(as *state.Account, buf *io.BufBinWriter) error {
key := storage.AppendPrefix(storage.STAccount, as.ScriptHash.BytesBE()) key := storage.AppendPrefix(storage.STAccount, as.ScriptHash.BytesBE())
return dao.putWithBuffer(as, key, buf) return dao.putWithBuffer(as, key, buf)
} }
@ -154,7 +156,7 @@ func (dao *simpleDao) putAccountState(as *state.Account, buf *io.BufBinWriter) e
// -- start assets. // -- start assets.
// GetAssetState returns given asset state as recorded in the given store. // GetAssetState returns given asset state as recorded in the given store.
func (dao *simpleDao) GetAssetState(assetID util.Uint256) (*state.Asset, error) { func (dao *Simple) GetAssetState(assetID util.Uint256) (*state.Asset, error) {
asset := &state.Asset{} asset := &state.Asset{}
key := storage.AppendPrefix(storage.STAsset, assetID.BytesBE()) key := storage.AppendPrefix(storage.STAsset, assetID.BytesBE())
err := dao.GetAndDecode(asset, key) err := dao.GetAndDecode(asset, key)
@ -168,7 +170,7 @@ func (dao *simpleDao) GetAssetState(assetID util.Uint256) (*state.Asset, error)
} }
// PutAssetState puts given asset state into the given store. // PutAssetState puts given asset state into the given store.
func (dao *simpleDao) PutAssetState(as *state.Asset) error { func (dao *Simple) PutAssetState(as *state.Asset) error {
key := storage.AppendPrefix(storage.STAsset, as.ID.BytesBE()) key := storage.AppendPrefix(storage.STAsset, as.ID.BytesBE())
return dao.Put(as, key) return dao.Put(as, key)
} }
@ -179,7 +181,7 @@ func (dao *simpleDao) PutAssetState(as *state.Asset) error {
// GetContractState returns contract state as recorded in the given // GetContractState returns contract state as recorded in the given
// store by the given script hash. // store by the given script hash.
func (dao *simpleDao) GetContractState(hash util.Uint160) (*state.Contract, error) { func (dao *Simple) GetContractState(hash util.Uint160) (*state.Contract, error) {
contract := &state.Contract{} contract := &state.Contract{}
key := storage.AppendPrefix(storage.STContract, hash.BytesBE()) key := storage.AppendPrefix(storage.STContract, hash.BytesBE())
err := dao.GetAndDecode(contract, key) err := dao.GetAndDecode(contract, key)
@ -194,15 +196,15 @@ func (dao *simpleDao) GetContractState(hash util.Uint160) (*state.Contract, erro
} }
// PutContractState puts given contract state into the given store. // PutContractState puts given contract state into the given store.
func (dao *simpleDao) PutContractState(cs *state.Contract) error { func (dao *Simple) PutContractState(cs *state.Contract) error {
key := storage.AppendPrefix(storage.STContract, cs.ScriptHash().BytesBE()) key := storage.AppendPrefix(storage.STContract, cs.ScriptHash().BytesBE())
return dao.Put(cs, key) return dao.Put(cs, key)
} }
// DeleteContractState deletes given contract state in the given store. // DeleteContractState deletes given contract state in the given store.
func (dao *simpleDao) DeleteContractState(hash util.Uint160) error { func (dao *Simple) DeleteContractState(hash util.Uint160) error {
key := storage.AppendPrefix(storage.STContract, hash.BytesBE()) key := storage.AppendPrefix(storage.STContract, hash.BytesBE())
return dao.store.Delete(key) return dao.Store.Delete(key)
} }
// -- end contracts. // -- end contracts.
@ -210,7 +212,7 @@ func (dao *simpleDao) DeleteContractState(hash util.Uint160) error {
// -- start nep5 balances. // -- start nep5 balances.
// GetNEP5Balances retrieves nep5 balances from the cache. // GetNEP5Balances retrieves nep5 balances from the cache.
func (dao *simpleDao) GetNEP5Balances(acc util.Uint160) (*state.NEP5Balances, error) { func (dao *Simple) GetNEP5Balances(acc util.Uint160) (*state.NEP5Balances, error) {
key := storage.AppendPrefix(storage.STNEP5Balances, acc.BytesBE()) key := storage.AppendPrefix(storage.STNEP5Balances, acc.BytesBE())
bs := state.NewNEP5Balances() bs := state.NewNEP5Balances()
err := dao.GetAndDecode(bs, key) err := dao.GetAndDecode(bs, key)
@ -221,11 +223,11 @@ func (dao *simpleDao) GetNEP5Balances(acc util.Uint160) (*state.NEP5Balances, er
} }
// PutNEP5Balances saves nep5 balances from the cache. // PutNEP5Balances saves nep5 balances from the cache.
func (dao *simpleDao) PutNEP5Balances(acc util.Uint160, bs *state.NEP5Balances) error { func (dao *Simple) PutNEP5Balances(acc util.Uint160, bs *state.NEP5Balances) error {
return dao.putNEP5Balances(acc, bs, io.NewBufBinWriter()) return dao.putNEP5Balances(acc, bs, io.NewBufBinWriter())
} }
func (dao *simpleDao) putNEP5Balances(acc util.Uint160, bs *state.NEP5Balances, buf *io.BufBinWriter) error { func (dao *Simple) putNEP5Balances(acc util.Uint160, bs *state.NEP5Balances, buf *io.BufBinWriter) error {
key := storage.AppendPrefix(storage.STNEP5Balances, acc.BytesBE()) key := storage.AppendPrefix(storage.STNEP5Balances, acc.BytesBE())
return dao.putWithBuffer(bs, key, buf) return dao.putWithBuffer(bs, key, buf)
} }
@ -245,9 +247,9 @@ func getNEP5TransferLogKey(acc util.Uint160, index uint32) []byte {
} }
// GetNEP5TransferLog retrieves transfer log from the cache. // GetNEP5TransferLog retrieves transfer log from the cache.
func (dao *simpleDao) GetNEP5TransferLog(acc util.Uint160, index uint32) (*state.NEP5TransferLog, error) { func (dao *Simple) GetNEP5TransferLog(acc util.Uint160, index uint32) (*state.NEP5TransferLog, error) {
key := getNEP5TransferLogKey(acc, index) key := getNEP5TransferLogKey(acc, index)
value, err := dao.store.Get(key) value, err := dao.Store.Get(key)
if err != nil { if err != nil {
if err == storage.ErrKeyNotFound { if err == storage.ErrKeyNotFound {
return new(state.NEP5TransferLog), nil return new(state.NEP5TransferLog), nil
@ -258,14 +260,14 @@ func (dao *simpleDao) GetNEP5TransferLog(acc util.Uint160, index uint32) (*state
} }
// PutNEP5TransferLog saves given transfer log in the cache. // PutNEP5TransferLog saves given transfer log in the cache.
func (dao *simpleDao) PutNEP5TransferLog(acc util.Uint160, index uint32, lg *state.NEP5TransferLog) error { func (dao *Simple) PutNEP5TransferLog(acc util.Uint160, index uint32, lg *state.NEP5TransferLog) error {
key := getNEP5TransferLogKey(acc, index) key := getNEP5TransferLogKey(acc, index)
return dao.store.Put(key, lg.Raw) return dao.Store.Put(key, lg.Raw)
} }
// AppendNEP5Transfer appends a single NEP5 transfer to a log. // AppendNEP5Transfer appends a single NEP5 transfer to a log.
// First return value signalizes that log size has exceeded batch size. // First return value signalizes that log size has exceeded batch size.
func (dao *simpleDao) AppendNEP5Transfer(acc util.Uint160, index uint32, tr *state.NEP5Transfer) (bool, error) { func (dao *Simple) AppendNEP5Transfer(acc util.Uint160, index uint32, tr *state.NEP5Transfer) (bool, error) {
lg, err := dao.GetNEP5TransferLog(acc, index) lg, err := dao.GetNEP5TransferLog(acc, index)
if err != nil { if err != nil {
if err != storage.ErrKeyNotFound { if err != storage.ErrKeyNotFound {
@ -284,7 +286,7 @@ func (dao *simpleDao) AppendNEP5Transfer(acc util.Uint160, index uint32, tr *sta
// -- start unspent coins. // -- start unspent coins.
// GetUnspentCoinState retrieves UnspentCoinState from the given store. // GetUnspentCoinState retrieves UnspentCoinState from the given store.
func (dao *simpleDao) GetUnspentCoinState(hash util.Uint256) (*state.UnspentCoin, error) { func (dao *Simple) GetUnspentCoinState(hash util.Uint256) (*state.UnspentCoin, error) {
unspent := &state.UnspentCoin{} unspent := &state.UnspentCoin{}
key := storage.AppendPrefix(storage.STCoin, hash.BytesLE()) key := storage.AppendPrefix(storage.STCoin, hash.BytesLE())
err := dao.GetAndDecode(unspent, key) err := dao.GetAndDecode(unspent, key)
@ -295,11 +297,11 @@ func (dao *simpleDao) GetUnspentCoinState(hash util.Uint256) (*state.UnspentCoin
} }
// PutUnspentCoinState puts given UnspentCoinState into the given store. // PutUnspentCoinState puts given UnspentCoinState into the given store.
func (dao *simpleDao) PutUnspentCoinState(hash util.Uint256, ucs *state.UnspentCoin) error { func (dao *Simple) PutUnspentCoinState(hash util.Uint256, ucs *state.UnspentCoin) error {
return dao.putUnspentCoinState(hash, ucs, io.NewBufBinWriter()) return dao.putUnspentCoinState(hash, ucs, io.NewBufBinWriter())
} }
func (dao *simpleDao) putUnspentCoinState(hash util.Uint256, ucs *state.UnspentCoin, buf *io.BufBinWriter) error { func (dao *Simple) putUnspentCoinState(hash util.Uint256, ucs *state.UnspentCoin, buf *io.BufBinWriter) error {
key := storage.AppendPrefix(storage.STCoin, hash.BytesLE()) key := storage.AppendPrefix(storage.STCoin, hash.BytesLE())
return dao.putWithBuffer(ucs, key, buf) return dao.putWithBuffer(ucs, key, buf)
} }
@ -309,7 +311,7 @@ func (dao *simpleDao) putUnspentCoinState(hash util.Uint256, ucs *state.UnspentC
// -- start validator. // -- start validator.
// GetValidatorStateOrNew gets validator from store or created new one in case of error. // GetValidatorStateOrNew gets validator from store or created new one in case of error.
func (dao *simpleDao) GetValidatorStateOrNew(publicKey *keys.PublicKey) (*state.Validator, error) { func (dao *Simple) GetValidatorStateOrNew(publicKey *keys.PublicKey) (*state.Validator, error) {
validatorState, err := dao.GetValidatorState(publicKey) validatorState, err := dao.GetValidatorState(publicKey)
if err != nil { if err != nil {
if err != storage.ErrKeyNotFound { if err != storage.ErrKeyNotFound {
@ -322,9 +324,9 @@ func (dao *simpleDao) GetValidatorStateOrNew(publicKey *keys.PublicKey) (*state.
} }
// GetValidators returns all validators from store. // GetValidators returns all validators from store.
func (dao *simpleDao) GetValidators() []*state.Validator { func (dao *Simple) GetValidators() []*state.Validator {
var validators []*state.Validator var validators []*state.Validator
dao.store.Seek(storage.STValidator.Bytes(), func(k, v []byte) { dao.Store.Seek(storage.STValidator.Bytes(), func(k, v []byte) {
r := io.NewBinReaderFromBuf(v) r := io.NewBinReaderFromBuf(v)
validator := &state.Validator{} validator := &state.Validator{}
validator.DecodeBinary(r) validator.DecodeBinary(r)
@ -337,7 +339,7 @@ func (dao *simpleDao) GetValidators() []*state.Validator {
} }
// GetValidatorState returns validator by publicKey. // GetValidatorState returns validator by publicKey.
func (dao *simpleDao) GetValidatorState(publicKey *keys.PublicKey) (*state.Validator, error) { func (dao *Simple) GetValidatorState(publicKey *keys.PublicKey) (*state.Validator, error) {
validatorState := &state.Validator{} validatorState := &state.Validator{}
key := storage.AppendPrefix(storage.STValidator, publicKey.Bytes()) key := storage.AppendPrefix(storage.STValidator, publicKey.Bytes())
err := dao.GetAndDecode(validatorState, key) err := dao.GetAndDecode(validatorState, key)
@ -348,20 +350,20 @@ func (dao *simpleDao) GetValidatorState(publicKey *keys.PublicKey) (*state.Valid
} }
// PutValidatorState puts given Validator into the given store. // PutValidatorState puts given Validator into the given store.
func (dao *simpleDao) PutValidatorState(vs *state.Validator) error { func (dao *Simple) PutValidatorState(vs *state.Validator) error {
key := storage.AppendPrefix(storage.STValidator, vs.PublicKey.Bytes()) key := storage.AppendPrefix(storage.STValidator, vs.PublicKey.Bytes())
return dao.Put(vs, key) return dao.Put(vs, key)
} }
// DeleteValidatorState deletes given Validator into the given store. // DeleteValidatorState deletes given Validator into the given store.
func (dao *simpleDao) DeleteValidatorState(vs *state.Validator) error { func (dao *Simple) DeleteValidatorState(vs *state.Validator) error {
key := storage.AppendPrefix(storage.STValidator, vs.PublicKey.Bytes()) key := storage.AppendPrefix(storage.STValidator, vs.PublicKey.Bytes())
return dao.store.Delete(key) return dao.Store.Delete(key)
} }
// GetValidatorsCount returns current ValidatorsCount or new one if there is none // GetValidatorsCount returns current ValidatorsCount or new one if there is none
// in the DB. // in the DB.
func (dao *simpleDao) GetValidatorsCount() (*state.ValidatorsCount, error) { func (dao *Simple) GetValidatorsCount() (*state.ValidatorsCount, error) {
vc := &state.ValidatorsCount{} vc := &state.ValidatorsCount{}
key := []byte{byte(storage.IXValidatorsCount)} key := []byte{byte(storage.IXValidatorsCount)}
err := dao.GetAndDecode(vc, key) err := dao.GetAndDecode(vc, key)
@ -372,7 +374,7 @@ func (dao *simpleDao) GetValidatorsCount() (*state.ValidatorsCount, error) {
} }
// PutValidatorsCount put given ValidatorsCount in the store. // PutValidatorsCount put given ValidatorsCount in the store.
func (dao *simpleDao) PutValidatorsCount(vc *state.ValidatorsCount) error { func (dao *Simple) PutValidatorsCount(vc *state.ValidatorsCount) error {
key := []byte{byte(storage.IXValidatorsCount)} key := []byte{byte(storage.IXValidatorsCount)}
return dao.Put(vc, key) return dao.Put(vc, key)
} }
@ -383,7 +385,7 @@ func (dao *simpleDao) PutValidatorsCount(vc *state.ValidatorsCount) error {
// GetAppExecResult gets application execution result from the // GetAppExecResult gets application execution result from the
// given store. // given store.
func (dao *simpleDao) GetAppExecResult(hash util.Uint256) (*state.AppExecResult, error) { func (dao *Simple) GetAppExecResult(hash util.Uint256) (*state.AppExecResult, error) {
aer := &state.AppExecResult{} aer := &state.AppExecResult{}
key := storage.AppendPrefix(storage.STNotification, hash.BytesBE()) key := storage.AppendPrefix(storage.STNotification, hash.BytesBE())
err := dao.GetAndDecode(aer, key) err := dao.GetAndDecode(aer, key)
@ -395,7 +397,7 @@ func (dao *simpleDao) GetAppExecResult(hash util.Uint256) (*state.AppExecResult,
// PutAppExecResult puts given application execution result into the // PutAppExecResult puts given application execution result into the
// given store. // given store.
func (dao *simpleDao) PutAppExecResult(aer *state.AppExecResult) error { func (dao *Simple) PutAppExecResult(aer *state.AppExecResult) error {
key := storage.AppendPrefix(storage.STNotification, aer.TxHash.BytesBE()) key := storage.AppendPrefix(storage.STNotification, aer.TxHash.BytesBE())
return dao.Put(aer, key) return dao.Put(aer, key)
} }
@ -404,9 +406,9 @@ func (dao *simpleDao) PutAppExecResult(aer *state.AppExecResult) error {
// -- start storage item. // -- start storage item.
// GetStorageItem returns StorageItem if it exists in the given Store. // GetStorageItem returns StorageItem if it exists in the given store.
func (dao *simpleDao) GetStorageItem(scripthash util.Uint160, key []byte) *state.StorageItem { func (dao *Simple) GetStorageItem(scripthash util.Uint160, key []byte) *state.StorageItem {
b, err := dao.store.Get(makeStorageItemKey(scripthash, key)) b, err := dao.Store.Get(makeStorageItemKey(scripthash, key))
if err != nil { if err != nil {
return nil return nil
} }
@ -422,19 +424,19 @@ func (dao *simpleDao) GetStorageItem(scripthash util.Uint160, key []byte) *state
} }
// PutStorageItem puts given StorageItem for given script with given // PutStorageItem puts given StorageItem for given script with given
// key into the given Store. // key into the given store.
func (dao *simpleDao) PutStorageItem(scripthash util.Uint160, key []byte, si *state.StorageItem) error { func (dao *Simple) PutStorageItem(scripthash util.Uint160, key []byte, si *state.StorageItem) error {
return dao.Put(si, makeStorageItemKey(scripthash, key)) return dao.Put(si, makeStorageItemKey(scripthash, key))
} }
// DeleteStorageItem drops storage item for the given script with the // DeleteStorageItem drops storage item for the given script with the
// given key from the Store. // given key from the store.
func (dao *simpleDao) DeleteStorageItem(scripthash util.Uint160, key []byte) error { func (dao *Simple) DeleteStorageItem(scripthash util.Uint160, key []byte) error {
return dao.store.Delete(makeStorageItemKey(scripthash, key)) return dao.Store.Delete(makeStorageItemKey(scripthash, key))
} }
// GetStorageItems returns all storage items for a given scripthash. // GetStorageItems returns all storage items for a given scripthash.
func (dao *simpleDao) GetStorageItems(hash util.Uint160) (map[string]*state.StorageItem, error) { func (dao *Simple) GetStorageItems(hash util.Uint160) (map[string]*state.StorageItem, error) {
var siMap = make(map[string]*state.StorageItem) var siMap = make(map[string]*state.StorageItem)
var err error var err error
@ -453,7 +455,7 @@ func (dao *simpleDao) GetStorageItems(hash util.Uint160) (map[string]*state.Stor
// Cut prefix and hash. // Cut prefix and hash.
siMap[string(k[21:])] = si siMap[string(k[21:])] = si
} }
dao.store.Seek(storage.AppendPrefix(storage.STStorage, hash.BytesLE()), saveToMap) dao.Store.Seek(storage.AppendPrefix(storage.STStorage, hash.BytesLE()), saveToMap)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -470,9 +472,9 @@ func makeStorageItemKey(scripthash util.Uint160, key []byte) []byte {
// -- other. // -- other.
// GetBlock returns Block by the given hash if it exists in the store. // GetBlock returns Block by the given hash if it exists in the store.
func (dao *simpleDao) GetBlock(hash util.Uint256) (*block.Block, uint32, error) { func (dao *Simple) GetBlock(hash util.Uint256) (*block.Block, uint32, error) {
key := storage.AppendPrefix(storage.DataBlock, hash.BytesLE()) key := storage.AppendPrefix(storage.DataBlock, hash.BytesLE())
b, err := dao.store.Get(key) b, err := dao.Store.Get(key)
if err != nil { if err != nil {
return nil, 0, err return nil, 0, err
} }
@ -485,16 +487,16 @@ func (dao *simpleDao) GetBlock(hash util.Uint256) (*block.Block, uint32, error)
} }
// GetVersion attempts to get the current version stored in the // GetVersion attempts to get the current version stored in the
// underlying Store. // underlying store.
func (dao *simpleDao) GetVersion() (string, error) { func (dao *Simple) GetVersion() (string, error) {
version, err := dao.store.Get(storage.SYSVersion.Bytes()) version, err := dao.Store.Get(storage.SYSVersion.Bytes())
return string(version), err return string(version), err
} }
// GetCurrentBlockHeight returns the current block height found in the // GetCurrentBlockHeight returns the current block height found in the
// underlying Store. // underlying store.
func (dao *simpleDao) GetCurrentBlockHeight() (uint32, error) { func (dao *Simple) GetCurrentBlockHeight() (uint32, error) {
b, err := dao.store.Get(storage.SYSCurrentBlock.Bytes()) b, err := dao.Store.Get(storage.SYSCurrentBlock.Bytes())
if err != nil { if err != nil {
return 0, err return 0, err
} }
@ -502,10 +504,10 @@ func (dao *simpleDao) GetCurrentBlockHeight() (uint32, error) {
} }
// GetCurrentHeaderHeight returns the current header height and hash from // GetCurrentHeaderHeight returns the current header height and hash from
// the underlying Store. // the underlying store.
func (dao *simpleDao) GetCurrentHeaderHeight() (i uint32, h util.Uint256, err error) { func (dao *Simple) GetCurrentHeaderHeight() (i uint32, h util.Uint256, err error) {
var b []byte var b []byte
b, err = dao.store.Get(storage.SYSCurrentHeader.Bytes()) b, err = dao.Store.Get(storage.SYSCurrentHeader.Bytes())
if err != nil { if err != nil {
return return
} }
@ -515,10 +517,10 @@ func (dao *simpleDao) GetCurrentHeaderHeight() (i uint32, h util.Uint256, err er
} }
// GetHeaderHashes returns a sorted list of header hashes retrieved from // GetHeaderHashes returns a sorted list of header hashes retrieved from
// the given underlying Store. // the given underlying store.
func (dao *simpleDao) GetHeaderHashes() ([]util.Uint256, error) { func (dao *Simple) GetHeaderHashes() ([]util.Uint256, error) {
hashMap := make(map[uint32][]util.Uint256) hashMap := make(map[uint32][]util.Uint256)
dao.store.Seek(storage.IXHeaderHashList.Bytes(), func(k, v []byte) { dao.Store.Seek(storage.IXHeaderHashList.Bytes(), func(k, v []byte) {
storedCount := binary.LittleEndian.Uint32(k[1:]) storedCount := binary.LittleEndian.Uint32(k[1:])
hashes, err := read2000Uint256Hashes(v) hashes, err := read2000Uint256Hashes(v)
if err != nil { if err != nil {
@ -535,7 +537,7 @@ func (dao *simpleDao) GetHeaderHashes() ([]util.Uint256, error) {
for k := range hashMap { for k := range hashMap {
sortedKeys = append(sortedKeys, k) sortedKeys = append(sortedKeys, k)
} }
sort.Sort(slice(sortedKeys)) sort.Slice(sortedKeys, func(i, j int) bool { return sortedKeys[i] < sortedKeys[j] })
for _, key := range sortedKeys { for _, key := range sortedKeys {
hashes = append(hashes[:key], hashMap[key]...) hashes = append(hashes[:key], hashMap[key]...)
@ -546,9 +548,9 @@ func (dao *simpleDao) GetHeaderHashes() ([]util.Uint256, error) {
// GetTransaction returns Transaction and its height by the given hash // GetTransaction returns Transaction and its height by the given hash
// if it exists in the store. // if it exists in the store.
func (dao *simpleDao) GetTransaction(hash util.Uint256) (*transaction.Transaction, uint32, error) { func (dao *Simple) GetTransaction(hash util.Uint256) (*transaction.Transaction, uint32, error) {
key := storage.AppendPrefix(storage.DataTransaction, hash.BytesLE()) key := storage.AppendPrefix(storage.DataTransaction, hash.BytesLE())
b, err := dao.store.Get(key) b, err := dao.Store.Get(key)
if err != nil { if err != nil {
return nil, 0, err return nil, 0, err
} }
@ -565,14 +567,14 @@ func (dao *simpleDao) GetTransaction(hash util.Uint256) (*transaction.Transactio
return tx, height, nil return tx, height, nil
} }
// PutVersion stores the given version in the underlying Store. // PutVersion stores the given version in the underlying store.
func (dao *simpleDao) PutVersion(v string) error { func (dao *Simple) PutVersion(v string) error {
return dao.store.Put(storage.SYSVersion.Bytes(), []byte(v)) return dao.Store.Put(storage.SYSVersion.Bytes(), []byte(v))
} }
// PutCurrentHeader stores current header. // PutCurrentHeader stores current header.
func (dao *simpleDao) PutCurrentHeader(hashAndIndex []byte) error { func (dao *Simple) PutCurrentHeader(hashAndIndex []byte) error {
return dao.store.Put(storage.SYSCurrentHeader.Bytes(), hashAndIndex) return dao.Store.Put(storage.SYSCurrentHeader.Bytes(), hashAndIndex)
} }
// read2000Uint256Hashes attempts to read 2000 Uint256 hashes from // read2000Uint256Hashes attempts to read 2000 Uint256 hashes from
@ -590,16 +592,16 @@ func read2000Uint256Hashes(b []byte) ([]util.Uint256, error) {
// HasTransaction returns true if the given store contains the given // HasTransaction returns true if the given store contains the given
// Transaction hash. // Transaction hash.
func (dao *simpleDao) HasTransaction(hash util.Uint256) bool { func (dao *Simple) HasTransaction(hash util.Uint256) bool {
key := storage.AppendPrefix(storage.DataTransaction, hash.BytesLE()) key := storage.AppendPrefix(storage.DataTransaction, hash.BytesLE())
if _, err := dao.store.Get(key); err == nil { if _, err := dao.Store.Get(key); err == nil {
return true return true
} }
return false return false
} }
// StoreAsBlock stores the given block as DataBlock. // StoreAsBlock stores the given block as DataBlock.
func (dao *simpleDao) StoreAsBlock(block *block.Block, sysFee uint32) error { func (dao *Simple) StoreAsBlock(block *block.Block, sysFee uint32) error {
var ( var (
key = storage.AppendPrefix(storage.DataBlock, block.Hash().BytesLE()) key = storage.AppendPrefix(storage.DataBlock, block.Hash().BytesLE())
buf = io.NewBufBinWriter() buf = io.NewBufBinWriter()
@ -613,20 +615,20 @@ func (dao *simpleDao) StoreAsBlock(block *block.Block, sysFee uint32) error {
if buf.Err != nil { if buf.Err != nil {
return buf.Err return buf.Err
} }
return dao.store.Put(key, buf.Bytes()) return dao.Store.Put(key, buf.Bytes())
} }
// StoreAsCurrentBlock stores the given block witch prefix SYSCurrentBlock. // StoreAsCurrentBlock stores the given block witch prefix SYSCurrentBlock.
func (dao *simpleDao) StoreAsCurrentBlock(block *block.Block) error { func (dao *Simple) StoreAsCurrentBlock(block *block.Block) error {
buf := io.NewBufBinWriter() buf := io.NewBufBinWriter()
h := block.Hash() h := block.Hash()
h.EncodeBinary(buf.BinWriter) h.EncodeBinary(buf.BinWriter)
buf.WriteU32LE(block.Index) buf.WriteU32LE(block.Index)
return dao.store.Put(storage.SYSCurrentBlock.Bytes(), buf.Bytes()) return dao.Store.Put(storage.SYSCurrentBlock.Bytes(), buf.Bytes())
} }
// StoreAsTransaction stores the given TX as DataTransaction. // StoreAsTransaction stores the given TX as DataTransaction.
func (dao *simpleDao) StoreAsTransaction(tx *transaction.Transaction, index uint32) error { func (dao *Simple) StoreAsTransaction(tx *transaction.Transaction, index uint32) error {
key := storage.AppendPrefix(storage.DataTransaction, tx.Hash().BytesLE()) key := storage.AppendPrefix(storage.DataTransaction, tx.Hash().BytesLE())
buf := io.NewBufBinWriter() buf := io.NewBufBinWriter()
buf.WriteU32LE(index) buf.WriteU32LE(index)
@ -634,20 +636,20 @@ func (dao *simpleDao) StoreAsTransaction(tx *transaction.Transaction, index uint
if buf.Err != nil { if buf.Err != nil {
return buf.Err return buf.Err
} }
return dao.store.Put(key, buf.Bytes()) return dao.Store.Put(key, buf.Bytes())
} }
// IsDoubleSpend verifies that the input transactions are not double spent. // IsDoubleSpend verifies that the input transactions are not double spent.
func (dao *simpleDao) IsDoubleSpend(tx *transaction.Transaction) bool { func (dao *Simple) IsDoubleSpend(tx *transaction.Transaction) bool {
return dao.checkUsedInputs(tx.Inputs, state.CoinSpent) return dao.checkUsedInputs(tx.Inputs, state.CoinSpent)
} }
// IsDoubleClaim verifies that given claim inputs are not already claimed by another tx. // IsDoubleClaim verifies that given claim inputs are not already claimed by another tx.
func (dao *simpleDao) IsDoubleClaim(claim *transaction.ClaimTX) bool { func (dao *Simple) IsDoubleClaim(claim *transaction.ClaimTX) bool {
return dao.checkUsedInputs(claim.Claims, state.CoinClaimed) return dao.checkUsedInputs(claim.Claims, state.CoinClaimed)
} }
func (dao *simpleDao) checkUsedInputs(inputs []transaction.Input, coin state.Coin) bool { func (dao *Simple) checkUsedInputs(inputs []transaction.Input, coin state.Coin) bool {
if len(inputs) == 0 { if len(inputs) == 0 {
return false return false
} }
@ -668,6 +670,6 @@ func (dao *simpleDao) checkUsedInputs(inputs []transaction.Input, coin state.Coi
// Persist flushes all the changes made into the (supposedly) persistent // Persist flushes all the changes made into the (supposedly) persistent
// underlying store. // underlying store.
func (dao *simpleDao) Persist() (int, error) { func (dao *Simple) Persist() (int, error) {
return dao.store.Persist() return dao.Store.Persist()
} }

View file

@ -1,4 +1,4 @@
package core package dao
import ( import (
"testing" "testing"
@ -16,7 +16,7 @@ import (
) )
func TestPutGetAndDecode(t *testing.T) { func TestPutGetAndDecode(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
serializable := &TestSerializable{field: random.String(4)} serializable := &TestSerializable{field: random.String(4)}
hash := []byte{1} hash := []byte{1}
err := dao.Put(serializable, hash) err := dao.Put(serializable, hash)
@ -41,7 +41,7 @@ func (t *TestSerializable) DecodeBinary(reader *io.BinReader) {
} }
func TestGetAccountStateOrNew_New(t *testing.T) { func TestGetAccountStateOrNew_New(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
hash := random.Uint160() hash := random.Uint160()
createdAccount, err := dao.GetAccountStateOrNew(hash) createdAccount, err := dao.GetAccountStateOrNew(hash)
require.NoError(t, err) require.NoError(t, err)
@ -49,7 +49,7 @@ func TestGetAccountStateOrNew_New(t *testing.T) {
} }
func TestPutAndGetAccountStateOrNew(t *testing.T) { func TestPutAndGetAccountStateOrNew(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
hash := random.Uint160() hash := random.Uint160()
accountState := &state.Account{ScriptHash: hash} accountState := &state.Account{ScriptHash: hash}
err := dao.PutAccountState(accountState) err := dao.PutAccountState(accountState)
@ -60,7 +60,7 @@ func TestPutAndGetAccountStateOrNew(t *testing.T) {
} }
func TestPutAndGetAssetState(t *testing.T) { func TestPutAndGetAssetState(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
id := random.Uint256() id := random.Uint256()
assetState := &state.Asset{ID: id, Owner: keys.PublicKey{}} assetState := &state.Asset{ID: id, Owner: keys.PublicKey{}}
err := dao.PutAssetState(assetState) err := dao.PutAssetState(assetState)
@ -71,7 +71,7 @@ func TestPutAndGetAssetState(t *testing.T) {
} }
func TestPutAndGetContractState(t *testing.T) { func TestPutAndGetContractState(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
contractState := &state.Contract{Script: []byte{}, ParamList: []smartcontract.ParamType{}} contractState := &state.Contract{Script: []byte{}, ParamList: []smartcontract.ParamType{}}
hash := contractState.ScriptHash() hash := contractState.ScriptHash()
err := dao.PutContractState(contractState) err := dao.PutContractState(contractState)
@ -82,7 +82,7 @@ func TestPutAndGetContractState(t *testing.T) {
} }
func TestDeleteContractState(t *testing.T) { func TestDeleteContractState(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
contractState := &state.Contract{Script: []byte{}, ParamList: []smartcontract.ParamType{}} contractState := &state.Contract{Script: []byte{}, ParamList: []smartcontract.ParamType{}}
hash := contractState.ScriptHash() hash := contractState.ScriptHash()
err := dao.PutContractState(contractState) err := dao.PutContractState(contractState)
@ -95,7 +95,7 @@ func TestDeleteContractState(t *testing.T) {
} }
func TestGetUnspentCoinState_Err(t *testing.T) { func TestGetUnspentCoinState_Err(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
hash := random.Uint256() hash := random.Uint256()
gotUnspentCoinState, err := dao.GetUnspentCoinState(hash) gotUnspentCoinState, err := dao.GetUnspentCoinState(hash)
require.Error(t, err) require.Error(t, err)
@ -103,7 +103,7 @@ func TestGetUnspentCoinState_Err(t *testing.T) {
} }
func TestPutGetUnspentCoinState(t *testing.T) { func TestPutGetUnspentCoinState(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
hash := random.Uint256() hash := random.Uint256()
unspentCoinState := &state.UnspentCoin{Height: 42, States: []state.OutputState{}} unspentCoinState := &state.UnspentCoin{Height: 42, States: []state.OutputState{}}
err := dao.PutUnspentCoinState(hash, unspentCoinState) err := dao.PutUnspentCoinState(hash, unspentCoinState)
@ -114,7 +114,7 @@ func TestPutGetUnspentCoinState(t *testing.T) {
} }
func TestGetValidatorStateOrNew_New(t *testing.T) { func TestGetValidatorStateOrNew_New(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
publicKey := &keys.PublicKey{} publicKey := &keys.PublicKey{}
validatorState, err := dao.GetValidatorStateOrNew(publicKey) validatorState, err := dao.GetValidatorStateOrNew(publicKey)
require.NoError(t, err) require.NoError(t, err)
@ -122,7 +122,7 @@ func TestGetValidatorStateOrNew_New(t *testing.T) {
} }
func TestPutGetValidatorState(t *testing.T) { func TestPutGetValidatorState(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
publicKey := &keys.PublicKey{} publicKey := &keys.PublicKey{}
validatorState := &state.Validator{ validatorState := &state.Validator{
PublicKey: publicKey, PublicKey: publicKey,
@ -137,7 +137,7 @@ func TestPutGetValidatorState(t *testing.T) {
} }
func TestDeleteValidatorState(t *testing.T) { func TestDeleteValidatorState(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
publicKey := &keys.PublicKey{} publicKey := &keys.PublicKey{}
validatorState := &state.Validator{ validatorState := &state.Validator{
PublicKey: publicKey, PublicKey: publicKey,
@ -154,7 +154,7 @@ func TestDeleteValidatorState(t *testing.T) {
} }
func TestGetValidators(t *testing.T) { func TestGetValidators(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
publicKey := &keys.PublicKey{} publicKey := &keys.PublicKey{}
validatorState := &state.Validator{ validatorState := &state.Validator{
PublicKey: publicKey, PublicKey: publicKey,
@ -169,7 +169,7 @@ func TestGetValidators(t *testing.T) {
} }
func TestPutGetAppExecResult(t *testing.T) { func TestPutGetAppExecResult(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
hash := random.Uint256() hash := random.Uint256()
appExecResult := &state.AppExecResult{ appExecResult := &state.AppExecResult{
TxHash: hash, TxHash: hash,
@ -184,7 +184,7 @@ func TestPutGetAppExecResult(t *testing.T) {
} }
func TestPutGetStorageItem(t *testing.T) { func TestPutGetStorageItem(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
hash := random.Uint160() hash := random.Uint160()
key := []byte{0} key := []byte{0}
storageItem := &state.StorageItem{Value: []uint8{}} storageItem := &state.StorageItem{Value: []uint8{}}
@ -195,7 +195,7 @@ func TestPutGetStorageItem(t *testing.T) {
} }
func TestDeleteStorageItem(t *testing.T) { func TestDeleteStorageItem(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
hash := random.Uint160() hash := random.Uint160()
key := []byte{0} key := []byte{0}
storageItem := &state.StorageItem{Value: []uint8{}} storageItem := &state.StorageItem{Value: []uint8{}}
@ -208,7 +208,7 @@ func TestDeleteStorageItem(t *testing.T) {
} }
func TestGetBlock_NotExists(t *testing.T) { func TestGetBlock_NotExists(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
hash := random.Uint256() hash := random.Uint256()
block, _, err := dao.GetBlock(hash) block, _, err := dao.GetBlock(hash)
require.Error(t, err) require.Error(t, err)
@ -216,7 +216,7 @@ func TestGetBlock_NotExists(t *testing.T) {
} }
func TestPutGetBlock(t *testing.T) { func TestPutGetBlock(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
b := &block.Block{ b := &block.Block{
Base: block.Base{ Base: block.Base{
Script: transaction.Witness{ Script: transaction.Witness{
@ -235,14 +235,14 @@ func TestPutGetBlock(t *testing.T) {
} }
func TestGetVersion_NoVersion(t *testing.T) { func TestGetVersion_NoVersion(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
version, err := dao.GetVersion() version, err := dao.GetVersion()
require.Error(t, err) require.Error(t, err)
require.Equal(t, "", version) require.Equal(t, "", version)
} }
func TestGetVersion(t *testing.T) { func TestGetVersion(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
err := dao.PutVersion("testVersion") err := dao.PutVersion("testVersion")
require.NoError(t, err) require.NoError(t, err)
version, err := dao.GetVersion() version, err := dao.GetVersion()
@ -251,14 +251,14 @@ func TestGetVersion(t *testing.T) {
} }
func TestGetCurrentHeaderHeight_NoHeader(t *testing.T) { func TestGetCurrentHeaderHeight_NoHeader(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
height, err := dao.GetCurrentBlockHeight() height, err := dao.GetCurrentBlockHeight()
require.Error(t, err) require.Error(t, err)
require.Equal(t, uint32(0), height) require.Equal(t, uint32(0), height)
} }
func TestGetCurrentHeaderHeight_Store(t *testing.T) { func TestGetCurrentHeaderHeight_Store(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
b := &block.Block{ b := &block.Block{
Base: block.Base{ Base: block.Base{
Script: transaction.Witness{ Script: transaction.Witness{
@ -275,7 +275,7 @@ func TestGetCurrentHeaderHeight_Store(t *testing.T) {
} }
func TestStoreAsTransaction(t *testing.T) { func TestStoreAsTransaction(t *testing.T) {
dao := newSimpleDao(storage.NewMemoryStore()) dao := NewSimple(storage.NewMemoryStore())
tx := &transaction.Transaction{Type: transaction.IssueType, Data: &transaction.IssueTX{}} tx := &transaction.Transaction{Type: transaction.IssueType, Data: &transaction.IssueTX{}}
hash := tx.Hash() hash := tx.Hash()
err := dao.StoreAsTransaction(tx, 0) err := dao.StoreAsTransaction(tx, 0)

View file

@ -3,6 +3,7 @@ package core
import ( import (
"testing" "testing"
"github.com/nspcc-dev/neo-go/pkg/core/dao"
"github.com/nspcc-dev/neo-go/pkg/core/storage" "github.com/nspcc-dev/neo-go/pkg/core/storage"
"github.com/nspcc-dev/neo-go/pkg/smartcontract/trigger" "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"
@ -15,10 +16,10 @@ import (
func TestGetPrice(t *testing.T) { func TestGetPrice(t *testing.T) {
bc := newTestChain(t) bc := newTestChain(t)
defer bc.Close() defer bc.Close()
sdao := newSimpleDao(storage.NewMemoryStore()) sdao := dao.NewSimple(storage.NewMemoryStore())
systemInterop := bc.newInteropContext(trigger.Application, sdao, nil, nil) systemInterop := bc.newInteropContext(trigger.Application, sdao, nil, nil)
v := bc.spawnVMWithInterops(systemInterop) v := systemInterop.SpawnVM()
v.SetPriceGetter(getPrice) v.SetPriceGetter(getPrice)
t.Run("Neo.Asset.Create", func(t *testing.T) { t.Run("Neo.Asset.Create", func(t *testing.T) {

View file

@ -663,7 +663,7 @@ func (ic *interopContext) assetCreate(v *vm.VM) error {
} }
err = ic.dao.PutAssetState(asset) err = ic.dao.PutAssetState(asset)
if err != nil { if err != nil {
return gherr.Wrap(err, "failed to store asset") return gherr.Wrap(err, "failed to Store asset")
} }
v.Estack().PushVal(vm.NewInteropItem(asset)) v.Estack().PushVal(vm.NewInteropItem(asset))
return nil return nil
@ -783,7 +783,7 @@ func (ic *interopContext) assetRenew(v *vm.VM) error {
asset.Expiration = uint32(expiration) asset.Expiration = uint32(expiration)
err = ic.dao.PutAssetState(asset) err = ic.dao.PutAssetState(asset)
if err != nil { if err != nil {
return gherr.Wrap(err, "failed to store asset") return gherr.Wrap(err, "failed to Store asset")
} }
v.Estack().PushVal(expiration) v.Estack().PushVal(expiration)
return nil return nil

View file

@ -5,6 +5,7 @@ import (
"testing" "testing"
"github.com/nspcc-dev/neo-go/pkg/core/block" "github.com/nspcc-dev/neo-go/pkg/core/block"
"github.com/nspcc-dev/neo-go/pkg/core/dao"
"github.com/nspcc-dev/neo-go/pkg/core/state" "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/storage"
"github.com/nspcc-dev/neo-go/pkg/core/transaction" "github.com/nspcc-dev/neo-go/pkg/core/transaction"
@ -123,7 +124,7 @@ func TestHeaderGetVersion_Negative(t *testing.T) {
block := newDumbBlock() block := newDumbBlock()
chain := newTestChain(t) chain := newTestChain(t)
defer chain.Close() defer chain.Close()
context := chain.newInteropContext(trigger.Application, newSimpleDao(storage.NewMemoryStore()), block, nil) context := chain.newInteropContext(trigger.Application, dao.NewSimple(storage.NewMemoryStore()), block, nil)
v.Estack().PushVal(vm.NewBoolItem(false)) v.Estack().PushVal(vm.NewBoolItem(false))
err := context.headerGetVersion(v) err := context.headerGetVersion(v)
@ -219,7 +220,7 @@ func TestWitnessGetVerificationScript(t *testing.T) {
chain := newTestChain(t) chain := newTestChain(t)
defer chain.Close() defer chain.Close()
context := chain.newInteropContext(trigger.Application, newSimpleDao(storage.NewMemoryStore()), nil, nil) context := chain.newInteropContext(trigger.Application, dao.NewSimple(storage.NewMemoryStore()), nil, nil)
v.Estack().PushVal(vm.NewInteropItem(&witness)) v.Estack().PushVal(vm.NewInteropItem(&witness))
err := context.witnessGetVerificationScript(v) err := context.witnessGetVerificationScript(v)
require.NoError(t, err) require.NoError(t, err)
@ -462,7 +463,7 @@ func createVMAndPushBlock(t *testing.T) (*vm.VM, *block.Block, *interopContext,
v := vm.New() v := vm.New()
block := newDumbBlock() block := newDumbBlock()
chain := newTestChain(t) chain := newTestChain(t)
context := chain.newInteropContext(trigger.Application, newSimpleDao(storage.NewMemoryStore()), block, nil) context := chain.newInteropContext(trigger.Application, dao.NewSimple(storage.NewMemoryStore()), block, nil)
v.Estack().PushVal(vm.NewInteropItem(block)) v.Estack().PushVal(vm.NewInteropItem(block))
return v, block, context, chain return v, block, context, chain
} }
@ -492,7 +493,7 @@ func createVMAndAssetState(t *testing.T) (*vm.VM, *state.Asset, *interopContext,
} }
chain := newTestChain(t) chain := newTestChain(t)
context := chain.newInteropContext(trigger.Application, newSimpleDao(storage.NewMemoryStore()), nil, nil) context := chain.newInteropContext(trigger.Application, dao.NewSimple(storage.NewMemoryStore()), nil, nil)
return v, assetState, context, chain return v, assetState, context, chain
} }
@ -511,7 +512,7 @@ func createVMAndContractState(t *testing.T) (*vm.VM, *state.Contract, *interopCo
} }
chain := newTestChain(t) chain := newTestChain(t)
context := chain.newInteropContext(trigger.Application, newSimpleDao(storage.NewMemoryStore()), nil, nil) context := chain.newInteropContext(trigger.Application, dao.NewSimple(storage.NewMemoryStore()), nil, nil)
return v, contractState, context, chain return v, contractState, context, chain
} }
@ -526,7 +527,7 @@ func createVMAndAccState(t *testing.T) (*vm.VM, *state.Account, *interopContext,
require.NoError(t, err) require.NoError(t, err)
chain := newTestChain(t) chain := newTestChain(t)
context := chain.newInteropContext(trigger.Application, newSimpleDao(storage.NewMemoryStore()), nil, nil) context := chain.newInteropContext(trigger.Application, dao.NewSimple(storage.NewMemoryStore()), nil, nil)
return v, accountState, context, chain return v, accountState, context, chain
} }
@ -557,6 +558,6 @@ func createVMAndTX(t *testing.T) (*vm.VM, *transaction.Transaction, *interopCont
tx.Inputs = inputs tx.Inputs = inputs
tx.Outputs = outputs tx.Outputs = outputs
chain := newTestChain(t) chain := newTestChain(t)
context := chain.newInteropContext(trigger.Application, newSimpleDao(storage.NewMemoryStore()), nil, tx) context := chain.newInteropContext(trigger.Application, dao.NewSimple(storage.NewMemoryStore()), nil, tx)
return v, tx, context, chain return v, tx, context, chain
} }

View file

@ -6,6 +6,7 @@ import (
"math" "math"
"github.com/nspcc-dev/neo-go/pkg/core/block" "github.com/nspcc-dev/neo-go/pkg/core/block"
"github.com/nspcc-dev/neo-go/pkg/core/dao"
"github.com/nspcc-dev/neo-go/pkg/core/state" "github.com/nspcc-dev/neo-go/pkg/core/state"
"github.com/nspcc-dev/neo-go/pkg/core/transaction" "github.com/nspcc-dev/neo-go/pkg/core/transaction"
"github.com/nspcc-dev/neo-go/pkg/crypto/keys" "github.com/nspcc-dev/neo-go/pkg/crypto/keys"
@ -100,7 +101,7 @@ func (ic *interopContext) bcGetHeight(v *vm.VM) error {
// getTransactionAndHeight gets parameter from the vm evaluation stack and // getTransactionAndHeight gets parameter from the vm evaluation stack and
// returns transaction and its height if it's present in the blockchain. // returns transaction and its height if it's present in the blockchain.
func getTransactionAndHeight(cd *cachedDao, v *vm.VM) (*transaction.Transaction, uint32, error) { func getTransactionAndHeight(cd *dao.Cached, v *vm.VM) (*transaction.Transaction, uint32, error) {
hashbytes := v.Estack().Pop().Bytes() hashbytes := v.Estack().Pop().Bytes()
hash, err := util.Uint256DecodeBytesBE(hashbytes) hash, err := util.Uint256DecodeBytesBE(hashbytes)
if err != nil { if err != nil {

View file

@ -11,9 +11,12 @@ import (
"sort" "sort"
"github.com/nspcc-dev/neo-go/pkg/core/block" "github.com/nspcc-dev/neo-go/pkg/core/block"
"github.com/nspcc-dev/neo-go/pkg/core/dao"
"github.com/nspcc-dev/neo-go/pkg/core/state" "github.com/nspcc-dev/neo-go/pkg/core/state"
"github.com/nspcc-dev/neo-go/pkg/core/transaction" "github.com/nspcc-dev/neo-go/pkg/core/transaction"
"github.com/nspcc-dev/neo-go/pkg/smartcontract"
"github.com/nspcc-dev/neo-go/pkg/smartcontract/trigger" "github.com/nspcc-dev/neo-go/pkg/smartcontract/trigger"
"github.com/nspcc-dev/neo-go/pkg/util"
"github.com/nspcc-dev/neo-go/pkg/vm" "github.com/nspcc-dev/neo-go/pkg/vm"
"go.uber.org/zap" "go.uber.org/zap"
) )
@ -23,17 +26,34 @@ type interopContext struct {
trigger trigger.Type trigger trigger.Type
block *block.Block block *block.Block
tx *transaction.Transaction tx *transaction.Transaction
dao *cachedDao dao *dao.Cached
notifications []state.NotificationEvent notifications []state.NotificationEvent
log *zap.Logger log *zap.Logger
} }
func newInteropContext(trigger trigger.Type, bc Blockchainer, d dao, block *block.Block, tx *transaction.Transaction, log *zap.Logger) *interopContext { func newInteropContext(trigger trigger.Type, bc Blockchainer, d dao.DAO, block *block.Block, tx *transaction.Transaction, log *zap.Logger) *interopContext {
dao := newCachedDao(d) dao := dao.NewCached(d)
nes := make([]state.NotificationEvent, 0) nes := make([]state.NotificationEvent, 0)
return &interopContext{bc, trigger, block, tx, dao, nes, log} return &interopContext{bc, trigger, block, tx, dao, nes, log}
} }
// SpawnVM returns a VM with script getter and interop functions set
// up for current blockchain.
func (ic *interopContext) SpawnVM() *vm.VM {
vm := vm.New()
vm.SetScriptGetter(func(hash util.Uint160) ([]byte, bool) {
cs, err := ic.dao.GetContractState(hash)
if err != nil {
return nil, false
}
hasDynamicInvoke := (cs.Properties & smartcontract.HasDynamicInvoke) != 0
return cs.Script, hasDynamicInvoke
})
vm.RegisterInteropGetter(ic.getSystemInterop)
vm.RegisterInteropGetter(ic.getNeoInterop)
return vm
}
// interopedFunction binds function name, id with the function itself and price, // interopedFunction binds function name, id with the function itself and price,
// it's supposed to be inited once for all interopContexts, so it doesn't use // it's supposed to be inited once for all interopContexts, so it doesn't use
// vm.InteropFuncPrice directly. // vm.InteropFuncPrice directly.

View file

@ -5,6 +5,7 @@ import (
"runtime" "runtime"
"testing" "testing"
"github.com/nspcc-dev/neo-go/pkg/core/dao"
"github.com/nspcc-dev/neo-go/pkg/core/storage" "github.com/nspcc-dev/neo-go/pkg/core/storage"
"github.com/nspcc-dev/neo-go/pkg/smartcontract/trigger" "github.com/nspcc-dev/neo-go/pkg/smartcontract/trigger"
"github.com/nspcc-dev/neo-go/pkg/vm" "github.com/nspcc-dev/neo-go/pkg/vm"
@ -16,7 +17,7 @@ func testNonInterop(t *testing.T, value interface{}, f func(*interopContext, *vm
v.Estack().PushVal(value) v.Estack().PushVal(value)
chain := newTestChain(t) chain := newTestChain(t)
defer chain.Close() defer chain.Close()
context := chain.newInteropContext(trigger.Application, newSimpleDao(storage.NewMemoryStore()), nil, nil) context := chain.newInteropContext(trigger.Application, dao.NewSimple(storage.NewMemoryStore()), nil, nil)
require.Error(t, f(context, v)) require.Error(t, f(context, v))
} }

View file

@ -1,8 +0,0 @@
package core
// slice attaches the methods of Interface to []int, sorting in increasing order.
type slice []uint32
func (p slice) Len() int { return len(p) }
func (p slice) Less(i, j int) bool { return p[i] < p[j] }
func (p slice) Swap(i, j int) { p[i], p[j] = p[j], p[i] }