From 7dc8fc443f2f32fefdb00945a832aede6350b2b9 Mon Sep 17 00:00:00 2001 From: Roman Khimov Date: Wed, 16 Feb 2022 23:33:53 +0300 Subject: [PATCH] dao: simplify buffer management for private DAO Private DAO is only used in a single thread which means we can safely reuse key/data buffers most of the time and handle it all in DAO. Doesn't affect any benchmarks. --- pkg/core/blockchain.go | 20 ++---- pkg/core/blockchain_test.go | 2 +- pkg/core/dao/dao.go | 132 ++++++++++++++++++++++------------- pkg/core/dao/dao_test.go | 19 ++--- pkg/core/statesync/module.go | 7 +- 5 files changed, 102 insertions(+), 78 deletions(-) diff --git a/pkg/core/blockchain.go b/pkg/core/blockchain.go index 12e51cfee..3b9d0c8a2 100644 --- a/pkg/core/blockchain.go +++ b/pkg/core/blockchain.go @@ -499,7 +499,6 @@ 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: @@ -530,8 +529,7 @@ func (bc *Blockchain) jumpToStateInternal(p uint32, stage stateJumpStage) error // 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 { - 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) } @@ -559,8 +557,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() - bc.dao.StoreAsCurrentBlock(block, writeBuf) + bc.dao.StoreAsCurrentBlock(block) bc.topBlock.Store(block) atomic.StoreUint32(&bc.blockHeight, p) atomic.StoreUint32(&bc.persistedHeight, p) @@ -1006,14 +1003,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) ) - kvcache.StoreAsCurrentBlock(block, writeBuf) - writeBuf.Reset() + kvcache.StoreAsCurrentBlock(block) if bc.config.RemoveUntraceableBlocks { var start, stop uint32 if bc.config.P2PStateExchangeExtensions { @@ -1031,13 +1026,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 { @@ -1048,7 +1042,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 { @@ -1060,17 +1054,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 { diff --git a/pkg/core/blockchain_test.go b/pkg/core/blockchain_test.go index 8272a130d..14c687a19 100644 --- a/pkg/core/blockchain_test.go +++ b/pkg/core/blockchain_test.go @@ -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) { diff --git a/pkg/core/dao/dao.go b/pkg/core/dao/dao.go index f402e44c1..0e95b74d7 100644 --- a/pkg/core/dao/dao.go +++ b/pkg/core/dao/dao.go @@ -16,6 +16,7 @@ import ( "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. @@ -31,6 +32,8 @@ var ( type Simple struct { Version Version Store *storage.MemCachedStore + keyBuf []byte + dataBuf *io.BufBinWriter } // NewSimple creates new simple dao using provided backend store. @@ -69,6 +72,16 @@ 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 } @@ -93,8 +106,8 @@ func (dao *Simple) putWithBuffer(entity io.Serializable, key []byte, buf *io.Buf 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 @@ -102,18 +115,18 @@ func makeContractIDKey(id int32) []byte { // DeleteContractID deletes contract's id to hash mapping. func (dao *Simple) DeleteContractID(id int32) { - dao.Store.Delete(makeContractIDKey(id)) + 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) { - dao.Store.Put(makeContractIDKey(id), hash.BytesBE()) + 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 @@ -121,9 +134,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 { @@ -134,20 +154,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 { @@ -163,7 +182,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{ @@ -185,7 +204,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{ @@ -205,7 +224,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 { @@ -218,7 +237,7 @@ 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) { - key := getTokenTransferLogKey(acc, start, index, isNEP11) + key := dao.getTokenTransferLogKey(acc, start, index, isNEP11) dao.Store.Put(key, lg.Raw) } @@ -226,10 +245,17 @@ func (dao *Simple) PutTokenTransferLog(acc util.Uint160, start uint64, index uin // -- 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 @@ -272,7 +298,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 } @@ -282,14 +308,14 @@ 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) { - stKey := makeStorageItemKey(dao.Version.StoragePrefix, id, key) + 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) { - stKey := makeStorageItemKey(dao.Version.StoragePrefix, id, key) + stKey := dao.makeStorageItemKey(id, key) dao.Store.Delete(stKey) } @@ -320,7 +346,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) }) @@ -330,15 +356,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 @@ -350,7 +376,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 @@ -519,7 +545,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 @@ -560,14 +586,14 @@ func (dao *Simple) PutCurrentHeader(hashAndIndex []byte) { // PutStateSyncPoint stores current state synchronisation point P. func (dao *Simple) PutStateSyncPoint(p uint32) { - buf := make([]byte, 4) + buf := dao.getKeyBuf(4) // It's very small, no point in using BufBinWriter. binary.LittleEndian.PutUint32(buf, p) dao.Store.Put(storage.SYSStateSyncPoint.Bytes(), buf) } // PutStateSyncCurrentBlockHeight stores current block height during state synchronisation process. func (dao *Simple) PutStateSyncCurrentBlockHeight(h uint32) { - buf := make([]byte, 4) + buf := dao.getKeyBuf(4) // It's very small, no point in using BufBinWriter. binary.LittleEndian.PutUint32(buf, h) dao.Store.Put(storage.SYSStateSyncCurrentBlockHeight.Bytes(), buf) } @@ -589,7 +615,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 @@ -606,13 +632,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 { @@ -634,10 +658,8 @@ func (dao *Simple) StoreAsBlock(block *block.Block, aer1 *state.AppExecResult, a // 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, w *io.BufBinWriter) error { - key := make([]byte, util.Uint256Size+1) - key[0] = byte(storage.DataExecutable) - copy(key[1:], h.BytesBE()) +func (dao *Simple) DeleteBlock(h util.Uint256) error { + key := dao.makeExecutableKey(h) bs, err := dao.Store.Get(key) if err != nil { return err @@ -652,9 +674,7 @@ 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) @@ -681,10 +701,8 @@ func (dao *Simple) DeleteBlock(h util.Uint256, w *io.BufBinWriter) error { // StoreAsCurrentBlock stores a hash of the given block with prefix // SYSCurrentBlock. It can reuse given buffer for the purpose of value // serialization. -func (dao *Simple) StoreAsCurrentBlock(block *block.Block, buf *io.BufBinWriter) { - 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) @@ -694,11 +712,10 @@ func (dao *Simple) StoreAsCurrentBlock(block *block.Block, buf *io.BufBinWriter) // 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) @@ -727,6 +744,21 @@ func (dao *Simple) StoreAsTransaction(tx *transaction.Transaction, index uint32, 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) { diff --git a/pkg/core/dao/dao_test.go b/pkg/core/dao/dao_test.go index 4b7397235..feb2e2055 100644 --- a/pkg/core/dao/dao_test.go +++ b/pkg/core/dao/dao_test.go @@ -98,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) @@ -164,7 +164,7 @@ func TestGetCurrentHeaderHeight_Store(t *testing.T) { }, }, } - dao.StoreAsCurrentBlock(b, nil) + dao.StoreAsCurrentBlock(b) height, err := dao.GetCurrentBlockHeight() require.NoError(t, err) require.Equal(t, uint32(0), height) @@ -185,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) @@ -216,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)) @@ -265,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() } @@ -275,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) } diff --git a/pkg/core/statesync/module.go b/pkg/core/statesync/module.go index ca616167c..38d603e1d 100644 --- a/pkg/core/statesync/module.go +++ b/pkg/core/statesync/module.go @@ -330,19 +330,16 @@ func (s *Module) AddBlock(block *block.Block) error { } } cache := s.dao.GetPrivate() - writeBuf := io.NewBufBinWriter() - if err := cache.StoreAsBlock(block, nil, nil, writeBuf); err != nil { + if err := cache.StoreAsBlock(block, nil, nil); err != nil { return err } - writeBuf.Reset() 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()