From 1f852bc04dc4ff09b9457ec523023c364c78ea6e Mon Sep 17 00:00:00 2001 From: dustinxie Date: Thu, 5 Dec 2024 15:15:04 -0800 Subject: [PATCH 01/21] [blockindex] remove DeleteTipBlock() (#4495) --- blockchain/blockdao/blockindexer.go | 1 - blockindex/contractstaking/indexer.go | 5 -- blockindex/indexer.go | 71 ++----------------- blockindex/indexer_test.go | 48 ------------- blockindex/sync_indexers.go | 10 --- state/factory/factory.go | 5 -- state/factory/statedb.go | 4 -- systemcontractindex/stakingindex/index.go | 6 -- test/mock/mock_blockdao/mock_blockindexer.go | 14 ---- .../mock_blockindexer_withstart.go | 14 ---- test/mock/mock_blockindex/mock_blockindex.go | 14 ---- test/mock/mock_blockindex/mock_indexer.go | 14 ---- test/mock/mock_factory/mock_factory.go | 14 ---- 13 files changed, 7 insertions(+), 213 deletions(-) diff --git a/blockchain/blockdao/blockindexer.go b/blockchain/blockdao/blockindexer.go index 526c092c6b..7b3d40c3a1 100644 --- a/blockchain/blockdao/blockindexer.go +++ b/blockchain/blockdao/blockindexer.go @@ -25,7 +25,6 @@ type ( Stop(ctx context.Context) error Height() (uint64, error) PutBlock(context.Context, *block.Block) error - DeleteTipBlock(context.Context, *block.Block) error } // BlockIndexerWithStart defines an interface to accept block to build index from a start height diff --git a/blockindex/contractstaking/indexer.go b/blockindex/contractstaking/indexer.go index caa9df0932..f3ae9ea560 100644 --- a/blockindex/contractstaking/indexer.go +++ b/blockindex/contractstaking/indexer.go @@ -195,11 +195,6 @@ func (s *Indexer) PutBlock(ctx context.Context, blk *block.Block) error { return s.commit(handler, blk.Height()) } -// DeleteTipBlock deletes the tip block from indexer -func (s *Indexer) DeleteTipBlock(context.Context, *block.Block) error { - return errors.New("not implemented") -} - func (s *Indexer) commit(handler *contractStakingEventHandler, height uint64) error { batch, delta := handler.Result() // update cache diff --git a/blockindex/indexer.go b/blockindex/indexer.go index c9e96b9949..c7cd07d845 100644 --- a/blockindex/indexer.go +++ b/blockindex/indexer.go @@ -49,7 +49,6 @@ type ( Stop(context.Context) error PutBlock(context.Context, *block.Block) error PutBlocks(context.Context, []*block.Block) error - DeleteTipBlock(context.Context, *block.Block) error Height() (uint64, error) GetBlockHash(height uint64) (hash.Hash256, error) GetBlockHeight(hash hash.Hash256) (uint64, error) @@ -143,49 +142,6 @@ func (x *blockIndexer) PutBlock(ctx context.Context, blk *block.Block) error { return x.commit() } -// DeleteTipBlock deletes a block's index -func (x *blockIndexer) DeleteTipBlock(ctx context.Context, blk *block.Block) error { - x.mutex.Lock() - defer x.mutex.Unlock() - - // the block to be deleted must be exactly current top, otherwise counting index would not work correctly - height := blk.Height() - if height != x.tbk.Size()-1 { - return errors.Wrapf(db.ErrInvalid, "wrong block height %d, expecting %d", height, x.tbk.Size()-1) - } - // delete hash --> height - hash := blk.HashBlock() - x.batch.Delete(_blockHashToHeightNS, hash[_hashOffset:], fmt.Sprintf("failed to delete block at height %d", height)) - // delete from total block index - if err := x.tbk.Revert(1); err != nil { - return err - } - - // delete action index - fCtx := protocol.MustGetFeatureCtx(protocol.WithFeatureCtx(protocol.WithBlockCtx(ctx, protocol.BlockCtx{ - BlockHeight: blk.Height(), - }))) - for _, selp := range blk.Actions { - actHash, err := selp.Hash() - if err != nil { - return err - } - x.batch.Delete(_actionToBlockHashNS, actHash[_hashOffset:], fmt.Sprintf("failed to delete action hash %x", actHash)) - if err := x.indexAction(actHash, selp, false, fCtx.TolerateLegacyAddress); err != nil { - return err - } - } - // delete from total action index - if err := x.tac.Revert(uint64(len(blk.Actions))); err != nil { - return err - } - if err := x.kvStore.WriteBatch(x.batch); err != nil { - return err - } - x.batch.Clear() - return nil -} - // Height return the blockchain height func (x *blockIndexer) Height() (uint64, error) { x.mutex.RLock() @@ -340,7 +296,7 @@ func (x *blockIndexer) putBlock(ctx context.Context, blk *block.Block) error { if err := x.tac.Add(actHash[:], true); err != nil { return err } - if err := x.indexAction(actHash, selp, true, fCtx.TolerateLegacyAddress); err != nil { + if err := x.indexAction(actHash, selp, fCtx.TolerateLegacyAddress); err != nil { return err } } @@ -377,10 +333,7 @@ func (x *blockIndexer) commit() error { // getIndexerForAddr returns the counting indexer for an address // if batch is true, the indexer will be placed into a dirty map, to be committed later -func (x *blockIndexer) getIndexerForAddr(addr []byte, batch bool) (db.CountingIndex, error) { - if !batch { - return db.NewCountingIndexNX(x.kvStore, addr) - } +func (x *blockIndexer) getIndexerForAddr(addr []byte) (db.CountingIndex, error) { address := hash.BytesToHash160(addr) indexer, ok := x.dirtyAddr[address] if !ok { @@ -399,19 +352,14 @@ func (x *blockIndexer) getIndexerForAddr(addr []byte, batch bool) (db.CountingIn } // indexAction builds index for an action -func (x *blockIndexer) indexAction(actHash hash.Hash256, elp *action.SealedEnvelope, insert, tolerateLegacyAddress bool) error { +func (x *blockIndexer) indexAction(actHash hash.Hash256, elp *action.SealedEnvelope, tolerateLegacyAddress bool) error { // add to sender's index callerAddrBytes := elp.SrcPubkey().Hash() - sender, err := x.getIndexerForAddr(callerAddrBytes, insert) + sender, err := x.getIndexerForAddr(callerAddrBytes) if err != nil { return err } - if insert { - err = sender.Add(actHash[:], insert) - } else { - err = sender.Revert(1) - } - if err != nil { + if err = sender.Add(actHash[:], true); err != nil { return err } @@ -437,14 +385,9 @@ func (x *blockIndexer) indexAction(actHash hash.Hash256, elp *action.SealedEnvel } // add to recipient's index - recipient, err := x.getIndexerForAddr(dstAddrBytes, insert) + recipient, err := x.getIndexerForAddr(dstAddrBytes) if err != nil { return err } - if insert { - err = recipient.Add(actHash[:], insert) - } else { - err = recipient.Revert(1) - } - return err + return recipient.Add(actHash[:], true) } diff --git a/blockindex/indexer_test.go b/blockindex/indexer_test.go index a8e1769e9e..cbbb2ad942 100644 --- a/blockindex/indexer_test.go +++ b/blockindex/indexer_test.go @@ -258,54 +258,6 @@ func TestIndexer(t *testing.T) { require.NoError(err) require.EqualValues(len(indexTests[0].actions[i].hashes), actionCount) } - - // delete tip block one by one, verify address/action after each deletion - for i := range indexTests { - if i == 0 { - // tests[0] is the whole address/action data at block height 3 - continue - } - - require.NoError(indexer.DeleteTipBlock(ctx, blks[3-i])) - tipHeight, err := indexer.Height() - require.NoError(err) - require.EqualValues(uint64(3-i), tipHeight) - h, err := indexer.GetBlockHash(tipHeight) - require.NoError(err) - if i <= 2 { - require.Equal(blks[2-i].HashBlock(), h) - } else { - require.Equal(hash.ZeroHash256, h) - } - - total, err := indexer.GetTotalActions() - require.NoError(err) - require.EqualValues(indexTests[i].total, total) - if total > 0 { - _, err = indexer.GetActionHashFromIndex(1, total) - require.Equal(db.ErrInvalid, errors.Cause(err)) - actions, err := indexer.GetActionHashFromIndex(0, total) - require.NoError(err) - require.Equal(actions, indexTests[i].hashTotal) - } - for j := range indexTests[i].actions { - actionCount, err := indexer.GetActionCountByAddress(indexTests[i].actions[j].addr) - require.NoError(err) - require.EqualValues(len(indexTests[i].actions[j].hashes), actionCount) - if actionCount > 0 { - actions, err := indexer.GetActionsByAddress(indexTests[i].actions[j].addr, 0, actionCount) - require.NoError(err) - require.Equal(actions, indexTests[i].actions[j].hashes) - } - } - } - - tipHeight, err := indexer.Height() - require.NoError(err) - require.EqualValues(0, tipHeight) - total, err := indexer.GetTotalActions() - require.NoError(err) - require.EqualValues(0, total) } t.Run("In-memory KV indexer", func(t *testing.T) { diff --git a/blockindex/sync_indexers.go b/blockindex/sync_indexers.go index 9006c73258..f35cc6247f 100644 --- a/blockindex/sync_indexers.go +++ b/blockindex/sync_indexers.go @@ -69,16 +69,6 @@ func (ig *SyncIndexers) PutBlock(ctx context.Context, blk *block.Block) error { return nil } -// DeleteTipBlock deletes the tip block from the indexers in the group -func (ig *SyncIndexers) DeleteTipBlock(ctx context.Context, blk *block.Block) error { - for _, indexer := range ig.indexers { - if err := indexer.DeleteTipBlock(ctx, blk); err != nil { - return err - } - } - return nil -} - // StartHeight returns the minimum start height of the indexers in the group func (ig *SyncIndexers) StartHeight() uint64 { return ig.minStartHeight diff --git a/state/factory/factory.go b/state/factory/factory.go index 2da378d74a..228f5ca268 100644 --- a/state/factory/factory.go +++ b/state/factory/factory.go @@ -86,7 +86,6 @@ type ( // NewBlockBuilder creates block builder NewBlockBuilder(context.Context, actpool.ActPool, func(action.Envelope) (*action.SealedEnvelope, error)) (*block.Builder, error) PutBlock(context.Context, *block.Block) error - DeleteTipBlock(context.Context, *block.Block) error WorkingSet(context.Context) (protocol.StateManager, error) WorkingSetAtHeight(context.Context, uint64) (protocol.StateManager, error) } @@ -478,10 +477,6 @@ func (sf *factory) PutBlock(ctx context.Context, blk *block.Block) error { return nil } -func (sf *factory) DeleteTipBlock(_ context.Context, _ *block.Block) error { - return errors.Wrap(ErrNotSupported, "cannot delete tip block from factory") -} - // State returns a confirmed state in the state factory func (sf *factory) State(s interface{}, opts ...protocol.StateOption) (uint64, error) { sf.mutex.RLock() diff --git a/state/factory/statedb.go b/state/factory/statedb.go index 5dc4e08176..cf5fc01f84 100644 --- a/state/factory/statedb.go +++ b/state/factory/statedb.go @@ -321,10 +321,6 @@ func (sdb *stateDB) PutBlock(ctx context.Context, blk *block.Block) error { return nil } -func (sdb *stateDB) DeleteTipBlock(_ context.Context, _ *block.Block) error { - return errors.Wrap(ErrNotSupported, "cannot delete tip block from state db") -} - // State returns a confirmed state in the state factory func (sdb *stateDB) State(s interface{}, opts ...protocol.StateOption) (uint64, error) { cfg, err := processOptions(opts...) diff --git a/systemcontractindex/stakingindex/index.go b/systemcontractindex/stakingindex/index.go index b5e2b13c03..b0301f6347 100644 --- a/systemcontractindex/stakingindex/index.go +++ b/systemcontractindex/stakingindex/index.go @@ -40,7 +40,6 @@ type ( BucketsByCandidate(candidate address.Address, height uint64) ([]*VoteBucket, error) TotalBucketCount(height uint64) (uint64, error) PutBlock(ctx context.Context, blk *block.Block) error - DeleteTipBlock(ctx context.Context, blk *block.Block) error } // Indexer is the staking indexer Indexer struct { @@ -213,11 +212,6 @@ func (s *Indexer) PutBlock(ctx context.Context, blk *block.Block) error { return s.commit(handler, blk.Height()) } -// DeleteTipBlock deletes the tip block from indexer -func (s *Indexer) DeleteTipBlock(context.Context, *block.Block) error { - return errors.New("not implemented") -} - func (s *Indexer) commit(handler *eventHandler, height uint64) error { delta, dirty := handler.Finalize() // update db diff --git a/test/mock/mock_blockdao/mock_blockindexer.go b/test/mock/mock_blockdao/mock_blockindexer.go index 14d2b35096..b9ce6c019f 100644 --- a/test/mock/mock_blockdao/mock_blockindexer.go +++ b/test/mock/mock_blockdao/mock_blockindexer.go @@ -35,20 +35,6 @@ func (m *MockBlockIndexer) EXPECT() *MockBlockIndexerMockRecorder { return m.recorder } -// DeleteTipBlock mocks base method. -func (m *MockBlockIndexer) DeleteTipBlock(arg0 context.Context, arg1 *block.Block) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "DeleteTipBlock", arg0, arg1) - ret0, _ := ret[0].(error) - return ret0 -} - -// DeleteTipBlock indicates an expected call of DeleteTipBlock. -func (mr *MockBlockIndexerMockRecorder) DeleteTipBlock(arg0, arg1 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteTipBlock", reflect.TypeOf((*MockBlockIndexer)(nil).DeleteTipBlock), arg0, arg1) -} - // Height mocks base method. func (m *MockBlockIndexer) Height() (uint64, error) { m.ctrl.T.Helper() diff --git a/test/mock/mock_blockdao/mock_blockindexer_withstart.go b/test/mock/mock_blockdao/mock_blockindexer_withstart.go index 2fe01a3da5..25fc7848d6 100644 --- a/test/mock/mock_blockdao/mock_blockindexer_withstart.go +++ b/test/mock/mock_blockdao/mock_blockindexer_withstart.go @@ -35,20 +35,6 @@ func (m *MockBlockIndexerWithStart) EXPECT() *MockBlockIndexerWithStartMockRecor return m.recorder } -// DeleteTipBlock mocks base method. -func (m *MockBlockIndexerWithStart) DeleteTipBlock(arg0 context.Context, arg1 *block.Block) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "DeleteTipBlock", arg0, arg1) - ret0, _ := ret[0].(error) - return ret0 -} - -// DeleteTipBlock indicates an expected call of DeleteTipBlock. -func (mr *MockBlockIndexerWithStartMockRecorder) DeleteTipBlock(arg0, arg1 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteTipBlock", reflect.TypeOf((*MockBlockIndexerWithStart)(nil).DeleteTipBlock), arg0, arg1) -} - // Height mocks base method. func (m *MockBlockIndexerWithStart) Height() (uint64, error) { m.ctrl.T.Helper() diff --git a/test/mock/mock_blockindex/mock_blockindex.go b/test/mock/mock_blockindex/mock_blockindex.go index efd3f7bece..8fcd9ae8d4 100644 --- a/test/mock/mock_blockindex/mock_blockindex.go +++ b/test/mock/mock_blockindex/mock_blockindex.go @@ -52,20 +52,6 @@ func (mr *MockBloomFilterIndexerMockRecorder) BlockFilterByHeight(arg0 interface return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "BlockFilterByHeight", reflect.TypeOf((*MockBloomFilterIndexer)(nil).BlockFilterByHeight), arg0) } -// DeleteTipBlock mocks base method. -func (m *MockBloomFilterIndexer) DeleteTipBlock(arg0 context.Context, arg1 *block.Block) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "DeleteTipBlock", arg0, arg1) - ret0, _ := ret[0].(error) - return ret0 -} - -// DeleteTipBlock indicates an expected call of DeleteTipBlock. -func (mr *MockBloomFilterIndexerMockRecorder) DeleteTipBlock(arg0, arg1 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteTipBlock", reflect.TypeOf((*MockBloomFilterIndexer)(nil).DeleteTipBlock), arg0, arg1) -} - // FilterBlocksInRange mocks base method. func (m *MockBloomFilterIndexer) FilterBlocksInRange(arg0 *logfilter.LogFilter, arg1, arg2, arg3 uint64) ([]uint64, error) { m.ctrl.T.Helper() diff --git a/test/mock/mock_blockindex/mock_indexer.go b/test/mock/mock_blockindex/mock_indexer.go index e564b3df0b..a763b098a8 100644 --- a/test/mock/mock_blockindex/mock_indexer.go +++ b/test/mock/mock_blockindex/mock_indexer.go @@ -37,20 +37,6 @@ func (m *MockIndexer) EXPECT() *MockIndexerMockRecorder { return m.recorder } -// DeleteTipBlock mocks base method. -func (m *MockIndexer) DeleteTipBlock(arg0 context.Context, arg1 *block.Block) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "DeleteTipBlock", arg0, arg1) - ret0, _ := ret[0].(error) - return ret0 -} - -// DeleteTipBlock indicates an expected call of DeleteTipBlock. -func (mr *MockIndexerMockRecorder) DeleteTipBlock(arg0, arg1 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteTipBlock", reflect.TypeOf((*MockIndexer)(nil).DeleteTipBlock), arg0, arg1) -} - // GetActionCountByAddress mocks base method. func (m *MockIndexer) GetActionCountByAddress(arg0 hash.Hash160) (uint64, error) { m.ctrl.T.Helper() diff --git a/test/mock/mock_factory/mock_factory.go b/test/mock/mock_factory/mock_factory.go index a18c978ca3..475f831700 100644 --- a/test/mock/mock_factory/mock_factory.go +++ b/test/mock/mock_factory/mock_factory.go @@ -39,20 +39,6 @@ func (m *MockFactory) EXPECT() *MockFactoryMockRecorder { return m.recorder } -// DeleteTipBlock mocks base method. -func (m *MockFactory) DeleteTipBlock(arg0 context.Context, arg1 *block.Block) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "DeleteTipBlock", arg0, arg1) - ret0, _ := ret[0].(error) - return ret0 -} - -// DeleteTipBlock indicates an expected call of DeleteTipBlock. -func (mr *MockFactoryMockRecorder) DeleteTipBlock(arg0, arg1 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteTipBlock", reflect.TypeOf((*MockFactory)(nil).DeleteTipBlock), arg0, arg1) -} - // Height mocks base method. func (m *MockFactory) Height() (uint64, error) { m.ctrl.T.Helper() From d5df01a9cfeb7db3af3bcddad0592c5199c994f7 Mon Sep 17 00:00:00 2001 From: dustinxie Date: Sun, 8 Dec 2024 18:32:28 -0800 Subject: [PATCH 02/21] [blocksync] remove redundant clean-up (#4505) --- blocksync/blocksync.go | 1 - blocksync/buffer.go | 21 --------------------- 2 files changed, 22 deletions(-) diff --git a/blocksync/blocksync.go b/blocksync/blocksync.go index d06d30dd98..661e11c827 100644 --- a/blocksync/blocksync.go +++ b/blocksync/blocksync.go @@ -290,7 +290,6 @@ func (bs *blockSyncer) ProcessBlock(ctx context.Context, peer string, blk *block } syncedHeight++ } - bs.buf.Cleanup(syncedHeight) log.L().Debug("flush blocks", zap.Uint64("start", tip), zap.Uint64("end", syncedHeight)) if syncedHeight > bs.lastTip { bs.lastTip = syncedHeight diff --git a/blocksync/buffer.go b/blocksync/buffer.go index 459047f522..ea1538f015 100644 --- a/blocksync/buffer.go +++ b/blocksync/buffer.go @@ -7,10 +7,6 @@ package blocksync import ( "sync" - - "go.uber.org/zap" - - "github.com/iotexproject/iotex-core/v2/pkg/log" ) // blockBuffer is used to keep in-coming block in order. @@ -47,23 +43,6 @@ func (b *blockBuffer) Pop(height uint64) []*peerBlock { return blks } -func (b *blockBuffer) Cleanup(height uint64) { - b.mu.Lock() - defer b.mu.Unlock() - - size := len(b.blockQueues) - if size > int(b.bufferSize)*2 { - log.L().Warn("blockBuffer is leaking memory.", zap.Int("bufferSize", size)) - newQueues := map[uint64]*uniQueue{} - for h := range b.blockQueues { - if h > height { - newQueues[h] = b.blockQueues[h] - } - } - b.blockQueues = newQueues - } -} - // AddBlock tries to put given block into buffer and flush buffer into blockchain. func (b *blockBuffer) AddBlock(tipHeight uint64, blk *peerBlock) (bool, uint64) { b.mu.Lock() From 25d345f41a5d904b5892cb68fd33813de000bbf8 Mon Sep 17 00:00:00 2001 From: dustinxie Date: Sun, 8 Dec 2024 23:13:40 -0800 Subject: [PATCH 03/21] [db] implement Get/Put/Delete/Version() for BoltDBVersioned (#4256) --- db/db_versioned.go | 252 +++++++++++++++++++--------- db/db_versioned_test.go | 313 ++++++++++++++++++++++++++++++++++- db/db_versioned_util.go | 3 - db/db_versioned_util_test.go | 26 +++ db/kvstore_versioned.go | 50 ++++++ db/versionpb/version.pb.go | 129 ++------------- db/versionpb/version.proto | 8 - 7 files changed, 566 insertions(+), 215 deletions(-) create mode 100644 db/db_versioned_util_test.go create mode 100644 db/kvstore_versioned.go diff --git a/db/db_versioned.go b/db/db_versioned.go index 9818004778..0d356b9b0d 100644 --- a/db/db_versioned.go +++ b/db/db_versioned.go @@ -7,47 +7,42 @@ package db import ( + "bytes" "context" + "fmt" + "math" + "github.com/pkg/errors" + bolt "go.etcd.io/bbolt" + + "github.com/iotexproject/iotex-core/v2/db/batch" "github.com/iotexproject/iotex-core/v2/pkg/lifecycle" + "github.com/iotexproject/iotex-core/v2/pkg/util/byteutil" +) + +var ( + ErrDeleted = errors.New("deleted in DB") + _minKey = []byte{0} // the minimum key, used to store namespace's metadata ) type ( - // KvVersioned is a versioned key-value store, where each key has multiple - // versions of value (corresponding to different heights in a blockchain) - // - // Versioning is achieved by using (key + 8-byte version) as the actual - // storage key in the underlying DB. For each bucket, a metadata is stored - // at the special key = []byte{0}. The metadata specifies the bucket's name - // and the key length. - // - // For each versioned key, the special location = key + []byte{0} stores the - // key's metadata, which includes the following info: - // 1. the version when the key is first created - // 2. the version when the key is lastly written - // 3. the version when the key is deleted - // 4. hash of the key's last written value (to detect/avoid same write) - // If the location does not store a value, the key has never been written. - // - // How to use a versioned DB: - // - // db := NewBoltDBVersioned(cfg) // creates a versioned DB - // db.Start(ctx) - // defer func() { db.Stop(ctx) }() - // - // kv := db.SetVersion(5) - // value, err := kv.Get("ns", key) // read 'key' at version 5 - // kv = db.SetVersion(8) - // err := kv.Put("ns", key, value) // write 'key' at version 8 - - KvVersioned interface { + VersionedDB interface { lifecycle.StartStopper + // Put insert or update a record identified by (namespace, key) + Put(uint64, string, []byte, []byte) error + + // Get gets a record by (namespace, key) + Get(uint64, string, []byte) ([]byte, error) + + // Delete deletes a record by (namespace, key) + Delete(uint64, string, []byte) error + + // Base returns the underlying KVStore + Base() KVStore + // Version returns the key's most recent version Version(string, []byte) (uint64, error) - - // SetVersion sets the version, and returns a KVStore to call Put()/Get() - SetVersion(uint64) KVStoreBasic } // BoltDBVersioned is KvVersioned implementation based on bolt DB @@ -56,18 +51,12 @@ type ( } ) -// Option sets an option -type Option func(b *BoltDBVersioned) - -// NewBoltDBVersioned instantiates an BoltDB which implements KvVersioned -func NewBoltDBVersioned(cfg Config, opts ...Option) *BoltDBVersioned { - b := &BoltDBVersioned{ +// NewBoltDBVersioned instantiates an BoltDB which implements VersionedDB +func NewBoltDBVersioned(cfg Config) *BoltDBVersioned { + b := BoltDBVersioned{ db: NewBoltDB(cfg), } - for _, opt := range opts { - opt(b) - } - return b + return &b } // Start starts the DB @@ -80,31 +69,114 @@ func (b *BoltDBVersioned) Stop(ctx context.Context) error { return b.db.Stop(ctx) } +// Base returns the underlying KVStore +func (b *BoltDBVersioned) Base() KVStore { + return b.db +} + // Put writes a record -func (b *BoltDBVersioned) Put(ns string, version uint64, key, value []byte) error { +func (b *BoltDBVersioned) Put(version uint64, ns string, key, value []byte) error { if !b.db.IsReady() { return ErrDBNotStarted } - // TODO: implement Put - return nil + // check namespace + vn, err := b.checkNamespace(ns) + if err != nil { + return err + } + buf := batch.NewBatch() + if vn == nil { + // namespace not yet created + buf.Put(ns, _minKey, (&versionedNamespace{ + keyLen: uint32(len(key)), + }).serialize(), "failed to create metadata") + } else { + if len(key) != int(vn.keyLen) { + return errors.Wrapf(ErrInvalid, "invalid key length, expecting %d, got %d", vn.keyLen, len(key)) + } + last, _, err := b.get(math.MaxUint64, ns, key) + if !isNotExist(err) && version < last { + // not allowed to perform write on an earlier version + return ErrInvalid + } + buf.Delete(ns, keyForDelete(key, version), fmt.Sprintf("failed to delete key %x", key)) + } + buf.Put(ns, keyForWrite(key, version), value, fmt.Sprintf("failed to put key %x", key)) + return b.db.WriteBatch(buf) } // Get retrieves the most recent version -func (b *BoltDBVersioned) Get(ns string, version uint64, key []byte) ([]byte, error) { +func (b *BoltDBVersioned) Get(version uint64, ns string, key []byte) ([]byte, error) { if !b.db.IsReady() { return nil, ErrDBNotStarted } - // TODO: implement Get - return nil, nil + // check key's metadata + if err := b.checkNamespaceAndKey(ns, key); err != nil { + return nil, err + } + _, v, err := b.get(version, ns, key) + return v, err } -// Delete deletes a record,if key is nil,this will delete the whole bucket -func (b *BoltDBVersioned) Delete(ns string, key []byte) error { +func (b *BoltDBVersioned) get(version uint64, ns string, key []byte) (uint64, []byte, error) { + var ( + last uint64 + isDelete bool + value []byte + ) + err := b.db.db.View(func(tx *bolt.Tx) error { + bucket := tx.Bucket([]byte(ns)) + if bucket == nil { + return ErrBucketNotExist + } + var ( + c = bucket.Cursor() + min = keyForDelete(key, 0) + key = keyForWrite(key, version) + k, v = c.Seek(key) + ) + if k == nil || bytes.Compare(k, key) == 1 { + k, v = c.Prev() + if k == nil || bytes.Compare(k, min) <= 0 { + // cursor is at the beginning/end of the bucket or smaller than minimum key + return ErrNotExist + } + } + isDelete, last = parseKey(k) + value = make([]byte, len(v)) + copy(value, v) + return nil + }) + if err != nil { + return last, nil, err + } + if isDelete { + return last, nil, ErrDeleted + } + return last, value, nil +} + +// Delete deletes a record, if key does not exist, it returns nil +func (b *BoltDBVersioned) Delete(version uint64, ns string, key []byte) error { if !b.db.IsReady() { return ErrDBNotStarted } - // TODO: implement Delete - return nil + // check key's metadata + if err := b.checkNamespaceAndKey(ns, key); err != nil { + return err + } + last, _, err := b.get(math.MaxUint64, ns, key) + if isNotExist(err) { + return err + } + if version < last { + // not allowed to perform delete on an earlier version + return ErrInvalid + } + buf := batch.NewBatch() + buf.Put(ns, keyForDelete(key, version), nil, fmt.Sprintf("failed to delete key %x", key)) + buf.Delete(ns, keyForWrite(key, version), fmt.Sprintf("failed to delete key %x", key)) + return b.db.WriteBatch(buf) } // Version returns the key's most recent version @@ -112,45 +184,67 @@ func (b *BoltDBVersioned) Version(ns string, key []byte) (uint64, error) { if !b.db.IsReady() { return 0, ErrDBNotStarted } - // TODO: implement Version - return 0, nil -} - -// SetVersion sets the version, and returns a KVStore to call Put()/Get() -func (b *BoltDBVersioned) SetVersion(v uint64) KVStoreBasic { - return &KvWithVersion{ - db: b, - version: v, + // check key's metadata + if err := b.checkNamespaceAndKey(ns, key); err != nil { + return 0, err + } + last, _, err := b.get(math.MaxUint64, ns, key) + if isNotExist(err) { + // key not yet written + err = errors.Wrapf(ErrNotExist, "key = %x doesn't exist", key) } + return last, err } -// KvWithVersion wraps the BoltDBVersioned with a certain version -type KvWithVersion struct { - db *BoltDBVersioned - version uint64 // version for Get/Put() +func isNotExist(err error) bool { + return err == ErrNotExist || err == ErrBucketNotExist } -// Start starts the DB -func (b *KvWithVersion) Start(context.Context) error { - panic("should call BoltDBVersioned's Start method") +func keyForWrite(key []byte, v uint64) []byte { + k := make([]byte, len(key), len(key)+9) + copy(k, key) + k = append(k, byteutil.Uint64ToBytesBigEndian(v)...) + return append(k, 1) } -// Stop stops the DB -func (b *KvWithVersion) Stop(context.Context) error { - panic("should call BoltDBVersioned's Stop method") +func keyForDelete(key []byte, v uint64) []byte { + k := make([]byte, len(key), len(key)+9) + copy(k, key) + k = append(k, byteutil.Uint64ToBytesBigEndian(v)...) + return append(k, 0) } -// Put writes a record -func (b *KvWithVersion) Put(ns string, key, value []byte) error { - return b.db.Put(ns, b.version, key, value) +func parseKey(key []byte) (bool, uint64) { + size := len(key) + return (key[size-1] == 0), byteutil.BytesToUint64BigEndian(key[size-9 : size-1]) } -// Get retrieves a key's value -func (b *KvWithVersion) Get(ns string, key []byte) ([]byte, error) { - return b.db.Get(ns, b.version, key) +func (b *BoltDBVersioned) checkNamespace(ns string) (*versionedNamespace, error) { + data, err := b.db.Get(ns, _minKey) + switch errors.Cause(err) { + case nil: + vn, err := deserializeVersionedNamespace(data) + if err != nil { + return nil, err + } + return vn, nil + case ErrNotExist, ErrBucketNotExist: + return nil, nil + default: + return nil, err + } } -// Delete deletes a key -func (b *KvWithVersion) Delete(ns string, key []byte) error { - return b.db.Delete(ns, key) +func (b *BoltDBVersioned) checkNamespaceAndKey(ns string, key []byte) error { + vn, err := b.checkNamespace(ns) + if err != nil { + return err + } + if vn == nil { + return errors.Wrapf(ErrNotExist, "namespace = %x doesn't exist", ns) + } + if len(key) != int(vn.keyLen) { + return errors.Wrapf(ErrInvalid, "invalid key length, expecting %d, got %d", vn.keyLen, len(key)) + } + return nil } diff --git a/db/db_versioned_test.go b/db/db_versioned_test.go index d132a422b1..d13e9afec0 100644 --- a/db/db_versioned_test.go +++ b/db/db_versioned_test.go @@ -7,21 +7,316 @@ package db import ( - "encoding/hex" + "context" "testing" + "github.com/pkg/errors" "github.com/stretchr/testify/require" + + "github.com/iotexproject/iotex-core/v2/testutil" ) -func TestPb(t *testing.T) { +type versionTest struct { + ns string + k, v []byte + height uint64 + err error +} + +func TestVersionedDB(t *testing.T) { r := require.New(t) + testPath, err := testutil.PathOfTempFile("test-version") + r.NoError(err) + defer func() { + testutil.CleanupPath(testPath) + }() + + cfg := DefaultConfig + cfg.DbPath = testPath + db := NewBoltDBVersioned(cfg) + ctx := context.Background() + r.NoError(db.Start(ctx)) + defer func() { + db.Stop(ctx) + }() - vn := &versionedNamespace{ - name: "3jfsp5@(%)EW*#)_#¡ªº–ƒ˚œade∆…", - keyLen: 5} - data := vn.serialize() - r.Equal("0a29336a667370354028252945572a23295f23c2a1c2aac2bae28093c692cb9ac593616465e28886e280a61005", hex.EncodeToString(data)) - vn1, err := deserializeVersionedNamespace(data) + // namespace and key does not exist + vn, err := db.checkNamespace(_bucket1) + r.Nil(vn) + r.Nil(err) + // write first key, namespace and key now exist + r.NoError(db.Put(0, _bucket1, _k2, _v2)) + vn, err = db.checkNamespace(_bucket1) + r.NoError(err) + r.EqualValues(len(_k2), vn.keyLen) + // check more Put/Get + var ( + _k5 = []byte("key_5") + _k10 = []byte("key_10") + ) + err = db.Put(1, _bucket1, _k10, _v1) + r.Equal("invalid key length, expecting 5, got 6: invalid input", err.Error()) + r.NoError(db.Put(1, _bucket1, _k2, _v1)) + r.NoError(db.Put(3, _bucket1, _k2, _v3)) + r.NoError(db.Put(6, _bucket1, _k2, _v2)) + r.NoError(db.Put(2, _bucket1, _k4, _v2)) + r.NoError(db.Put(4, _bucket1, _k4, _v1)) + r.NoError(db.Put(7, _bucket1, _k4, _v3)) + for _, e := range []versionTest{ + {_bucket2, _k1, nil, 0, ErrNotExist}, // bucket not exist + {_bucket1, _k1, nil, 0, ErrNotExist}, + {_bucket1, _k2, _v2, 0, nil}, + {_bucket1, _k2, _v1, 1, nil}, + {_bucket1, _k2, _v1, 2, nil}, + {_bucket1, _k2, _v3, 3, nil}, + {_bucket1, _k2, _v3, 5, nil}, + {_bucket1, _k2, _v2, 6, nil}, + {_bucket1, _k2, _v2, 7, nil}, // after last write version + {_bucket1, _k3, nil, 0, ErrNotExist}, + {_bucket1, _k4, nil, 1, ErrNotExist}, // before first write version + {_bucket1, _k4, _v2, 2, nil}, + {_bucket1, _k4, _v2, 3, nil}, + {_bucket1, _k4, _v1, 4, nil}, + {_bucket1, _k4, _v1, 6, nil}, + {_bucket1, _k4, _v3, 7, nil}, + {_bucket1, _k4, _v3, 8, nil}, // larger than last key in bucket + {_bucket1, _k5, nil, 0, ErrNotExist}, + {_bucket1, _k10, nil, 0, ErrInvalid}, + } { + value, err := db.Get(e.height, e.ns, e.k) + r.Equal(e.err, errors.Cause(err)) + r.Equal(e.v, value) + } + // overwrite the same height again + r.NoError(db.Put(6, _bucket1, _k2, _v4)) + r.NoError(db.Put(7, _bucket1, _k4, _v4)) + // write to earlier version again is invalid + r.Equal(ErrInvalid, db.Put(3, _bucket1, _k2, _v4)) + r.Equal(ErrInvalid, db.Put(4, _bucket1, _k4, _v4)) + // write with same value + r.NoError(db.Put(9, _bucket1, _k2, _v4)) + r.NoError(db.Put(10, _bucket1, _k4, _v4)) + for _, e := range []versionTest{ + {_bucket2, _k1, nil, 0, ErrNotExist}, // bucket not exist + {_bucket1, _k1, nil, 0, ErrNotExist}, + {_bucket1, _k2, _v2, 0, nil}, + {_bucket1, _k2, _v1, 1, nil}, + {_bucket1, _k2, _v1, 2, nil}, + {_bucket1, _k2, _v3, 3, nil}, + {_bucket1, _k2, _v3, 5, nil}, + {_bucket1, _k2, _v4, 6, nil}, + {_bucket1, _k2, _v4, 8, nil}, + {_bucket1, _k2, _v4, 9, nil}, + {_bucket1, _k2, _v4, 10, nil}, // after last write version + {_bucket1, _k3, nil, 0, ErrNotExist}, + {_bucket1, _k4, nil, 1, ErrNotExist}, // before first write version + {_bucket1, _k4, _v2, 2, nil}, + {_bucket1, _k4, _v2, 3, nil}, + {_bucket1, _k4, _v1, 4, nil}, + {_bucket1, _k4, _v1, 6, nil}, + {_bucket1, _k4, _v4, 7, nil}, + {_bucket1, _k4, _v4, 9, nil}, + {_bucket1, _k4, _v4, 10, nil}, + {_bucket1, _k4, _v4, 11, nil}, // larger than last key in bucket + {_bucket1, _k5, nil, 0, ErrNotExist}, + {_bucket1, _k10, nil, 0, ErrInvalid}, + } { + value, err := db.Get(e.height, e.ns, e.k) + r.Equal(e.err, errors.Cause(err)) + r.Equal(e.v, value) + } + // check version + for _, e := range []versionTest{ + {_bucket1, _k1, nil, 0, ErrNotExist}, + {_bucket1, _k2, nil, 9, nil}, + {_bucket1, _k3, nil, 0, ErrNotExist}, + {_bucket1, _k4, nil, 10, nil}, + {_bucket1, _k5, nil, 0, ErrNotExist}, + {_bucket1, _k10, nil, 0, ErrInvalid}, + } { + value, err := db.Version(e.ns, e.k) + r.Equal(e.err, errors.Cause(err)) + r.Equal(e.height, value) + } + // test delete + r.Equal(ErrNotExist, errors.Cause(db.Delete(10, _bucket2, _k1))) + for _, k := range [][]byte{_k2, _k4} { + r.NoError(db.Delete(11, _bucket1, k)) + } + for _, k := range [][]byte{_k1, _k3, _k5} { + r.Equal(ErrNotExist, errors.Cause(db.Delete(10, _bucket1, k))) + } + r.Equal(ErrInvalid, errors.Cause(db.Delete(10, _bucket1, _k10))) + // key still can be read before delete version + for _, e := range []versionTest{ + {_bucket2, _k1, nil, 0, ErrNotExist}, // bucket not exist + {_bucket1, _k1, nil, 0, ErrNotExist}, + {_bucket1, _k2, _v2, 0, nil}, + {_bucket1, _k2, _v1, 1, nil}, + {_bucket1, _k2, _v1, 2, nil}, + {_bucket1, _k2, _v3, 3, nil}, + {_bucket1, _k2, _v3, 5, nil}, + {_bucket1, _k2, _v4, 6, nil}, + {_bucket1, _k2, _v4, 8, nil}, + {_bucket1, _k2, _v4, 9, nil}, + {_bucket1, _k2, _v4, 10, nil}, // before delete version + {_bucket1, _k2, nil, 11, ErrDeleted}, // after delete version + {_bucket1, _k3, nil, 0, ErrNotExist}, + {_bucket1, _k4, nil, 1, ErrNotExist}, // before first write version + {_bucket1, _k4, _v2, 2, nil}, + {_bucket1, _k4, _v2, 3, nil}, + {_bucket1, _k4, _v1, 4, nil}, + {_bucket1, _k4, _v1, 6, nil}, + {_bucket1, _k4, _v4, 7, nil}, + {_bucket1, _k4, _v4, 10, nil}, // before delete version + {_bucket1, _k4, nil, 11, ErrDeleted}, // after delete version + {_bucket1, _k5, nil, 0, ErrNotExist}, + {_bucket1, _k10, nil, 0, ErrInvalid}, + } { + value, err := db.Get(e.height, e.ns, e.k) + r.Equal(e.err, errors.Cause(err)) + r.Equal(e.v, value) + } + // write before delete version is invalid + r.Equal(ErrInvalid, db.Put(9, _bucket1, _k2, _k2)) + r.Equal(ErrInvalid, db.Put(9, _bucket1, _k4, _k4)) + for _, e := range []versionTest{ + {_bucket1, _k2, _v4, 10, nil}, // before delete version + {_bucket1, _k2, nil, 11, ErrDeleted}, // after delete version + {_bucket1, _k4, _v4, 10, nil}, // before delete version + {_bucket1, _k4, nil, 11, ErrDeleted}, // after delete version + } { + value, err := db.Get(e.height, e.ns, e.k) + r.Equal(e.err, errors.Cause(err)) + r.Equal(e.v, value) + } + // write after delete version + r.NoError(db.Put(12, _bucket1, _k2, _k2)) + r.NoError(db.Put(12, _bucket1, _k4, _k4)) + for _, e := range []versionTest{ + {_bucket2, _k1, nil, 0, ErrNotExist}, // bucket not exist + {_bucket1, _k1, nil, 0, ErrNotExist}, + {_bucket1, _k2, _v2, 0, nil}, + {_bucket1, _k2, _v1, 1, nil}, + {_bucket1, _k2, _v1, 2, nil}, + {_bucket1, _k2, _v3, 3, nil}, + {_bucket1, _k2, _v3, 5, nil}, + {_bucket1, _k2, _v4, 6, nil}, + {_bucket1, _k2, _v4, 8, nil}, + {_bucket1, _k2, _v4, 10, nil}, // before delete version + {_bucket1, _k2, nil, 11, ErrDeleted}, // after delete version + {_bucket1, _k2, _k2, 12, nil}, // after next write version + {_bucket1, _k3, nil, 0, ErrNotExist}, + {_bucket1, _k4, nil, 1, ErrNotExist}, // before first write version + {_bucket1, _k4, _v2, 2, nil}, + {_bucket1, _k4, _v2, 3, nil}, + {_bucket1, _k4, _v1, 4, nil}, + {_bucket1, _k4, _v1, 6, nil}, + {_bucket1, _k4, _v4, 7, nil}, + {_bucket1, _k4, _v4, 10, nil}, // before delete version + {_bucket1, _k4, nil, 11, ErrDeleted}, // after delete version + {_bucket1, _k4, _k4, 12, nil}, // after next write version + {_bucket1, _k5, nil, 0, ErrNotExist}, + {_bucket1, _k10, nil, 0, ErrInvalid}, + } { + value, err := db.Get(e.height, e.ns, e.k) + r.Equal(e.err, errors.Cause(err)) + r.Equal(e.v, value) + } + // check version after delete + for _, e := range []versionTest{ + {_bucket1, _k1, nil, 0, ErrNotExist}, + {_bucket1, _k2, nil, 12, nil}, + {_bucket1, _k3, nil, 0, ErrNotExist}, + {_bucket1, _k4, nil, 12, nil}, + {_bucket1, _k5, nil, 0, ErrNotExist}, + {_bucket1, _k10, nil, 0, ErrInvalid}, + } { + value, err := db.Version(e.ns, e.k) + r.Equal(e.err, errors.Cause(err)) + r.Equal(e.height, value) + } +} + +func TestMultipleWriteDelete(t *testing.T) { + r := require.New(t) + testPath, err := testutil.PathOfTempFile("test-version") + r.NoError(err) + defer func() { + testutil.CleanupPath(testPath) + }() + + cfg := DefaultConfig + cfg.DbPath = testPath + db := NewBoltDBVersioned(cfg) + ctx := context.Background() + r.NoError(db.Start(ctx)) + defer func() { + db.Stop(ctx) + }() + + // multiple writes and deletes + r.NoError(db.Put(1, _bucket1, _k2, _v1)) + r.NoError(db.Put(3, _bucket1, _k2, _v3)) + v, err := db.Version(_bucket1, _k2) + r.NoError(err) + r.EqualValues(3, v) + r.NoError(db.Delete(7, _bucket1, _k2)) + _, err = db.Version(_bucket1, _k2) + r.Equal(ErrDeleted, errors.Cause(err)) + r.NoError(db.Put(10, _bucket1, _k2, _v2)) + v, err = db.Version(_bucket1, _k2) + r.NoError(err) + r.EqualValues(10, v) + r.NoError(db.Delete(15, _bucket1, _k2)) + _, err = db.Version(_bucket1, _k2) + r.Equal(ErrDeleted, errors.Cause(err)) + r.NoError(db.Put(18, _bucket1, _k2, _v3)) + r.NoError(db.Delete(18, _bucket1, _k2)) // delete-after-write + _, err = db.Version(_bucket1, _k2) + r.Equal(ErrDeleted, errors.Cause(err)) + r.NoError(db.Put(18, _bucket1, _k2, _v3)) // write again + value, err := db.Get(18, _bucket1, _k2) + r.NoError(err) + r.Equal(_v3, value) + v, err = db.Version(_bucket1, _k2) + r.NoError(err) + r.EqualValues(18, v) + r.NoError(db.Delete(18, _bucket1, _k2)) // delete-after-write + _, err = db.Version(_bucket1, _k2) + r.Equal(ErrDeleted, errors.Cause(err)) + r.NoError(db.Put(21, _bucket1, _k2, _v4)) + v, err = db.Version(_bucket1, _k2) + r.NoError(err) + r.EqualValues(21, v) + r.NoError(db.Delete(25, _bucket1, _k2)) + r.NoError(db.Put(25, _bucket1, _k2, _k2)) // write-after-delete + v, err = db.Version(_bucket1, _k2) r.NoError(err) - r.Equal(vn, vn1) + r.EqualValues(25, v) + for _, e := range []versionTest{ + {_bucket1, _k2, nil, 0, ErrNotExist}, + {_bucket1, _k2, _v1, 1, nil}, + {_bucket1, _k2, _v1, 2, nil}, + {_bucket1, _k2, _v3, 3, nil}, + {_bucket1, _k2, _v3, 6, nil}, + {_bucket1, _k2, nil, 7, ErrDeleted}, + {_bucket1, _k2, nil, 9, ErrDeleted}, + {_bucket1, _k2, _v2, 10, nil}, + {_bucket1, _k2, _v2, 14, nil}, + {_bucket1, _k2, nil, 15, ErrDeleted}, + {_bucket1, _k2, nil, 17, ErrDeleted}, + {_bucket1, _k2, nil, 18, ErrDeleted}, + {_bucket1, _k2, nil, 20, ErrDeleted}, + {_bucket1, _k2, _v4, 21, nil}, + {_bucket1, _k2, _v4, 22, nil}, + {_bucket1, _k2, _v4, 24, nil}, + {_bucket1, _k2, _k2, 25, nil}, + {_bucket1, _k2, _k2, 26, nil}, + {_bucket1, _k2, _k2, 25000, nil}, + } { + value, err := db.Get(e.height, e.ns, e.k) + r.Equal(e.err, errors.Cause(err)) + r.Equal(e.v, value) + } } diff --git a/db/db_versioned_util.go b/db/db_versioned_util.go index d40d1b921b..215d3651b4 100644 --- a/db/db_versioned_util.go +++ b/db/db_versioned_util.go @@ -15,7 +15,6 @@ import ( // versionedNamespace is the metadata for versioned namespace type versionedNamespace struct { - name string keyLen uint32 } @@ -26,14 +25,12 @@ func (vn *versionedNamespace) serialize() []byte { func (vn *versionedNamespace) toProto() *versionpb.VersionedNamespace { return &versionpb.VersionedNamespace{ - Name: vn.name, KeyLen: vn.keyLen, } } func fromProtoVN(pb *versionpb.VersionedNamespace) *versionedNamespace { return &versionedNamespace{ - name: pb.Name, keyLen: pb.KeyLen, } } diff --git a/db/db_versioned_util_test.go b/db/db_versioned_util_test.go new file mode 100644 index 0000000000..e32dfa88fa --- /dev/null +++ b/db/db_versioned_util_test.go @@ -0,0 +1,26 @@ +// Copyright (c) 2021 IoTeX Foundation +// This is an alpha (internal) release and is not suitable for production. This source code is provided 'as is' and no +// warranties are given as to title or non-infringement, merchantability or fitness for purpose and, to the extent +// permitted by law, all liability for your use of the code is disclaimed. This source code is governed by Apache +// License 2.0 that can be found in the LICENSE file. + +package db + +import ( + "encoding/hex" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestPb(t *testing.T) { + r := require.New(t) + + vn := &versionedNamespace{ + keyLen: 5} + data := vn.serialize() + r.Equal("1005", hex.EncodeToString(data)) + vn1, err := deserializeVersionedNamespace(data) + r.NoError(err) + r.Equal(vn, vn1) +} diff --git a/db/kvstore_versioned.go b/db/kvstore_versioned.go new file mode 100644 index 0000000000..e1e85b01b3 --- /dev/null +++ b/db/kvstore_versioned.go @@ -0,0 +1,50 @@ +// Copyright (c) 2024 IoTeX Foundation +// This is an alpha (internal) release and is not suitable for production. This source code is provided 'as is' and no +// warranties are given as to title or non-infringement, merchantability or fitness for purpose and, to the extent +// permitted by law, all liability for your use of the code is disclaimed. This source code is governed by Apache +// License 2.0 that can be found in the LICENSE file. + +package db + +import ( + "github.com/iotexproject/iotex-core/v2/pkg/lifecycle" +) + +type ( + // KvVersioned is a versioned key-value store, where each key has multiple + // versions of value (corresponding to different heights in a blockchain) + // + // Versioning is achieved by using (key + 8-byte version) as the actual + // storage key in the underlying DB. For each bucket, a metadata is stored + // at the special key = []byte{0}. The metadata specifies the bucket's name + // and the key length. + // + // For each versioned key, the special location = key + []byte{0} stores the + // key's metadata, which includes the following info: + // 1. the version when the key is first created + // 2. the version when the key is lastly written + // 3. the version when the key is deleted + // 4. the key's last written value (to fast-track read of current version) + // If the location does not store a value, the key has never been written. + // + // How to use a versioned key-value store: + // + // db := NewKVStoreWithVersion(cfg) // creates a versioned DB + // db.Start(ctx) + // defer func() { db.Stop(ctx) }() + // + // kv := db.SetVersion(5) + // value, err := kv.Get("ns", key) // read 'key' at version 5 + // kv = db.SetVersion(8) + // err := kv.Put("ns", key, value) // write 'key' at version 8 + + KvVersioned interface { + lifecycle.StartStopper + + // Version returns the key's most recent version + Version(string, []byte) (uint64, error) + + // SetVersion sets the version, and returns a KVStore to call Put()/Get() + SetVersion(uint64) KVStore + } +) diff --git a/db/versionpb/version.pb.go b/db/versionpb/version.pb.go index e15c6531f5..9e2f1f0618 100644 --- a/db/versionpb/version.pb.go +++ b/db/versionpb/version.pb.go @@ -9,8 +9,8 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.28.1 -// protoc v3.21.12 +// protoc-gen-go v1.34.2 +// protoc v5.27.1 // source: version.proto package versionpb @@ -34,7 +34,6 @@ type VersionedNamespace struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` KeyLen uint32 `protobuf:"varint,2,opt,name=keyLen,proto3" json:"keyLen,omitempty"` } @@ -70,13 +69,6 @@ func (*VersionedNamespace) Descriptor() ([]byte, []int) { return file_version_proto_rawDescGZIP(), []int{0} } -func (x *VersionedNamespace) GetName() string { - if x != nil { - return x.Name - } - return "" -} - func (x *VersionedNamespace) GetKeyLen() uint32 { if x != nil { return x.KeyLen @@ -84,100 +76,18 @@ func (x *VersionedNamespace) GetKeyLen() uint32 { return 0 } -type KeyMeta struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - LastWriteHash []byte `protobuf:"bytes,1,opt,name=lastWriteHash,proto3" json:"lastWriteHash,omitempty"` // hash of value that was last written - FirstVersion uint64 `protobuf:"varint,2,opt,name=firstVersion,proto3" json:"firstVersion,omitempty"` - LastVersion uint64 `protobuf:"varint,3,opt,name=lastVersion,proto3" json:"lastVersion,omitempty"` - DeleteVersion uint64 `protobuf:"varint,4,opt,name=deleteVersion,proto3" json:"deleteVersion,omitempty"` -} - -func (x *KeyMeta) Reset() { - *x = KeyMeta{} - if protoimpl.UnsafeEnabled { - mi := &file_version_proto_msgTypes[1] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *KeyMeta) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*KeyMeta) ProtoMessage() {} - -func (x *KeyMeta) ProtoReflect() protoreflect.Message { - mi := &file_version_proto_msgTypes[1] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use KeyMeta.ProtoReflect.Descriptor instead. -func (*KeyMeta) Descriptor() ([]byte, []int) { - return file_version_proto_rawDescGZIP(), []int{1} -} - -func (x *KeyMeta) GetLastWriteHash() []byte { - if x != nil { - return x.LastWriteHash - } - return nil -} - -func (x *KeyMeta) GetFirstVersion() uint64 { - if x != nil { - return x.FirstVersion - } - return 0 -} - -func (x *KeyMeta) GetLastVersion() uint64 { - if x != nil { - return x.LastVersion - } - return 0 -} - -func (x *KeyMeta) GetDeleteVersion() uint64 { - if x != nil { - return x.DeleteVersion - } - return 0 -} - var File_version_proto protoreflect.FileDescriptor var file_version_proto_rawDesc = []byte{ 0x0a, 0x0d, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, - 0x09, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x70, 0x62, 0x22, 0x40, 0x0a, 0x12, 0x56, 0x65, + 0x09, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x70, 0x62, 0x22, 0x2c, 0x0a, 0x12, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, - 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, - 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x6b, 0x65, 0x79, 0x4c, 0x65, 0x6e, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0d, 0x52, 0x06, 0x6b, 0x65, 0x79, 0x4c, 0x65, 0x6e, 0x22, 0x9b, 0x01, 0x0a, - 0x07, 0x4b, 0x65, 0x79, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x24, 0x0a, 0x0d, 0x6c, 0x61, 0x73, 0x74, - 0x57, 0x72, 0x69, 0x74, 0x65, 0x48, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, - 0x0d, 0x6c, 0x61, 0x73, 0x74, 0x57, 0x72, 0x69, 0x74, 0x65, 0x48, 0x61, 0x73, 0x68, 0x12, 0x22, - 0x0a, 0x0c, 0x66, 0x69, 0x72, 0x73, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x66, 0x69, 0x72, 0x73, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, - 0x6f, 0x6e, 0x12, 0x20, 0x0a, 0x0b, 0x6c, 0x61, 0x73, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, - 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x6c, 0x61, 0x73, 0x74, 0x56, 0x65, 0x72, - 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x24, 0x0a, 0x0d, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x56, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x64, 0x65, 0x6c, - 0x65, 0x74, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x42, 0x31, 0x5a, 0x2f, 0x67, 0x69, - 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x69, 0x6f, 0x74, 0x65, 0x78, 0x70, 0x72, - 0x6f, 0x6a, 0x65, 0x63, 0x74, 0x2f, 0x69, 0x6f, 0x74, 0x65, 0x78, 0x2d, 0x63, 0x6f, 0x72, 0x65, - 0x2f, 0x64, 0x62, 0x2f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x70, 0x62, 0x62, 0x06, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x12, 0x16, 0x0a, 0x06, 0x6b, 0x65, 0x79, 0x4c, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, + 0x52, 0x06, 0x6b, 0x65, 0x79, 0x4c, 0x65, 0x6e, 0x42, 0x31, 0x5a, 0x2f, 0x67, 0x69, 0x74, 0x68, + 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x69, 0x6f, 0x74, 0x65, 0x78, 0x70, 0x72, 0x6f, 0x6a, + 0x65, 0x63, 0x74, 0x2f, 0x69, 0x6f, 0x74, 0x65, 0x78, 0x2d, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x64, + 0x62, 0x2f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, } var ( @@ -192,10 +102,9 @@ func file_version_proto_rawDescGZIP() []byte { return file_version_proto_rawDescData } -var file_version_proto_msgTypes = make([]protoimpl.MessageInfo, 2) -var file_version_proto_goTypes = []interface{}{ +var file_version_proto_msgTypes = make([]protoimpl.MessageInfo, 1) +var file_version_proto_goTypes = []any{ (*VersionedNamespace)(nil), // 0: versionpb.VersionedNamespace - (*KeyMeta)(nil), // 1: versionpb.KeyMeta } var file_version_proto_depIdxs = []int32{ 0, // [0:0] is the sub-list for method output_type @@ -211,7 +120,7 @@ func file_version_proto_init() { return } if !protoimpl.UnsafeEnabled { - file_version_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + file_version_proto_msgTypes[0].Exporter = func(v any, i int) any { switch v := v.(*VersionedNamespace); i { case 0: return &v.state @@ -223,18 +132,6 @@ func file_version_proto_init() { return nil } } - file_version_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*KeyMeta); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } } type x struct{} out := protoimpl.TypeBuilder{ @@ -242,7 +139,7 @@ func file_version_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_version_proto_rawDesc, NumEnums: 0, - NumMessages: 2, + NumMessages: 1, NumExtensions: 0, NumServices: 0, }, diff --git a/db/versionpb/version.proto b/db/versionpb/version.proto index 69645bed67..b4a1c519d7 100644 --- a/db/versionpb/version.proto +++ b/db/versionpb/version.proto @@ -11,13 +11,5 @@ package versionpb; option go_package = "github.com/iotexproject/iotex-core/db/versionpb"; message VersionedNamespace { - string name = 1; uint32 keyLen = 2; } - -message KeyMeta { - bytes lastWriteHash = 1; // hash of value that was last written - uint64 firstVersion = 2; - uint64 lastVersion = 3; - uint64 deleteVersion = 4; -} From 31f35a6864e1dcd69f54797d141ff98b1e2f4d9d Mon Sep 17 00:00:00 2001 From: Chen Chen Date: Mon, 9 Dec 2024 20:55:12 +0800 Subject: [PATCH 04/21] Trace log (#4504) --- action/protocol/execution/evm/evm.go | 15 ++-- .../execution/evm/evmstatedbadapter.go | 82 +++++++++++-------- actpool/actpool.go | 2 +- api/coreservice.go | 6 +- api/web3server.go | 22 ++--- api/web3server_test.go | 16 ++-- 6 files changed, 77 insertions(+), 66 deletions(-) diff --git a/action/protocol/execution/evm/evm.go b/action/protocol/execution/evm/evm.go index 116e856a96..048cd625cf 100644 --- a/action/protocol/execution/evm/evm.go +++ b/action/protocol/execution/evm/evm.go @@ -245,7 +245,7 @@ func ExecuteContract( if err != nil { return nil, nil, err } - retval, depositGas, remainingGas, contractAddress, statusCode, err := executeInEVM(ps, stateDB) + retval, depositGas, remainingGas, contractAddress, statusCode, err := executeInEVM(ctx, ps, stateDB) if err != nil { return nil, nil, err } @@ -380,6 +380,7 @@ func prepareStateDB(ctx context.Context, sm protocol.StateManager) (*StateDBAdap } if featureCtx.FixRevertSnapshot || actionCtx.ReadOnly { opts = append(opts, FixRevertSnapshotOption()) + opts = append(opts, WithContext(ctx)) } return NewStateDBAdapter( sm, @@ -427,7 +428,7 @@ func getChainConfig(g genesis.Blockchain, height uint64, id uint32, getBlockTime } // Error in executeInEVM is a consensus issue -func executeInEVM(evmParams *Params, stateDB *StateDBAdapter) ([]byte, uint64, uint64, string, iotextypes.ReceiptStatus, error) { +func executeInEVM(ctx context.Context, evmParams *Params, stateDB *StateDBAdapter) ([]byte, uint64, uint64, string, iotextypes.ReceiptStatus, error) { var ( gasLimit = evmParams.blkCtx.GasLimit blockHeight = evmParams.blkCtx.BlockHeight @@ -436,7 +437,7 @@ func executeInEVM(evmParams *Params, stateDB *StateDBAdapter) ([]byte, uint64, u chainConfig = evmParams.chainConfig ) if err := securityDeposit(evmParams, stateDB, gasLimit); err != nil { - log.L().Warn("unexpected error: not enough security deposit", zap.Error(err)) + log.T(ctx).Warn("unexpected error: not enough security deposit", zap.Error(err)) return nil, 0, 0, action.EmptyAddress, iotextypes.ReceiptStatus_Failure, err } var ( @@ -472,7 +473,7 @@ func executeInEVM(evmParams *Params, stateDB *StateDBAdapter) ([]byte, uint64, u // create contract var evmContractAddress common.Address _, evmContractAddress, remainingGas, evmErr = evm.Create(executor, evmParams.data, remainingGas, amount) - log.L().Debug("evm Create.", log.Hex("addrHash", evmContractAddress[:])) + log.T(ctx).Debug("evm Create.", log.Hex("addrHash", evmContractAddress[:])) if evmErr == nil { if contractAddress, err := address.FromBytes(evmContractAddress.Bytes()); err == nil { contractRawAddress = contractAddress.String() @@ -484,7 +485,7 @@ func executeInEVM(evmParams *Params, stateDB *StateDBAdapter) ([]byte, uint64, u ret, remainingGas, evmErr = evm.Call(executor, *evmParams.contract, evmParams.data, remainingGas, amount) } if evmErr != nil { - log.L().Debug("evm error", zap.Error(evmErr)) + log.T(ctx).Debug("evm error", zap.Error(evmErr)) // The only possible consensus-error would be if there wasn't // sufficient balance to make the transfer happen. // Should be a hard fork (Bering) @@ -493,7 +494,7 @@ func executeInEVM(evmParams *Params, stateDB *StateDBAdapter) ([]byte, uint64, u } } if stateDB.Error() != nil { - log.L().Debug("statedb error", zap.Error(stateDB.Error())) + log.T(ctx).Debug("statedb error", zap.Error(stateDB.Error())) } if !rules.IsLondon { // Before EIP-3529: refunds were capped to gasUsed / 2 @@ -532,7 +533,7 @@ func executeInEVM(evmParams *Params, stateDB *StateDBAdapter) ([]byte, uint64, u } else { addr = "contract creation" } - log.L().Warn("evm internal error", zap.Error(evmErr), + log.T(ctx).Warn("evm internal error", zap.Error(evmErr), zap.String("address", addr), log.Hex("calldata", evmParams.data)) } diff --git a/action/protocol/execution/evm/evmstatedbadapter.go b/action/protocol/execution/evm/evmstatedbadapter.go index 851f2407d9..7c38333ae4 100644 --- a/action/protocol/execution/evm/evmstatedbadapter.go +++ b/action/protocol/execution/evm/evmstatedbadapter.go @@ -7,6 +7,7 @@ package evm import ( "bytes" + "context" "encoding/hex" "fmt" "math/big" @@ -46,6 +47,7 @@ type ( // StateDBAdapter represents the state db adapter for evm to access iotx blockchain StateDBAdapter struct { + ctx context.Context sm protocol.StateManager logs []*action.Log transactionLogs []*action.TransactionLog @@ -196,6 +198,13 @@ func FixRevertSnapshotOption() StateDBAdapterOption { } } +func WithContext(ctx context.Context) StateDBAdapterOption { + return func(adapter *StateDBAdapter) error { + adapter.ctx = ctx + return nil + } +} + // NewStateDBAdapter creates a new state db with iotex blockchain func NewStateDBAdapter( sm protocol.StateManager, @@ -221,6 +230,7 @@ func NewStateDBAdapter( accessListSnapshot: make(map[int]*accessList), logsSnapshot: make(map[int]int), txLogsSnapshot: make(map[int]int), + ctx: context.Background(), } for _, opt := range opts { if err := opt(s); err != nil { @@ -251,9 +261,9 @@ func (stateDB *StateDBAdapter) assertError(err error, msg string, fields ...zap. return false } if stateDB.panicUnrecoverableError { - log.L().Panic(msg, fields...) + log.T(stateDB.ctx).Panic(msg, fields...) } - log.L().Error(msg, fields...) + log.T(stateDB.ctx).Error(msg, fields...) stateDB.logError(err) return true } @@ -286,7 +296,7 @@ func (stateDB *StateDBAdapter) CreateAccount(evmAddr common.Address) { if stateDB.enableCancun { stateDB.createdAccount[evmAddr] = struct{}{} } - log.L().Debug("Called CreateAccount.", log.Hex("addrHash", evmAddr[:])) + log.T(stateDB.ctx).Debug("Called CreateAccount.", log.Hex("addrHash", evmAddr[:])) } // SubBalance subtracts balance from account @@ -296,7 +306,7 @@ func (stateDB *StateDBAdapter) SubBalance(evmAddr common.Address, a256 *uint256. return } // stateDB.GetBalance(evmAddr) - log.L().Debug(fmt.Sprintf("SubBalance %v from %s", amount, evmAddr.Hex())) + log.T(stateDB.ctx).Debug(fmt.Sprintf("SubBalance %v from %s", amount, evmAddr.Hex())) addr, err := address.FromBytes(evmAddr.Bytes()) if stateDB.assertError(err, "Failed to convert evm address.", zap.Error(err)) { return @@ -323,7 +333,7 @@ func (stateDB *StateDBAdapter) AddBalance(evmAddr common.Address, a256 *uint256. return } // stateDB.GetBalance(evmAddr) - log.L().Debug(fmt.Sprintf("AddBalance %v to %s", amount, evmAddr.Hex())) + log.T(stateDB.ctx).Debug(fmt.Sprintf("AddBalance %v to %s", amount, evmAddr.Hex())) addr, err := address.FromBytes(evmAddr.Bytes()) if stateDB.assertError(err, "Failed to convert evm address.", zap.Error(err)) { @@ -360,7 +370,7 @@ func (stateDB *StateDBAdapter) GetBalance(evmAddr common.Address) *uint256.Int { if stateDB.assertError(err, "Failed to get balance.", zap.Error(err), zap.String("address", evmAddr.Hex())) { return common.U2560 } - log.L().Debug(fmt.Sprintf("Balance of %s is %v", evmAddr.Hex(), state.Balance)) + log.T(stateDB.ctx).Debug(fmt.Sprintf("Balance of %s is %v", evmAddr.Hex(), state.Balance)) return uint256.MustFromBig(state.Balance) } @@ -386,9 +396,9 @@ func (stateDB *StateDBAdapter) GetNonce(evmAddr common.Address) uint64 { state, err := stateDB.accountState(evmAddr) if err != nil { if stateDB.panicUnrecoverableError { - log.L().Panic("Failed to get nonce.", zap.Error(err)) + log.T(stateDB.ctx).Panic("Failed to get nonce.", zap.Error(err)) } else { - log.L().Error("Failed to get nonce.", zap.Error(err)) + log.T(stateDB.ctx).Error("Failed to get nonce.", zap.Error(err)) stateDB.logError(err) } } else { @@ -404,7 +414,7 @@ func (stateDB *StateDBAdapter) GetNonce(evmAddr common.Address) uint64 { } pendingNonce-- } - log.L().Debug("Called GetNonce.", + log.T(stateDB.ctx).Debug("Called GetNonce.", zap.String("address", evmAddr.Hex()), zap.Uint64("pendingNonce", pendingNonce)) @@ -427,12 +437,12 @@ func (stateDB *StateDBAdapter) SetNonce(evmAddr common.Address, nonce uint64) { } nonce-- } - log.L().Debug("Called SetNonce.", + log.T(stateDB.ctx).Debug("Called SetNonce.", zap.String("address", addr.String()), zap.Uint64("nonce", nonce)) if !s.IsNewbieAccount() || s.AccountType() != 0 || nonce != 0 || stateDB.zeroNonceForFreshAccount { if err := s.SetPendingNonce(nonce + 1); err != nil { - log.L().Panic("Failed to set nonce.", zap.Error(err), zap.String("addr", addr.Hex()), zap.Uint64("pendingNonce", s.PendingNonce()), zap.Uint64("nonce", nonce), zap.String("execution", hex.EncodeToString(stateDB.executionHash[:]))) + log.T(stateDB.ctx).Panic("Failed to set nonce.", zap.Error(err), zap.String("addr", addr.Hex()), zap.Uint64("pendingNonce", s.PendingNonce()), zap.Uint64("nonce", nonce), zap.String("execution", hex.EncodeToString(stateDB.executionHash[:]))) stateDB.logError(err) } } @@ -442,7 +452,7 @@ func (stateDB *StateDBAdapter) SetNonce(evmAddr common.Address, nonce uint64) { // SubRefund subtracts refund func (stateDB *StateDBAdapter) SubRefund(gas uint64) { - log.L().Debug("Called SubRefund.", zap.Uint64("gas", gas)) + log.T(stateDB.ctx).Debug("Called SubRefund.", zap.Uint64("gas", gas)) // stateDB.journal.append(refundChange{prev: self.refund}) if gas > stateDB.refund { panic("Refund counter not enough!") @@ -452,21 +462,21 @@ func (stateDB *StateDBAdapter) SubRefund(gas uint64) { // AddRefund adds refund func (stateDB *StateDBAdapter) AddRefund(gas uint64) { - log.L().Debug("Called AddRefund.", zap.Uint64("gas", gas)) + log.T(stateDB.ctx).Debug("Called AddRefund.", zap.Uint64("gas", gas)) // stateDB.journal.append(refundChange{prev: self.refund}) stateDB.refund += gas } // GetRefund gets refund func (stateDB *StateDBAdapter) GetRefund() uint64 { - log.L().Debug("Called GetRefund.") + log.T(stateDB.ctx).Debug("Called GetRefund.") return stateDB.refund } // SelfDestruct kills the contract func (stateDB *StateDBAdapter) SelfDestruct(evmAddr common.Address) { if !stateDB.Exist(evmAddr) { - log.L().Debug("Account does not exist.", zap.String("address", evmAddr.Hex())) + log.T(stateDB.ctx).Debug("Account does not exist.", zap.String("address", evmAddr.Hex())) return } s, err := stateDB.accountState(evmAddr) @@ -475,7 +485,7 @@ func (stateDB *StateDBAdapter) SelfDestruct(evmAddr common.Address) { } // clears the account balance actBalance := new(big.Int).Set(s.Balance) - log.L().Info("SelfDestruct contract", zap.String("Balance", actBalance.String())) + log.T(stateDB.ctx).Info("SelfDestruct contract", zap.String("Balance", actBalance.String())) err = s.SubBalance(s.Balance) if stateDB.assertError(err, "Failed to clear balance.", zap.Error(err), zap.String("address", evmAddr.Hex())) { return @@ -501,7 +511,7 @@ func (stateDB *StateDBAdapter) HasSelfDestructed(evmAddr common.Address) bool { // Selfdestruct6780 implements EIP-6780 func (stateDB *StateDBAdapter) Selfdestruct6780(evmAddr common.Address) { if !stateDB.Exist(evmAddr) { - log.L().Debug("Account does not exist.", zap.String("address", evmAddr.Hex())) + log.T(stateDB.ctx).Debug("Account does not exist.", zap.String("address", evmAddr.Hex())) return } // opSelfdestruct6780 has already subtracted the contract's balance @@ -534,7 +544,7 @@ func (stateDB *StateDBAdapter) Exist(evmAddr common.Address) bool { if stateDB.assertError(err, "Failed to convert evm address.", zap.Error(err)) { return false } - log.L().Debug("Check existence.", zap.String("address", addr.String()), log.Hex("addrHash", evmAddr[:])) + log.T(stateDB.ctx).Debug("Check existence.", zap.String("address", addr.String()), log.Hex("addrHash", evmAddr[:])) if _, ok := stateDB.cachedContract[evmAddr]; ok { return true } @@ -543,7 +553,7 @@ func (stateDB *StateDBAdapter) Exist(evmAddr common.Address) bool { return false } if !recorded { - log.L().Debug("Account does not exist.", zap.String("address", addr.String())) + log.T(stateDB.ctx).Debug("Account does not exist.", zap.String("address", addr.String())) return false } return true @@ -609,7 +619,7 @@ func (stateDB *StateDBAdapter) AddSlotToAccessList(addr common.Address, slot com // Empty returns true if the the contract is empty func (stateDB *StateDBAdapter) Empty(evmAddr common.Address) bool { - log.L().Debug("Check whether the contract is empty.") + log.T(stateDB.ctx).Debug("Check whether the contract is empty.") s, err := stateDB.accountState(evmAddr) if stateDB.assertError(err, "Failed to get account.", zap.Error(err), zap.String("address", evmAddr.Hex())) { return true @@ -624,7 +634,7 @@ func (stateDB *StateDBAdapter) Empty(evmAddr common.Address) bool { func (stateDB *StateDBAdapter) RevertToSnapshot(snapshot int) { ds, ok := stateDB.selfDestructedSnapshot[snapshot] if !ok && stateDB.panicUnrecoverableError { - log.L().Panic("Failed to revert to snapshot.", zap.Int("snapshot", snapshot)) + log.T(stateDB.ctx).Panic("Failed to revert to snapshot.", zap.Int("snapshot", snapshot)) } err := stateDB.sm.Revert(snapshot) if stateDB.assertError(err, "state manager's Revert() failed.", zap.Error(err), zap.Int("snapshot", snapshot)) { @@ -632,7 +642,7 @@ func (stateDB *StateDBAdapter) RevertToSnapshot(snapshot int) { } if !ok { // this should not happen, b/c we save the SelfDestruct accounts on a successful return of Snapshot(), but check anyway - log.L().Error("Failed to revert to snapshot.", zap.Int("snapshot", snapshot)) + log.T(stateDB.ctx).Error("Failed to revert to snapshot.", zap.Int("snapshot", snapshot)) return } deleteSnapshot := snapshot @@ -720,7 +730,7 @@ func (stateDB *StateDBAdapter) RevertToSnapshot(snapshot int) { for _, addr := range stateDB.cachedContractAddrs() { c := stateDB.cachedContract[addr] if err := c.LoadRoot(); err != nil { - log.L().Error("Failed to load root for contract.", zap.Error(err), log.Hex("addrHash", addr[:])) + log.T(stateDB.ctx).Error("Failed to load root for contract.", zap.Error(err), log.Hex("addrHash", addr[:])) return } } @@ -770,9 +780,9 @@ func (stateDB *StateDBAdapter) Snapshot() int { if _, ok := stateDB.selfDestructedSnapshot[sn]; ok { err := errors.New("unexpected error: duplicate snapshot version") if stateDB.fixSnapshotOrder { - log.L().Panic("Failed to snapshot.", zap.Error(err)) + log.T(stateDB.ctx).Panic("Failed to snapshot.", zap.Error(err)) } else { - log.L().Error("Failed to snapshot.", zap.Error(err)) + log.T(stateDB.ctx).Error("Failed to snapshot.", zap.Error(err)) } // stateDB.err = err return sn @@ -819,7 +829,7 @@ func (stateDB *StateDBAdapter) Snapshot() int { // AddLog adds log whose transaction amount is larger than 0 func (stateDB *StateDBAdapter) AddLog(evmLog *types.Log) { - log.L().Debug("Called AddLog.", zap.Any("log", evmLog)) + log.T(stateDB.ctx).Debug("Called AddLog.", zap.Any("log", evmLog)) addr, err := address.FromBytes(evmLog.Address.Bytes()) if stateDB.assertError(err, "Failed to convert evm address.", zap.Error(err)) { return @@ -929,7 +939,7 @@ func (stateDB *StateDBAdapter) generateSelfDestructTransferLog(sender string, am }) } } else { - log.L().Panic("SelfDestruct contract's balance does not match", + log.T(stateDB.ctx).Panic("SelfDestruct contract's balance does not match", zap.String("beneficiary", stateDB.lastAddBalanceAmount.String())) } } @@ -963,7 +973,7 @@ func (stateDB *StateDBAdapter) GetCode(evmAddr common.Address) []byte { if contract, ok := stateDB.cachedContract[evmAddr]; ok { code, err := contract.GetCode() if err != nil { - log.L().Error("Failed to get code hash.", zap.Error(err)) + log.T(stateDB.ctx).Error("Failed to get code hash.", zap.Error(err)) return nil } return code @@ -984,7 +994,7 @@ func (stateDB *StateDBAdapter) GetCode(evmAddr common.Address) []byte { // GetCodeSize gets the code size saved in hash func (stateDB *StateDBAdapter) GetCodeSize(evmAddr common.Address) int { code := stateDB.GetCode(evmAddr) - log.L().Debug("Called GetCodeSize.", log.Hex("addrHash", evmAddr[:])) + log.T(stateDB.ctx).Debug("Called GetCodeSize.", log.Hex("addrHash", evmAddr[:])) return len(code) } @@ -992,7 +1002,7 @@ func (stateDB *StateDBAdapter) GetCodeSize(evmAddr common.Address) int { func (stateDB *StateDBAdapter) SetCode(evmAddr common.Address, code []byte) { contract, err := stateDB.getContract(evmAddr) if err != nil { - log.L().Error("Failed to get contract.", zap.Error(err), zap.String("address", evmAddr.Hex())) + log.T(stateDB.ctx).Error("Failed to get contract.", zap.Error(err), zap.String("address", evmAddr.Hex())) stateDB.logError(err) return } @@ -1003,13 +1013,13 @@ func (stateDB *StateDBAdapter) SetCode(evmAddr common.Address, code []byte) { func (stateDB *StateDBAdapter) GetCommittedState(evmAddr common.Address, k common.Hash) common.Hash { contract, err := stateDB.getContract(evmAddr) if err != nil { - log.L().Error("Failed to get contract.", zap.Error(err), zap.String("address", evmAddr.Hex())) + log.T(stateDB.ctx).Error("Failed to get contract.", zap.Error(err), zap.String("address", evmAddr.Hex())) stateDB.logError(err) return common.Hash{} } v, err := contract.GetCommittedState(hash.BytesToHash256(k[:])) if err != nil { - log.L().Debug("Failed to get committed state.", zap.Error(err)) + log.T(stateDB.ctx).Debug("Failed to get committed state.", zap.Error(err)) stateDB.logError(err) return common.Hash{} } @@ -1020,13 +1030,13 @@ func (stateDB *StateDBAdapter) GetCommittedState(evmAddr common.Address, k commo func (stateDB *StateDBAdapter) GetState(evmAddr common.Address, k common.Hash) common.Hash { contract, err := stateDB.getContract(evmAddr) if err != nil { - log.L().Error("Failed to get contract.", zap.Error(err), zap.String("address", evmAddr.Hex())) + log.T(stateDB.ctx).Error("Failed to get contract.", zap.Error(err), zap.String("address", evmAddr.Hex())) stateDB.logError(err) return common.Hash{} } v, err := contract.GetState(hash.BytesToHash256(k[:])) if err != nil { - log.L().Debug("Failed to get state.", zap.Error(err)) + log.T(stateDB.ctx).Debug("Failed to get state.", zap.Error(err)) stateDB.logError(err) return common.Hash{} } @@ -1037,11 +1047,11 @@ func (stateDB *StateDBAdapter) GetState(evmAddr common.Address, k common.Hash) c func (stateDB *StateDBAdapter) SetState(evmAddr common.Address, k, v common.Hash) { contract, err := stateDB.getContract(evmAddr) if err != nil { - log.L().Error("Failed to get contract.", zap.Error(err), zap.String("address", evmAddr.Hex())) + log.T(stateDB.ctx).Error("Failed to get contract.", zap.Error(err), zap.String("address", evmAddr.Hex())) stateDB.logError(err) return } - log.L().Debug("Called SetState", log.Hex("addrHash", evmAddr[:]), log.Hex("k", k[:])) + log.T(stateDB.ctx).Debug("Called SetState", log.Hex("addrHash", evmAddr[:]), log.Hex("k", k[:])) err = contract.SetState(hash.BytesToHash256(k[:]), v[:]) stateDB.assertError(err, "Failed to set state.", zap.Error(err), zap.String("address", evmAddr.Hex())) } diff --git a/actpool/actpool.go b/actpool/actpool.go index 5716435a2b..781ea5dbaf 100644 --- a/actpool/actpool.go +++ b/actpool/actpool.go @@ -534,7 +534,7 @@ func (ap *actPool) enqueue(ctx context.Context, act *action.SealedEnvelope, repl for { select { case <-ctx.Done(): - log.L().Error("enqueue actpool fails", zap.Error(ctx.Err())) + log.T(ctx).Error("enqueue actpool fails", zap.Error(ctx.Err())) return ctx.Err() case ret := <-errChan: return ret diff --git a/api/coreservice.go b/api/coreservice.go index 45b55dd764..a65f126572 100644 --- a/api/coreservice.go +++ b/api/coreservice.go @@ -445,7 +445,7 @@ func (core *coreService) ServerMeta() (packageVersion string, packageCommitID st // SendAction is the API to send an action to blockchain. func (core *coreService) SendAction(ctx context.Context, in *iotextypes.Action) (string, error) { - log.Logger("api").Debug("receive send action request") + log.T(ctx).Debug("receive send action request") selp, err := (&action.Deserializer{}).SetEvmNetworkID(core.EVMNetworkID()).ActionToSealedEnvelope(in) if err != nil { return "", status.Error(codes.InvalidArgument, err.Error()) @@ -470,7 +470,7 @@ func (core *coreService) SendAction(ctx context.Context, in *iotextypes.Action) if err != nil { return "", err } - l := log.Logger("api").With(zap.String("actionHash", hex.EncodeToString(hash[:]))) + l := log.T(ctx).Logger().With(zap.String("actionHash", hex.EncodeToString(hash[:]))) if err = core.ap.Add(ctx, selp); err != nil { txBytes, serErr := proto.Marshal(in) if serErr != nil { @@ -489,7 +489,7 @@ func (core *coreService) SendAction(ctx context.Context, in *iotextypes.Action) } st, err := st.WithDetails(br) if err != nil { - log.Logger("api").Panic("Unexpected error attaching metadata", zap.Error(err)) + log.T(ctx).Panic("Unexpected error attaching metadata", zap.Error(err)) } return "", st.Err() } diff --git a/api/web3server.go b/api/web3server.go index 185130bcf0..b9b65bf7f4 100644 --- a/api/web3server.go +++ b/api/web3server.go @@ -147,7 +147,7 @@ func (svr *web3Handler) handleWeb3Req(ctx context.Context, web3Req *gjson.Result ) defer func(start time.Time) { svr.coreService.Track(ctx, start, method.(string), int64(size), err == nil) }(time.Now()) - log.T(ctx).Debug("handleWeb3Req", zap.String("method", method.(string)), zap.String("requestParams", fmt.Sprintf("%+v", web3Req))) + log.T(ctx).Info("handleWeb3Req", zap.String("method", method.(string)), zap.String("requestParams", fmt.Sprintf("%+v", web3Req))) _web3ServerMtc.WithLabelValues(method.(string)).Inc() _web3ServerMtc.WithLabelValues("requests_total").Inc() switch method { @@ -168,7 +168,7 @@ func (svr *web3Handler) handleWeb3Req(ctx context.Context, web3Req *gjson.Result case "eth_getTransactionCount": res, err = svr.getTransactionCount(web3Req) case "eth_call": - res, err = svr.call(web3Req) + res, err = svr.call(ctx, web3Req) case "eth_getCode": res, err = svr.getCode(web3Req) case "eth_protocolVersion": @@ -198,9 +198,9 @@ func (svr *web3Handler) handleWeb3Req(ctx context.Context, web3Req *gjson.Result case "eth_getBlockByNumber": res, err = svr.getBlockByNumber(web3Req) case "eth_estimateGas": - res, err = svr.estimateGas(web3Req) + res, err = svr.estimateGas(ctx, web3Req) case "eth_sendRawTransaction": - res, err = svr.sendRawTransaction(web3Req) + res, err = svr.sendRawTransaction(ctx, web3Req) case "eth_getTransactionByHash": res, err = svr.getTransactionByHash(web3Req) case "eth_getTransactionByBlockNumberAndIndex": @@ -378,7 +378,7 @@ func (svr *web3Handler) getTransactionCount(in *gjson.Result) (interface{}, erro return uint64ToHex(pendingNonce), nil } -func (svr *web3Handler) call(in *gjson.Result) (interface{}, error) { +func (svr *web3Handler) call(ctx context.Context, in *gjson.Result) (interface{}, error) { callMsg, err := parseCallObject(in) if err != nil { return nil, err @@ -419,7 +419,7 @@ func (svr *web3Handler) call(in *gjson.Result) (interface{}, error) { } elp := (&action.EnvelopeBuilder{}).SetAction(action.NewExecution(to, value, data)). SetGasLimit(gasLimit).Build() - ret, receipt, err := svr.coreService.ReadContract(context.Background(), callerAddr, elp) + ret, receipt, err := svr.coreService.ReadContract(ctx, callerAddr, elp) if err != nil { return nil, err } @@ -432,7 +432,7 @@ func (svr *web3Handler) call(in *gjson.Result) (interface{}, error) { return "0x" + ret, nil } -func (svr *web3Handler) estimateGas(in *gjson.Result) (interface{}, error) { +func (svr *web3Handler) estimateGas(ctx context.Context, in *gjson.Result) (interface{}, error) { callMsg, err := parseCallObject(in) if err != nil { return nil, err @@ -450,9 +450,9 @@ func (svr *web3Handler) estimateGas(in *gjson.Result) (interface{}, error) { from := callMsg.From switch act := elp.Action().(type) { case *action.Execution: - estimatedGas, err = svr.coreService.EstimateExecutionGasConsumption(context.Background(), elp, from) + estimatedGas, err = svr.coreService.EstimateExecutionGasConsumption(ctx, elp, from) case *action.MigrateStake: - estimatedGas, err = svr.coreService.EstimateMigrateStakeGasConsumption(context.Background(), act, from) + estimatedGas, err = svr.coreService.EstimateMigrateStakeGasConsumption(ctx, act, from) default: estimatedGas, err = svr.coreService.EstimateGasForNonExecution(act) } @@ -465,7 +465,7 @@ func (svr *web3Handler) estimateGas(in *gjson.Result) (interface{}, error) { return uint64ToHex(estimatedGas), nil } -func (svr *web3Handler) sendRawTransaction(in *gjson.Result) (interface{}, error) { +func (svr *web3Handler) sendRawTransaction(ctx context.Context, in *gjson.Result) (interface{}, error) { dataStr := in.Get("params.0") if !dataStr.Exists() { return nil, errInvalidFormat @@ -529,7 +529,7 @@ func (svr *web3Handler) sendRawTransaction(in *gjson.Result) (interface{}, error Encoding: encoding, } } - actionHash, err := cs.SendAction(context.Background(), req) + actionHash, err := cs.SendAction(ctx, req) if err != nil { return nil, err } diff --git a/api/web3server_test.go b/api/web3server_test.go index 4a96298600..5bf0a15477 100644 --- a/api/web3server_test.go +++ b/api/web3server_test.go @@ -317,7 +317,7 @@ func TestCall(t *testing.T) { "data": "d201114a" }, 1]}`) - ret, err := web3svr.call(&in) + ret, err := web3svr.call(context.Background(), &in) require.NoError(err) require.Equal("0x0000000000000000000000000000000000000000000000056bc75e2d63100000", ret.(string)) }) @@ -336,7 +336,7 @@ func TestCall(t *testing.T) { "data": "ad7a672f" }, 1]}`) - ret, err := web3svr.call(&in) + ret, err := web3svr.call(context.Background(), &in) require.NoError(err) require.Equal("0x0000000000000000000000000000000000000000000000000000000000002710", ret.(string)) }) @@ -352,7 +352,7 @@ func TestCall(t *testing.T) { "data": "0x1" }, 1]}`) - ret, err := web3svr.call(&in) + ret, err := web3svr.call(context.Background(), &in) require.NoError(err) require.Equal("0x111111", ret.(string)) }) @@ -378,7 +378,7 @@ func TestCall(t *testing.T) { "data": "0x1" }, 1]}`) - _, err := web3svr.call(&in) + _, err := web3svr.call(context.Background(), &in) require.EqualError(err, "rpc error: code = InvalidArgument desc = execution reverted: "+receipt.GetExecutionRevertMsg()) }) } @@ -405,7 +405,7 @@ func TestEstimateGas(t *testing.T) { "data": "0x6d4ce63c" }, 1]}`) - ret, err := web3svr.estimateGas(&in) + ret, err := web3svr.estimateGas(context.Background(), &in) require.NoError(err) require.Equal(uint64ToHex(uint64(21000)), ret.(string)) }) @@ -423,7 +423,7 @@ func TestEstimateGas(t *testing.T) { "data": "0x1123123c" }, 1]}`) - ret, err := web3svr.estimateGas(&in) + ret, err := web3svr.estimateGas(context.Background(), &in) require.NoError(err) require.Equal(uint64ToHex(uint64(36000)), ret.(string)) }) @@ -444,13 +444,13 @@ func TestSendRawTransaction(t *testing.T) { t.Run("nil params", func(t *testing.T) { inNil := gjson.Parse(`{"params":[]}`) - _, err := web3svr.sendRawTransaction(&inNil) + _, err := web3svr.sendRawTransaction(context.Background(), &inNil) require.EqualError(err, errInvalidFormat.Error()) }) t.Run("send tx", func(t *testing.T) { in := gjson.Parse(`{"params":["f8600180830186a09412745fec82b585f239c01090882eb40702c32b04808025a0b0e1aab5b64d744ae01fc9f1c3e9919844a799e90c23129d611f7efe6aec8a29a0195e28d22d9b280e00d501ff63525bb76f5c87b8646c89d5d9c5485edcb1b498"]}`) - ret, err := web3svr.sendRawTransaction(&in) + ret, err := web3svr.sendRawTransaction(context.Background(), &in) require.NoError(err) require.Equal("0x111111111111111", ret.(string)) }) From 58482489e9f2ec4bce027fe066db46ca79d549fd Mon Sep 17 00:00:00 2001 From: dustinxie Date: Mon, 9 Dec 2024 09:31:20 -0800 Subject: [PATCH 05/21] [evm] add SelfDestruct6780 test (#4509) --- action/protocol/execution/protocol_test.go | 2 +- .../testdata-cancun/selfdestruct.json | 37 +++++++++ .../selfdestruct_oncreation.sol | 18 ++++ .../testdata-cancun/selfdestruction.sol | 24 ++++++ blockchain/integrity/integrity_test.go | 83 ++++++++++++++++--- blockchain/integrity/testcontract_test.go | 6 +- 6 files changed, 157 insertions(+), 13 deletions(-) create mode 100644 action/protocol/execution/testdata-cancun/selfdestruct.json create mode 100644 action/protocol/execution/testdata-cancun/selfdestruct_oncreation.sol create mode 100644 action/protocol/execution/testdata-cancun/selfdestruction.sol diff --git a/action/protocol/execution/protocol_test.go b/action/protocol/execution/protocol_test.go index a33e343e31..057c34f004 100644 --- a/action/protocol/execution/protocol_test.go +++ b/action/protocol/execution/protocol_test.go @@ -1331,7 +1331,7 @@ func TestCancunEVM(t *testing.T) { NewSmartContractTest(t, "testdata-cancun/basefee.json") }) t.Run("eip6780-selfdestruct", func(t *testing.T) { - t.Skip("TODO: test it ") + NewSmartContractTest(t, "testdata-cancun/selfdestruct.json") }) } diff --git a/action/protocol/execution/testdata-cancun/selfdestruct.json b/action/protocol/execution/testdata-cancun/selfdestruct.json new file mode 100644 index 0000000000..0573f56a35 --- /dev/null +++ b/action/protocol/execution/testdata-cancun/selfdestruct.json @@ -0,0 +1,37 @@ +{ + "initGenesis": { + "isBering" : true, + "isIceland" : true, + "isLondon" : true, + "isShanghai" : true, + "isCancun": true + }, + "initBalances": [{ + "account": "io1mflp9m6hcgm2qcghchsdqj3z3eccrnekx9p0ms", + "rawBalance": "1000000000000000000000000000" + }], + "deployments": [{ + "rawByteCode": "608060405234801561000f575f80fd5b50335f806101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055506102f88061005c5f395ff3fe608060405260043610610037575f3560e01c80630c08bf881461004257806312065fe0146100585780638da5cb5b146100825761003e565b3661003e57005b5f80fd5b34801561004d575f80fd5b506100566100ac565b005b348015610063575f80fd5b5061006c610171565b60405161007991906101b3565b60405180910390f35b34801561008d575f80fd5b50610096610178565b6040516100a3919061020b565b60405180910390f35b5f8054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614610139576040517f08c379a0000000000000000000000000000000000000000000000000000000008152600401610130906102a4565b60405180910390fd5b5f8054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16ff5b5f47905090565b5f8054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b5f819050919050565b6101ad8161019b565b82525050565b5f6020820190506101c65f8301846101a4565b92915050565b5f73ffffffffffffffffffffffffffffffffffffffff82169050919050565b5f6101f5826101cc565b9050919050565b610205816101eb565b82525050565b5f60208201905061021e5f8301846101fc565b92915050565b5f82825260208201905092915050565b7f4f6e6c7920746865206f776e65722063616e2064657374726f792074686520635f8201527f6f6e747261637400000000000000000000000000000000000000000000000000602082015250565b5f61028e602783610224565b915061029982610234565b604082019050919050565b5f6020820190508181035f8301526102bb81610282565b905091905056fea2646970667358221220b78ecca519bda7ff2a88d8d2e964b5eaa98e01e80322cffe2fc97d17b9dcb6e464736f6c63430008180033", + "rawPrivateKey": "cfa6ef757dee2e50351620dca002d32b9c090cfda55fb81f37f1d26b273743f1", + "rawAmount": "0", + "rawGasLimit": 5000000, + "rawGasPrice": "2000000000000", + "rawExpectedGasConsumed": 269559, + "expectedStatus": 1, + "expectedBalances": [], + "comment": "deploy selfdestruct" + }], + "executions": [{ + "rawPrivateKey": "cfa6ef757dee2e50351620dca002d32b9c090cfda55fb81f37f1d26b273743f1", + "rawByteCode": "0c08bf88", + "rawAmount": "0", + "rawGasLimit": 1000000, + "rawGasPrice": "2000000000000", + "rawAccessList": [], + "rawExpectedGasConsumed": 17819, + "expectedStatus": 1, + "readOnly": true, + "rawReturnValue": "", + "comment": "call selfdestruct" + }] +} diff --git a/action/protocol/execution/testdata-cancun/selfdestruct_oncreation.sol b/action/protocol/execution/testdata-cancun/selfdestruct_oncreation.sol new file mode 100644 index 0000000000..9674fe81b7 --- /dev/null +++ b/action/protocol/execution/testdata-cancun/selfdestruct_oncreation.sol @@ -0,0 +1,18 @@ +// SPDX-License-Identifier: MIT +pragma solidity ^0.8.24; + +// this is the contract for "_selfdestructOnCreationContract" under blockchain/integrity + +contract SelfDestructOnCreation { + address payable public owner; + + constructor() payable { + owner = payable(msg.sender); // Set contract deployer as owner + selfdestruct(owner); // Self-destruct the contract immediately, sending any funds to the recipient + } + + // Function to check balance of the contract + function getBalance() external view returns (uint256) { + return address(this).balance; + } +} diff --git a/action/protocol/execution/testdata-cancun/selfdestruction.sol b/action/protocol/execution/testdata-cancun/selfdestruction.sol new file mode 100644 index 0000000000..24e389b2ad --- /dev/null +++ b/action/protocol/execution/testdata-cancun/selfdestruction.sol @@ -0,0 +1,24 @@ +// SPDX-License-Identifier: MIT +pragma solidity ^0.8.24; + +contract SelfDestructExample { + address payable public owner; + + constructor() { + owner = payable(msg.sender); // Set contract deployer as owner + } + + // Function to self-destruct the contract + function terminate() external { + require(msg.sender == owner, "Only the owner can destroy the contract"); + selfdestruct(owner); // Send remaining funds to the owner and destroy the contract + } + + // Fallback function to receive Ether + receive() external payable {} + + // Function to check balance of the contract + function getBalance() external view returns (uint256) { + return address(this).balance; + } +} diff --git a/blockchain/integrity/integrity_test.go b/blockchain/integrity/integrity_test.go index ad47ed0360..a93b494513 100644 --- a/blockchain/integrity/integrity_test.go +++ b/blockchain/integrity/integrity_test.go @@ -49,6 +49,7 @@ import ( "github.com/iotexproject/iotex-core/v2/db/trie/mptrie" "github.com/iotexproject/iotex-core/v2/pkg/unit" . "github.com/iotexproject/iotex-core/v2/pkg/util/assertions" + "github.com/iotexproject/iotex-core/v2/pkg/util/blockutil" "github.com/iotexproject/iotex-core/v2/state" "github.com/iotexproject/iotex-core/v2/state/factory" "github.com/iotexproject/iotex-core/v2/test/identityset" @@ -926,15 +927,11 @@ func createChain(cfg config.Config, inMem bool) (blockchain.Blockchain, factory. if dao == nil { return nil, nil, nil, nil, err } - ep := execution.NewProtocol(dao.GetBlockHash, rewarding.DepositGas, fakeGetBlockTime) - if err = ep.Register(registry); err != nil { - return nil, nil, nil, nil, err - } rewardingProtocol := rewarding.NewProtocol(cfg.Genesis.Rewarding) if err = rewardingProtocol.Register(registry); err != nil { return nil, nil, nil, nil, err } - return blockchain.NewBlockchain( + bc := blockchain.NewBlockchain( cfg.Chain, cfg.Genesis, dao, @@ -943,7 +940,23 @@ func createChain(cfg config.Config, inMem bool) (blockchain.Blockchain, factory. sf, protocol.NewGenericValidator(sf, accountutil.AccountState), )), - ), sf, dao, ap, nil + ) + btc, err := blockutil.NewBlockTimeCalculator(func(uint64) time.Duration { return time.Second }, + bc.TipHeight, func(height uint64) (time.Time, error) { + blk, err := dao.GetBlockByHeight(height) + if err != nil { + return time.Time{}, err + } + return blk.Timestamp(), nil + }) + if err != nil { + return nil, nil, nil, nil, err + } + ep := execution.NewProtocol(dao.GetBlockHash, rewarding.DepositGas, btc.CalculateBlockTime) + if err = ep.Register(registry); err != nil { + return nil, nil, nil, nil, err + } + return bc, sf, dao, ap, nil } func TestBlockchainHardForkFeatures(t *testing.T) { @@ -993,7 +1006,7 @@ func TestBlockchainHardForkFeatures(t *testing.T) { cfg.Genesis.VanuatuBlockHeight = 4 cfg.Genesis.InitBalanceMap[identityset.Address(27).String()] = unit.ConvertIotxToRau(10000000000).String() - ctx := context.Background() + ctx := genesis.WithGenesisContext(context.Background(), cfg.Genesis) bc, sf, dao, ap, err := createChain(cfg, true) require.NoError(err) sk, err := iotexcrypto.HexStringToPrivateKey(cfg.Chain.ProducerPrivKey) @@ -1232,7 +1245,7 @@ func TestBlockchainHardForkFeatures(t *testing.T) { require.Equal(v.b, a.Balance.String()) } - // Add block 4 -- test the UseTxContainer and AddClaimRewardAddress flag + // Add block 4 -- test the UseTxContainer, AddClaimRewardAddress, and EIP-6780 selfdestruct var ( txs [2]*types.Transaction contractHash hash.Hash256 @@ -1283,14 +1296,39 @@ func TestBlockchainHardForkFeatures(t *testing.T) { tsf2, err = action.Sign(elp, priKey0) require.NoError(err) require.NoError(ap.Add(ctx, tsf2)) + elp = (&action.EnvelopeBuilder{}).SetNonce(7). + SetChainID(cfg.Chain.ID). + SetGasPrice(minGas). + SetGasLimit(500000). + SetAction(action.NewExecution(action.EmptyAddress, new(big.Int), _selfdestructContract)).Build() + ex1, err = action.Sign(elp, priKey0) + require.NoError(err) + require.NoError(ap.Add(ctx, ex1)) + elp = (&action.EnvelopeBuilder{}).SetNonce(8). + SetChainID(cfg.Chain.ID). + SetGasPrice(minGas). + SetGasLimit(500000). + SetAction(action.NewExecution(action.EmptyAddress, new(big.Int), _selfdestructOnCreationContract)).Build() + ex2, err = action.Sign(elp, priKey0) + require.NoError(err) + require.NoError(ap.Add(ctx, ex2)) + selfdestructContract := "io12fltnfupejreyl8fmd9jq6rcfextg5ra9zjwuz" + elp = (&action.EnvelopeBuilder{}).SetNonce(9). + SetChainID(cfg.Chain.ID). + SetGasPrice(minGas). + SetGasLimit(500000). + SetAction(action.NewExecution(selfdestructContract, new(big.Int), MustNoErrorV(hex.DecodeString("0c08bf88")))).Build() + ex3, err := action.Sign(elp, priKey0) + require.NoError(err) + require.NoError(ap.Add(ctx, ex3)) blockTime = blockTime.Add(time.Second) blk3, err := bc.MintNewBlock(blockTime) require.NoError(err) require.EqualValues(4, blk3.Height()) - require.EqualValues(40020, blk3.Header.GasUsed()) + require.EqualValues(377122, blk3.Header.GasUsed()) require.EqualValues(action.InitialBaseFee, blk3.Header.BaseFee().Uint64()) - require.Equal(4, len(blk3.Body.Actions)) + require.Equal(7, len(blk3.Body.Actions)) require.NoError(bc.CommitBlock(blk3)) // verify contract execution @@ -1312,6 +1350,31 @@ func TestBlockchainHardForkFeatures(t *testing.T) { require.EqualValues(1, a.AccountType()) require.Equal("200000000000", a.Balance.String()) + // verify EIP-6780 selfdestruct contract address + for _, v := range []struct { + h hash.Hash256 + txType string + gas uint64 + }{ + {MustNoErrorV(ex1.Hash()), "selfdestruct", 269559}, + {MustNoErrorV(ex2.Hash()), "selfdestruct-oncreation", 49724}, + {MustNoErrorV(ex3.Hash()), "selfdestruct-afterwards", 17819}, + } { + r = MustNoErrorV(receiptByActionHash(dao, 4, v.h)) + require.EqualValues(1, r.Status) + require.EqualValues(v.gas, r.GasConsumed) + if v.txType == "selfdestruct" { + require.Equal(selfdestructContract, r.ContractAddress) + a = MustNoErrorV(accountutil.AccountState(ctx, sf, MustNoErrorV(address.FromString(r.ContractAddress)))) + require.True(a.IsContract()) + require.False(a.IsNewbieAccount()) + } else if v.txType == "selfdestruct-oncreation" { + a = MustNoErrorV(accountutil.AccountState(ctx, sf, MustNoErrorV(address.FromString(r.ContractAddress)))) + require.False(a.IsContract()) + require.True(a.IsNewbieAccount()) + } + } + // commit 4 blocks to a new chain testTriePath2, err := testutil.PathOfTempFile("trie") require.NoError(err) diff --git a/blockchain/integrity/testcontract_test.go b/blockchain/integrity/testcontract_test.go index 183a04b944..2e9c3fa219 100644 --- a/blockchain/integrity/testcontract_test.go +++ b/blockchain/integrity/testcontract_test.go @@ -8,6 +8,8 @@ package integrity import "encoding/hex" var ( - _codeStoreOutOfGasContract, _ = hex.DecodeString("60806040526040516200332738038062003327833981810160405260808110156200002957600080fd5b8101908080519060200190929190805190602001909291908051906020019092919080519060200190929190505050838383836200006c620004c860201b60201c565b8411620000c5576040517f08c379a0000000000000000000000000000000000000000000000000000000008152600401808060200182810382526026815260200180620033016026913960400191505060405180910390fd5b6000620000d885620004d960201b60201c565b9050620000eb81620004f860201b60201c565b62000142576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252602d8152602001806200328b602d913960400191505060405180910390fd5b803410156200019d576040517f08c379a0000000000000000000000000000000000000000000000000000000008152600401808060200182810382526022815260200180620032b86022913960400191505060405180910390fd5b42620001ae620007e960201b60201c565b84031162000208576040517f08c379a0000000000000000000000000000000000000000000000000000000008152600401808060200182810382526027815260200180620032da6027913960400191505060405180910390fd5b806003819055508260058190555081600260006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff16021790555083600681905550336000806101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055506000349050600354810390506000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff167f327ca9276da9073b583440165cb887319e7aeaf4003f14e92c1bbeee913e9b9c6003546040518082815260200191505060405180910390a26000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f1935050505015801562000383573d6000803e3d6000fd5b506000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff167fb54ecacfeaef46d29d83a7ee9ffaae207283e8a0f5df648dba2d52a1454e489e826040518082815260200191505060405180910390a26000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff167f53ddc5f55aac282e58e964d06a34c46ab814f1ea04d4ccf6eed6df8ba36d3f5e600354600654604051808381526020018281526020019250505060405180910390a26000600760016101000a81548160ff0219169083600e8111156200048e57fe5b02179055506000600760006101000a81548160ff02191690836004811115620004b357fe5b02179055505050505050505050505062000a56565b6000683635c9adc5dea00000905090565b6000680ad78ebc5ac6200000808381620004ef57fe5b04029050919050565b6000806000905062000512836000620007f460201b60201c565b81101562000530576200052d836000620007f460201b60201c565b90505b62000543836001620007f460201b60201c565b81101562000561576200055e836001620007f460201b60201c565b90505b62000574836002620007f460201b60201c565b81101562000592576200058f836002620007f460201b60201c565b90505b620005a5836003620007f460201b60201c565b811015620005c357620005c0836003620007f460201b60201c565b90505b620005d6836004620007f460201b60201c565b811015620005f457620005f1836004620007f460201b60201c565b90505b62000607836005620007f460201b60201c565b811015620006255762000622836005620007f460201b60201c565b90505b62000638836006620007f460201b60201c565b811015620006565762000653836006620007f460201b60201c565b90505b62000669836007620007f460201b60201c565b811015620006875762000684836007620007f460201b60201c565b90505b6200069a836008620007f460201b60201c565b811015620006b857620006b5836008620007f460201b60201c565b90505b620006cb836009620007f460201b60201c565b811015620006e957620006e6836009620007f460201b60201c565b90505b620006fc83600a620007f460201b60201c565b8110156200071a576200071783600a620007f460201b60201c565b90505b6200072d83600b620007f460201b60201c565b8110156200074b576200074883600b620007f460201b60201c565b90505b6200075e83600c620007f460201b60201c565b8110156200077c576200077983600c620007f460201b60201c565b90505b6200078f83600d620007f460201b60201c565b811015620007ad57620007aa83600d620007f460201b60201c565b90505b620007c083600e620007f460201b60201c565b811015620007de57620007db83600e620007f460201b60201c565b90505b828114915050919050565b600062015180905090565b60006008600e8111156200080457fe5b82600e8111156200081157fe5b14806200083657506009600e8111156200082757fe5b82600e8111156200083457fe5b145b806200085a5750600a600e8111156200084b57fe5b82600e8111156200085857fe5b145b806200087e5750600b600e8111156200086f57fe5b82600e8111156200087c57fe5b145b80620008a25750600c600e8111156200089357fe5b82600e811115620008a057fe5b145b80620008c65750600d600e811115620008b757fe5b82600e811115620008c457fe5b145b80620008e95750600e80811115620008da57fe5b82600e811115620008e757fe5b145b15620009035760018381620008fa57fe5b04905062000a50565b6004600e8111156200091157fe5b82600e8111156200091e57fe5b14156200093957600883816200093057fe5b04905062000a50565b6005600e8111156200094757fe5b82600e8111156200095457fe5b14156200096f57600683816200096657fe5b04905062000a50565b6006600e8111156200097d57fe5b82600e8111156200098a57fe5b1415620009a557600383816200099c57fe5b04905062000a50565b6007600e811115620009b357fe5b82600e811115620009c057fe5b1415620009db5760028381620009d257fe5b04905062000a50565b6002600e811115620009e957fe5b82600e811115620009f657fe5b141562000a11576001838162000a0857fe5b04905062000a50565b6000600e81111562000a1f57fe5b82600e81111562000a2c57fe5b141562000a4b5762000a43620004c860201b60201c565b905062000a50565b600090505b92915050565b6128258062000a666000396000f3fe6080604052600436106101355760003560e01c8063996b6fc1116100ab578063b7213b521161006f578063b7213b5214610460578063c6ee20d2146104af578063c9744029146104e8578063cea1295914610513578063d7606c4114610541578063fa6642c61461059d57610135565b8063996b6fc11461034a578063a307afda14610375578063a81af5f7146103a0578063a89ae4ba146103cb578063ab7695121461042257610135565b80632b68bb2d116100fd5780632b68bb2d1461021c5780633e9ee9e5146102265780635e4ff9b7146102515780636f09caed1461028f5780637150d8ae146102c857806392e4a9ea1461031f57610135565b806308551a531461013a5780630a9020271461019157806311da60b4146101bc578063142dfaa6146101c65780631856c845146101f1575b600080fd5b34801561014657600080fd5b5061014f6105f0565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34801561019d57600080fd5b506101a6610615565b6040518082815260200191505060405180910390f35b6101c461061b565b005b3480156101d257600080fd5b506101db610b52565b6040518082815260200191505060405180910390f35b3480156101fd57600080fd5b50610206610b5d565b6040518082815260200191505060405180910390f35b610224610b63565b005b34801561023257600080fd5b5061023b610d48565b6040518082815260200191505060405180910390f35b34801561025d57600080fd5b5061028d6004803603602081101561027457600080fd5b81019080803560ff169060200190929190505050610d53565b005b34801561029b57600080fd5b506102a4610f88565b6040518082600e8111156102b457fe5b60ff16815260200191505060405180910390f35b3480156102d457600080fd5b506102dd610f9b565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34801561032b57600080fd5b50610334610fc1565b6040518082815260200191505060405180910390f35b34801561035657600080fd5b5061035f610fcc565b6040518082815260200191505060405180910390f35b34801561038157600080fd5b5061038a610fd7565b6040518082815260200191505060405180910390f35b3480156103ac57600080fd5b506103b5610fdd565b6040518082815260200191505060405180910390f35b3480156103d757600080fd5b506103e0610fe3565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34801561042e57600080fd5b5061045e6004803603602081101561044557600080fd5b81019080803560ff169060200190929190505050611009565b005b34801561046c57600080fd5b506104996004803603602081101561048357600080fd5b810190808035906020019092919050505061121a565b6040518082815260200191505060405180910390f35b3480156104bb57600080fd5b506104c4611238565b604051808260048111156104d457fe5b60ff16815260200191505060405180910390f35b3480156104f457600080fd5b506104fd61124b565b6040518082815260200191505060405180910390f35b61053f6004803603602081101561052957600080fd5b810190808035906020019092919050505061125c565b005b34801561054d57600080fd5b506105876004803603604081101561056457600080fd5b8101908080359060200190929190803560ff1690602001909291905050506116c6565b6040518082815260200191505060405180910390f35b3480156105a957600080fd5b506105d6600480360360208110156105c057600080fd5b81019080803590602001909291905050506118ec565b604051808215151515815260200191505060405180910390f35b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b60035481565b42610624610fcc565b600554011061067e576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260438152602001806127ae6043913960600191505060405180910390fd5b6002600481111561068b57fe5b600760009054906101000a900460ff1660048111156106a657fe5b14806106d75750600160048111156106ba57fe5b600760009054906101000a900460ff1660048111156106d557fe5b145b61072c576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252602e815260200180612716602e913960400191505060405180910390fd5b6000600260009054906101000a900473ffffffffffffffffffffffffffffffffffffffff16905060008173ffffffffffffffffffffffffffffffffffffffff16638da5cb5b6040518163ffffffff1660e01b815260040160206040518083038186803b15801561079b57600080fd5b505afa1580156107af573d6000803e3d6000fd5b505050506040513d60208110156107c557600080fd5b8101908080519060200190929190505050905060003073ffffffffffffffffffffffffffffffffffffffff163190506000610811600354600760019054906101000a900460ff166116c6565b9050808203915060008114610999576000600e81111561082d57fe5b600760019054906101000a900460ff16600e81111561084857fe5b1461092b5760008473ffffffffffffffffffffffffffffffffffffffff166329e2caad836040518263ffffffff1660e01b81526004018082815260200191505060206040518083038186803b1580156108a057600080fd5b505afa1580156108b4573d6000803e3d6000fd5b505050506040513d60208110156108ca57600080fd5b8101908080519060200190929190505050905080820391508373ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f19350505050158015610928573d6000803e3d6000fd5b50505b600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f19350505050158015610993573d6000803e3d6000fd5b50610aac565b6000600e8111156109a657fe5b600760019054906101000a900460ff16600e8111156109c157fe5b14610aab5760008473ffffffffffffffffffffffffffffffffffffffff166329e2caad6109ec61124b565b6040518263ffffffff1660e01b81526004018082815260200191505060206040518083038186803b158015610a2057600080fd5b505afa158015610a34573d6000803e3d6000fd5b505050506040513d6020811015610a4a57600080fd5b8101908080519060200190929190505050905080830392508373ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f19350505050158015610aa8573d6000803e3d6000fd5b50505b5b6000829050818303925060008114610b27576000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f19350505050158015610b25573d6000803e3d6000fd5b505b6003600760006101000a81548160ff02191690836004811115610b4657fe5b02179055505050505050565b600062015180905090565b60045481565b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614610c08576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260238152602001806126bb6023913960400191505060405180910390fd5b60006004811115610c1557fe5b600760009054906101000a900460ff166004811115610c3057fe5b14610ca3576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252601f8152602001807f636f6e747261637420737461747573206e65656420746f206265206f70656e0081525060200191505060405180910390fd5b6004600760006101000a81548160ff02191690836004811115610cc257fe5b02179055506000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166108fc3073ffffffffffffffffffffffffffffffffffffffff16319081150290604051600060405180830381858888f19350505050158015610d45573d6000803e3d6000fd5b50565b600062015180905090565b600260009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614610df9576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260288152602001806127866028913960400191505060405180910390fd5b42610e02610b52565b6005540110610e5c576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260448152602001806126076044913960600191505060405180910390fd5b42610e65610fc1565b600554011015610ec0576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252604381526020018061264b6043913960600191505060405180910390fd5b60016004811115610ecd57fe5b600760009054906101000a900460ff166004811115610ee857fe5b14610f3e576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252602d81526020018061268e602d913960400191505060405180910390fd5b80600760016101000a81548160ff0219169083600e811115610f5c57fe5b02179055506002600760006101000a81548160ff02191690836004811115610f8057fe5b021790555050565b600760019054906101000a900460ff1681565b600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b60006203f480905090565b600062054600905090565b60055481565b60065481565b600260009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600260009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16146110af576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252602881526020018061259b6028913960400191505060405180910390fd5b426110b8610fc1565b6005540110611112576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260428152602001806127446042913960600191505060405180910390fd5b4261111b610fcc565b600554011015611176576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260448152602001806125c36044913960600191505060405180910390fd5b6002600481111561118357fe5b600760009054906101000a900460ff16600481111561119e57fe5b146111f4576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252602e815260200180612716602e913960400191505060405180910390fd5b80600760016101000a81548160ff0219169083600e81111561121257fe5b021790555050565b6000680ad78ebc5ac620000080838161122f57fe5b04029050919050565b600760009054906101000a900460ff1681565b6000683635c9adc5dea00000905090565b61126461124b565b3410156112d9576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252601e8152602001807f696e73756666696369656e742066756e6420666f72207072656d69756d21000081525060200191505060405180910390fd5b426112e2610d48565b600554031161133c576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260388152602001806126de6038913960400191505060405180910390fd5b6000600481111561134957fe5b600760009054906101000a900460ff16600481111561136457fe5b146113d7576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252601f8152602001807f636f6e747261637420737461747573206e65656420746f206265206f70656e0081525060200191505060405180910390fd5b600115156113e4826118ec565b151514611459576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252601a8152602001807f646f206e6f7420737570706f7274207468697320666c6967687400000000000081525060200191505060405180910390fd5b33600160006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff16021790555060003490506114a761124b565b81039050600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff167f17c2f3e0d2d7ead2d68931154e83b5320959605ec7397093a1bd845a60e29c4761150d61124b565b6040518082815260200191505060405180910390a2600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f1935050505015801561158a573d6000803e3d6000fd5b5081600481905550600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff167fd6457c06d7a832c2724560b589156fc0a34d1eea0aae901b1e12d5e279a8f99a826040518082815260200191505060405180910390a2600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff167f596bcc7183bdee7c7371c9072d6e4c6b30dea4f278756da86ed4ec7e471c484030604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390a26001600760006101000a81548160ff021916908360048111156116bd57fe5b02179055505050565b60006008600e8111156116d557fe5b82600e8111156116e157fe5b148061170357506009600e8111156116f557fe5b82600e81111561170157fe5b145b806117245750600a600e81111561171657fe5b82600e81111561172257fe5b145b806117455750600b600e81111561173757fe5b82600e81111561174357fe5b145b806117665750600c600e81111561175857fe5b82600e81111561176457fe5b145b806117875750600d600e81111561177957fe5b82600e81111561178557fe5b145b806117a75750600e8081111561179957fe5b82600e8111156117a557fe5b145b156117be57600183816117b657fe5b0490506118e6565b6004600e8111156117cb57fe5b82600e8111156117d757fe5b14156117ef57600883816117e757fe5b0490506118e6565b6005600e8111156117fc57fe5b82600e81111561180857fe5b1415611820576006838161181857fe5b0490506118e6565b6006600e81111561182d57fe5b82600e81111561183957fe5b1415611851576003838161184957fe5b0490506118e6565b6007600e81111561185e57fe5b82600e81111561186a57fe5b1415611882576002838161187a57fe5b0490506118e6565b6002600e81111561188f57fe5b82600e81111561189b57fe5b14156118b357600183816118ab57fe5b0490506118e6565b6000600e8111156118c057fe5b82600e8111156118cc57fe5b14156118e1576118da61124b565b90506118e6565b600090505b92915050565b60007f415320323636000000000000000000000000000000000000000000000000000082148061193b57507f415320333236000000000000000000000000000000000000000000000000000082145b8061196557507f415320333430000000000000000000000000000000000000000000000000000082145b8061198f57507f415320333633000000000000000000000000000000000000000000000000000082145b806119b957507f415320333735000000000000000000000000000000000000000000000000000082145b806119e357507f415320373833000000000000000000000000000000000000000000000000000082145b80611a0d57507f415320363237000000000000000000000000000000000000000000000000000082145b80611a3757507f415320363239000000000000000000000000000000000000000000000000000082145b80611a6157507f415320363537000000000000000000000000000000000000000000000000000082145b80611a8b57507f415320363635000000000000000000000000000000000000000000000000000082145b80611ab557507f415320373836000000000000000000000000000000000000000000000000000082145b80611adf57507f415320313032320000000000000000000000000000000000000000000000000082145b80611b0957507f415320313032340000000000000000000000000000000000000000000000000082145b15611b175760019050612595565b7f5541203231330000000000000000000000000000000000000000000000000000821480611b6457507f554120323935000000000000000000000000000000000000000000000000000082145b80611b8e57507f554120343937000000000000000000000000000000000000000000000000000082145b80611bb857507f554120353335000000000000000000000000000000000000000000000000000082145b80611be257507f554120353737000000000000000000000000000000000000000000000000000082145b80611c0c57507f554120353833000000000000000000000000000000000000000000000000000082145b80611c3657507f554120373533000000000000000000000000000000000000000000000000000082145b80611c6057507f554120383430000000000000000000000000000000000000000000000000000082145b80611c8a57507f554120313235370000000000000000000000000000000000000000000000000082145b80611cb457507f554120313438330000000000000000000000000000000000000000000000000082145b80611cde57507f554120313532360000000000000000000000000000000000000000000000000082145b80611d0857507f554120313538340000000000000000000000000000000000000000000000000082145b80611d3257507f554120313739360000000000000000000000000000000000000000000000000082145b80611d5c57507f554120313834380000000000000000000000000000000000000000000000000082145b80611d8657507f554120313937380000000000000000000000000000000000000000000000000082145b80611db057507f554120323030360000000000000000000000000000000000000000000000000082145b80611dda57507f554120323034340000000000000000000000000000000000000000000000000082145b80611e0457507f554120323036350000000000000000000000000000000000000000000000000082145b80611e2e57507f554120323038300000000000000000000000000000000000000000000000000082145b80611e5857507f554120323136300000000000000000000000000000000000000000000000000082145b80611e8257507f554120323233390000000000000000000000000000000000000000000000000082145b80611eac57507f554120323331390000000000000000000000000000000000000000000000000082145b15611eba5760019050612595565b7f4141203136000000000000000000000000000000000000000000000000000000821480611f0757507f414120373600000000000000000000000000000000000000000000000000000082145b80611f3157507f414120313634000000000000000000000000000000000000000000000000000082145b80611f5b57507f414120313636000000000000000000000000000000000000000000000000000082145b80611f8557507f414120313737000000000000000000000000000000000000000000000000000082145b80611faf57507f414120313739000000000000000000000000000000000000000000000000000082145b80611fd957507f414120323334000000000000000000000000000000000000000000000000000082145b8061200357507f414120323736000000000000000000000000000000000000000000000000000082145b8061202d57507f414120323330350000000000000000000000000000000000000000000000000082145b8061205757507f414120323635320000000000000000000000000000000000000000000000000082145b156120655760019050612595565b7f42362031350000000000000000000000000000000000000000000000000000008214806120b257507f423620313600000000000000000000000000000000000000000000000000000082145b806120dc57507f423620313637000000000000000000000000000000000000000000000000000082145b8061210657507f423620313638000000000000000000000000000000000000000000000000000082145b8061213057507f423620343135000000000000000000000000000000000000000000000000000082145b8061215a57507f423620343136000000000000000000000000000000000000000000000000000082145b8061218457507f423620353136000000000000000000000000000000000000000000000000000082145b806121ae57507f423620363135000000000000000000000000000000000000000000000000000082145b806121d857507f423620363136000000000000000000000000000000000000000000000000000082145b8061220257507f423620363639000000000000000000000000000000000000000000000000000082145b8061222c57507f423620363730000000000000000000000000000000000000000000000000000082145b8061225657507f423620393135000000000000000000000000000000000000000000000000000082145b8061228057507f423620393136000000000000000000000000000000000000000000000000000082145b806122aa57507f423620313431350000000000000000000000000000000000000000000000000082145b806122d457507f423620313531360000000000000000000000000000000000000000000000000082145b806122fe57507f423620313731350000000000000000000000000000000000000000000000000082145b1561230c5760019050612595565b7f444c20343236000000000000000000000000000000000000000000000000000082148061235957507f444c20343330000000000000000000000000000000000000000000000000000082145b8061238357507f444c20343930000000000000000000000000000000000000000000000000000082145b806123ad57507f444c20363130000000000000000000000000000000000000000000000000000082145b806123d757507f444c20363433000000000000000000000000000000000000000000000000000082145b8061240157507f444c20383638000000000000000000000000000000000000000000000000000082145b8061242b57507f444c20393336000000000000000000000000000000000000000000000000000082145b8061245557507f444c20313534380000000000000000000000000000000000000000000000000082145b8061247f57507f444c20313835390000000000000000000000000000000000000000000000000082145b806124a957507f444c20323637300000000000000000000000000000000000000000000000000082145b156124b75760019050612595565b7f484120323335330000000000000000000000000000000000000000000000000082148061250457507f484120323335320000000000000000000000000000000000000000000000000082145b8061252e57507f484120323336300000000000000000000000000000000000000000000000000082145b8061255857507f484120323336310000000000000000000000000000000000000000000000000082145b8061258257507f484120323336320000000000000000000000000000000000000000000000000082145b156125905760019050612595565b600090505b91905056fe6f6e6c7920726567697374726564206f7261636c652063616e206368616e676520726573756c742173686f756c64206368616e6765206265666f726520287363686564756c655f74616b655f6f66665f74696d65202b20656e645f646973707574655f696e74657276616c2973686f756c64207265706f727420616674657220287363686564756c655f74616b655f6f66665f74696d65202b2073746172745f7265706f72745f696e74657276616c2973686f756c64207265706f7274206265666f726520287363686564756c655f74616b655f6f66665f74696d65202b20656e645f7265706f72745f696e74657276616c29636f6e747261637420737461747573206e65656420746f2062652077616974696e6720666f72207265706f72746f6e6c79206f776e65722063616e2063616e63656c207468697320636f6e747261637473686f756c64206275792067657446726f7a656e506572696f64206265666f7265207363686564756c652074616b65206f66662074696d65636f6e747261637420737461747573206e65656420746f2062652077616974696e6720666f72206469737075746573686f756c64206368616e676520616674657220287363686564756c655f74616b655f6f66665f74696d65202b20656e645f7265706f72745f696e74657276616c296f6e6c7920726567697374726564206f7261636c652063616e207265706f727420726573756c742173686f756c6420736574746c6520616674657220287363686564756c655f74616b655f6f66665f74696d65202b20656e645f646973707574655f696e74657276616c29a265627a7a72315820612ed13838b4a948337cce571ea68c9fc0e970d7cf3fb2271f2606a6dc6b4f4264736f6c634300050b00327265616c206d61782062656e656669742073686f756c6420657175616c20746f206d61782062656e6566697421696e73756666696369656e7420696e697469616c206465706f7369742066756e64217363686564756c652074616b65206f66662074696d652073686f756c64206174206c65617374206d61782062656e656669742073686f756c64206c6172676572207468616e207072656d69756d00000000000000000000000000000000000000000000021e19e0c9bab2400000323031392d31302d313700000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000005da9004400000000000000000000000079f3574d440c21029eecb8e9ee66b23b26cbcb38") - _constantinopleOpCodeContract, _ = hex.DecodeString("608060405234801561001057600080fd5b506104d5806100206000396000f3fe608060405234801561001057600080fd5b50600436106100885760003560e01c806381ea44081161005b57806381ea440814610101578063a91b336214610159578063c2bc2efc14610177578063f5eacece146101cf57610088565b80635bec9e671461008d57806360fe47b1146100975780636bc8ecaa146100c5578063744f5f83146100e3575b600080fd5b6100956101ed565b005b6100c3600480360360208110156100ad57600080fd5b8101908080359060200190929190505050610239565b005b6100cd610270565b6040518082815260200191505060405180910390f35b6100eb6102b3565b6040518082815260200191505060405180910390f35b6101436004803603602081101561011757600080fd5b81019080803573ffffffffffffffffffffffffffffffffffffffff1690602001909291905050506102f6565b6040518082815260200191505060405180910390f35b61016161036a565b6040518082815260200191505060405180910390f35b6101b96004803603602081101561018d57600080fd5b81019080803573ffffffffffffffffffffffffffffffffffffffff1690602001909291905050506103ad565b6040518082815260200191505060405180910390f35b6101d761045f565b6040518082815260200191505060405180910390f35b5b60011561020b5760008081548092919060010191905055506101ee565b7f8bfaa460932ccf8751604dd60efa3eafa220ec358fccb32ef703f91c509bc3ea60405160405180910390a1565b80600081905550807fdf7a95aebff315db1b7716215d602ab537373cdb769232aae6055c06e798425b60405160405180910390a250565b6000805460081d905080600081905550807fdf7a95aebff315db1b7716215d602ab537373cdb769232aae6055c06e798425b60405160405180910390a280905090565b6000805460081c905080600081905550807fdf7a95aebff315db1b7716215d602ab537373cdb769232aae6055c06e798425b60405160405180910390a280905090565b60008073ffffffffffffffffffffffffffffffffffffffff168273ffffffffffffffffffffffffffffffffffffffff16141561033157600080fd5b813f9050807fdf7a95aebff315db1b7716215d602ab537373cdb769232aae6055c06e798425b60405160405180910390a2809050919050565b6000805460081b905080600081905550807fdf7a95aebff315db1b7716215d602ab537373cdb769232aae6055c06e798425b60405160405180910390a280905090565b60008073ffffffffffffffffffffffffffffffffffffffff168273ffffffffffffffffffffffffffffffffffffffff1614156103e857600080fd5b7fbde7a70c2261170a87678200113c8e12f82f63d0a1d1cfa45681cbac328e87e382600054604051808373ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020018281526020019250505060405180910390a16000549050919050565b60008080602060406000f59150817fdf7a95aebff315db1b7716215d602ab537373cdb769232aae6055c06e798425b60405160405180910390a2819150509056fea265627a7a72305820209a8ef04c4d621759f34878b27b238650e8605c8a71d6efc619a769a64aa9cc64736f6c634300050a0032") + _codeStoreOutOfGasContract, _ = hex.DecodeString("60806040526040516200332738038062003327833981810160405260808110156200002957600080fd5b8101908080519060200190929190805190602001909291908051906020019092919080519060200190929190505050838383836200006c620004c860201b60201c565b8411620000c5576040517f08c379a0000000000000000000000000000000000000000000000000000000008152600401808060200182810382526026815260200180620033016026913960400191505060405180910390fd5b6000620000d885620004d960201b60201c565b9050620000eb81620004f860201b60201c565b62000142576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252602d8152602001806200328b602d913960400191505060405180910390fd5b803410156200019d576040517f08c379a0000000000000000000000000000000000000000000000000000000008152600401808060200182810382526022815260200180620032b86022913960400191505060405180910390fd5b42620001ae620007e960201b60201c565b84031162000208576040517f08c379a0000000000000000000000000000000000000000000000000000000008152600401808060200182810382526027815260200180620032da6027913960400191505060405180910390fd5b806003819055508260058190555081600260006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff16021790555083600681905550336000806101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055506000349050600354810390506000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff167f327ca9276da9073b583440165cb887319e7aeaf4003f14e92c1bbeee913e9b9c6003546040518082815260200191505060405180910390a26000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f1935050505015801562000383573d6000803e3d6000fd5b506000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff167fb54ecacfeaef46d29d83a7ee9ffaae207283e8a0f5df648dba2d52a1454e489e826040518082815260200191505060405180910390a26000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff167f53ddc5f55aac282e58e964d06a34c46ab814f1ea04d4ccf6eed6df8ba36d3f5e600354600654604051808381526020018281526020019250505060405180910390a26000600760016101000a81548160ff0219169083600e8111156200048e57fe5b02179055506000600760006101000a81548160ff02191690836004811115620004b357fe5b02179055505050505050505050505062000a56565b6000683635c9adc5dea00000905090565b6000680ad78ebc5ac6200000808381620004ef57fe5b04029050919050565b6000806000905062000512836000620007f460201b60201c565b81101562000530576200052d836000620007f460201b60201c565b90505b62000543836001620007f460201b60201c565b81101562000561576200055e836001620007f460201b60201c565b90505b62000574836002620007f460201b60201c565b81101562000592576200058f836002620007f460201b60201c565b90505b620005a5836003620007f460201b60201c565b811015620005c357620005c0836003620007f460201b60201c565b90505b620005d6836004620007f460201b60201c565b811015620005f457620005f1836004620007f460201b60201c565b90505b62000607836005620007f460201b60201c565b811015620006255762000622836005620007f460201b60201c565b90505b62000638836006620007f460201b60201c565b811015620006565762000653836006620007f460201b60201c565b90505b62000669836007620007f460201b60201c565b811015620006875762000684836007620007f460201b60201c565b90505b6200069a836008620007f460201b60201c565b811015620006b857620006b5836008620007f460201b60201c565b90505b620006cb836009620007f460201b60201c565b811015620006e957620006e6836009620007f460201b60201c565b90505b620006fc83600a620007f460201b60201c565b8110156200071a576200071783600a620007f460201b60201c565b90505b6200072d83600b620007f460201b60201c565b8110156200074b576200074883600b620007f460201b60201c565b90505b6200075e83600c620007f460201b60201c565b8110156200077c576200077983600c620007f460201b60201c565b90505b6200078f83600d620007f460201b60201c565b811015620007ad57620007aa83600d620007f460201b60201c565b90505b620007c083600e620007f460201b60201c565b811015620007de57620007db83600e620007f460201b60201c565b90505b828114915050919050565b600062015180905090565b60006008600e8111156200080457fe5b82600e8111156200081157fe5b14806200083657506009600e8111156200082757fe5b82600e8111156200083457fe5b145b806200085a5750600a600e8111156200084b57fe5b82600e8111156200085857fe5b145b806200087e5750600b600e8111156200086f57fe5b82600e8111156200087c57fe5b145b80620008a25750600c600e8111156200089357fe5b82600e811115620008a057fe5b145b80620008c65750600d600e811115620008b757fe5b82600e811115620008c457fe5b145b80620008e95750600e80811115620008da57fe5b82600e811115620008e757fe5b145b15620009035760018381620008fa57fe5b04905062000a50565b6004600e8111156200091157fe5b82600e8111156200091e57fe5b14156200093957600883816200093057fe5b04905062000a50565b6005600e8111156200094757fe5b82600e8111156200095457fe5b14156200096f57600683816200096657fe5b04905062000a50565b6006600e8111156200097d57fe5b82600e8111156200098a57fe5b1415620009a557600383816200099c57fe5b04905062000a50565b6007600e811115620009b357fe5b82600e811115620009c057fe5b1415620009db5760028381620009d257fe5b04905062000a50565b6002600e811115620009e957fe5b82600e811115620009f657fe5b141562000a11576001838162000a0857fe5b04905062000a50565b6000600e81111562000a1f57fe5b82600e81111562000a2c57fe5b141562000a4b5762000a43620004c860201b60201c565b905062000a50565b600090505b92915050565b6128258062000a666000396000f3fe6080604052600436106101355760003560e01c8063996b6fc1116100ab578063b7213b521161006f578063b7213b5214610460578063c6ee20d2146104af578063c9744029146104e8578063cea1295914610513578063d7606c4114610541578063fa6642c61461059d57610135565b8063996b6fc11461034a578063a307afda14610375578063a81af5f7146103a0578063a89ae4ba146103cb578063ab7695121461042257610135565b80632b68bb2d116100fd5780632b68bb2d1461021c5780633e9ee9e5146102265780635e4ff9b7146102515780636f09caed1461028f5780637150d8ae146102c857806392e4a9ea1461031f57610135565b806308551a531461013a5780630a9020271461019157806311da60b4146101bc578063142dfaa6146101c65780631856c845146101f1575b600080fd5b34801561014657600080fd5b5061014f6105f0565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34801561019d57600080fd5b506101a6610615565b6040518082815260200191505060405180910390f35b6101c461061b565b005b3480156101d257600080fd5b506101db610b52565b6040518082815260200191505060405180910390f35b3480156101fd57600080fd5b50610206610b5d565b6040518082815260200191505060405180910390f35b610224610b63565b005b34801561023257600080fd5b5061023b610d48565b6040518082815260200191505060405180910390f35b34801561025d57600080fd5b5061028d6004803603602081101561027457600080fd5b81019080803560ff169060200190929190505050610d53565b005b34801561029b57600080fd5b506102a4610f88565b6040518082600e8111156102b457fe5b60ff16815260200191505060405180910390f35b3480156102d457600080fd5b506102dd610f9b565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34801561032b57600080fd5b50610334610fc1565b6040518082815260200191505060405180910390f35b34801561035657600080fd5b5061035f610fcc565b6040518082815260200191505060405180910390f35b34801561038157600080fd5b5061038a610fd7565b6040518082815260200191505060405180910390f35b3480156103ac57600080fd5b506103b5610fdd565b6040518082815260200191505060405180910390f35b3480156103d757600080fd5b506103e0610fe3565b604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390f35b34801561042e57600080fd5b5061045e6004803603602081101561044557600080fd5b81019080803560ff169060200190929190505050611009565b005b34801561046c57600080fd5b506104996004803603602081101561048357600080fd5b810190808035906020019092919050505061121a565b6040518082815260200191505060405180910390f35b3480156104bb57600080fd5b506104c4611238565b604051808260048111156104d457fe5b60ff16815260200191505060405180910390f35b3480156104f457600080fd5b506104fd61124b565b6040518082815260200191505060405180910390f35b61053f6004803603602081101561052957600080fd5b810190808035906020019092919050505061125c565b005b34801561054d57600080fd5b506105876004803603604081101561056457600080fd5b8101908080359060200190929190803560ff1690602001909291905050506116c6565b6040518082815260200191505060405180910390f35b3480156105a957600080fd5b506105d6600480360360208110156105c057600080fd5b81019080803590602001909291905050506118ec565b604051808215151515815260200191505060405180910390f35b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b60035481565b42610624610fcc565b600554011061067e576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260438152602001806127ae6043913960600191505060405180910390fd5b6002600481111561068b57fe5b600760009054906101000a900460ff1660048111156106a657fe5b14806106d75750600160048111156106ba57fe5b600760009054906101000a900460ff1660048111156106d557fe5b145b61072c576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252602e815260200180612716602e913960400191505060405180910390fd5b6000600260009054906101000a900473ffffffffffffffffffffffffffffffffffffffff16905060008173ffffffffffffffffffffffffffffffffffffffff16638da5cb5b6040518163ffffffff1660e01b815260040160206040518083038186803b15801561079b57600080fd5b505afa1580156107af573d6000803e3d6000fd5b505050506040513d60208110156107c557600080fd5b8101908080519060200190929190505050905060003073ffffffffffffffffffffffffffffffffffffffff163190506000610811600354600760019054906101000a900460ff166116c6565b9050808203915060008114610999576000600e81111561082d57fe5b600760019054906101000a900460ff16600e81111561084857fe5b1461092b5760008473ffffffffffffffffffffffffffffffffffffffff166329e2caad836040518263ffffffff1660e01b81526004018082815260200191505060206040518083038186803b1580156108a057600080fd5b505afa1580156108b4573d6000803e3d6000fd5b505050506040513d60208110156108ca57600080fd5b8101908080519060200190929190505050905080820391508373ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f19350505050158015610928573d6000803e3d6000fd5b50505b600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f19350505050158015610993573d6000803e3d6000fd5b50610aac565b6000600e8111156109a657fe5b600760019054906101000a900460ff16600e8111156109c157fe5b14610aab5760008473ffffffffffffffffffffffffffffffffffffffff166329e2caad6109ec61124b565b6040518263ffffffff1660e01b81526004018082815260200191505060206040518083038186803b158015610a2057600080fd5b505afa158015610a34573d6000803e3d6000fd5b505050506040513d6020811015610a4a57600080fd5b8101908080519060200190929190505050905080830392508373ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f19350505050158015610aa8573d6000803e3d6000fd5b50505b5b6000829050818303925060008114610b27576000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f19350505050158015610b25573d6000803e3d6000fd5b505b6003600760006101000a81548160ff02191690836004811115610b4657fe5b02179055505050505050565b600062015180905090565b60045481565b6000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614610c08576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260238152602001806126bb6023913960400191505060405180910390fd5b60006004811115610c1557fe5b600760009054906101000a900460ff166004811115610c3057fe5b14610ca3576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252601f8152602001807f636f6e747261637420737461747573206e65656420746f206265206f70656e0081525060200191505060405180910390fd5b6004600760006101000a81548160ff02191690836004811115610cc257fe5b02179055506000809054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166108fc3073ffffffffffffffffffffffffffffffffffffffff16319081150290604051600060405180830381858888f19350505050158015610d45573d6000803e3d6000fd5b50565b600062015180905090565b600260009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614610df9576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260288152602001806127866028913960400191505060405180910390fd5b42610e02610b52565b6005540110610e5c576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260448152602001806126076044913960600191505060405180910390fd5b42610e65610fc1565b600554011015610ec0576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252604381526020018061264b6043913960600191505060405180910390fd5b60016004811115610ecd57fe5b600760009054906101000a900460ff166004811115610ee857fe5b14610f3e576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252602d81526020018061268e602d913960400191505060405180910390fd5b80600760016101000a81548160ff0219169083600e811115610f5c57fe5b02179055506002600760006101000a81548160ff02191690836004811115610f8057fe5b021790555050565b600760019054906101000a900460ff1681565b600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b60006203f480905090565b600062054600905090565b60055481565b60065481565b600260009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b600260009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff16146110af576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252602881526020018061259b6028913960400191505060405180910390fd5b426110b8610fc1565b6005540110611112576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260428152602001806127446042913960600191505060405180910390fd5b4261111b610fcc565b600554011015611176576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260448152602001806125c36044913960600191505060405180910390fd5b6002600481111561118357fe5b600760009054906101000a900460ff16600481111561119e57fe5b146111f4576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252602e815260200180612716602e913960400191505060405180910390fd5b80600760016101000a81548160ff0219169083600e81111561121257fe5b021790555050565b6000680ad78ebc5ac620000080838161122f57fe5b04029050919050565b600760009054906101000a900460ff1681565b6000683635c9adc5dea00000905090565b61126461124b565b3410156112d9576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252601e8152602001807f696e73756666696369656e742066756e6420666f72207072656d69756d21000081525060200191505060405180910390fd5b426112e2610d48565b600554031161133c576040517f08c379a00000000000000000000000000000000000000000000000000000000081526004018080602001828103825260388152602001806126de6038913960400191505060405180910390fd5b6000600481111561134957fe5b600760009054906101000a900460ff16600481111561136457fe5b146113d7576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252601f8152602001807f636f6e747261637420737461747573206e65656420746f206265206f70656e0081525060200191505060405180910390fd5b600115156113e4826118ec565b151514611459576040517f08c379a000000000000000000000000000000000000000000000000000000000815260040180806020018281038252601a8152602001807f646f206e6f7420737570706f7274207468697320666c6967687400000000000081525060200191505060405180910390fd5b33600160006101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff16021790555060003490506114a761124b565b81039050600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff167f17c2f3e0d2d7ead2d68931154e83b5320959605ec7397093a1bd845a60e29c4761150d61124b565b6040518082815260200191505060405180910390a2600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff166108fc829081150290604051600060405180830381858888f1935050505015801561158a573d6000803e3d6000fd5b5081600481905550600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff167fd6457c06d7a832c2724560b589156fc0a34d1eea0aae901b1e12d5e279a8f99a826040518082815260200191505060405180910390a2600160009054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff167f596bcc7183bdee7c7371c9072d6e4c6b30dea4f278756da86ed4ec7e471c484030604051808273ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16815260200191505060405180910390a26001600760006101000a81548160ff021916908360048111156116bd57fe5b02179055505050565b60006008600e8111156116d557fe5b82600e8111156116e157fe5b148061170357506009600e8111156116f557fe5b82600e81111561170157fe5b145b806117245750600a600e81111561171657fe5b82600e81111561172257fe5b145b806117455750600b600e81111561173757fe5b82600e81111561174357fe5b145b806117665750600c600e81111561175857fe5b82600e81111561176457fe5b145b806117875750600d600e81111561177957fe5b82600e81111561178557fe5b145b806117a75750600e8081111561179957fe5b82600e8111156117a557fe5b145b156117be57600183816117b657fe5b0490506118e6565b6004600e8111156117cb57fe5b82600e8111156117d757fe5b14156117ef57600883816117e757fe5b0490506118e6565b6005600e8111156117fc57fe5b82600e81111561180857fe5b1415611820576006838161181857fe5b0490506118e6565b6006600e81111561182d57fe5b82600e81111561183957fe5b1415611851576003838161184957fe5b0490506118e6565b6007600e81111561185e57fe5b82600e81111561186a57fe5b1415611882576002838161187a57fe5b0490506118e6565b6002600e81111561188f57fe5b82600e81111561189b57fe5b14156118b357600183816118ab57fe5b0490506118e6565b6000600e8111156118c057fe5b82600e8111156118cc57fe5b14156118e1576118da61124b565b90506118e6565b600090505b92915050565b60007f415320323636000000000000000000000000000000000000000000000000000082148061193b57507f415320333236000000000000000000000000000000000000000000000000000082145b8061196557507f415320333430000000000000000000000000000000000000000000000000000082145b8061198f57507f415320333633000000000000000000000000000000000000000000000000000082145b806119b957507f415320333735000000000000000000000000000000000000000000000000000082145b806119e357507f415320373833000000000000000000000000000000000000000000000000000082145b80611a0d57507f415320363237000000000000000000000000000000000000000000000000000082145b80611a3757507f415320363239000000000000000000000000000000000000000000000000000082145b80611a6157507f415320363537000000000000000000000000000000000000000000000000000082145b80611a8b57507f415320363635000000000000000000000000000000000000000000000000000082145b80611ab557507f415320373836000000000000000000000000000000000000000000000000000082145b80611adf57507f415320313032320000000000000000000000000000000000000000000000000082145b80611b0957507f415320313032340000000000000000000000000000000000000000000000000082145b15611b175760019050612595565b7f5541203231330000000000000000000000000000000000000000000000000000821480611b6457507f554120323935000000000000000000000000000000000000000000000000000082145b80611b8e57507f554120343937000000000000000000000000000000000000000000000000000082145b80611bb857507f554120353335000000000000000000000000000000000000000000000000000082145b80611be257507f554120353737000000000000000000000000000000000000000000000000000082145b80611c0c57507f554120353833000000000000000000000000000000000000000000000000000082145b80611c3657507f554120373533000000000000000000000000000000000000000000000000000082145b80611c6057507f554120383430000000000000000000000000000000000000000000000000000082145b80611c8a57507f554120313235370000000000000000000000000000000000000000000000000082145b80611cb457507f554120313438330000000000000000000000000000000000000000000000000082145b80611cde57507f554120313532360000000000000000000000000000000000000000000000000082145b80611d0857507f554120313538340000000000000000000000000000000000000000000000000082145b80611d3257507f554120313739360000000000000000000000000000000000000000000000000082145b80611d5c57507f554120313834380000000000000000000000000000000000000000000000000082145b80611d8657507f554120313937380000000000000000000000000000000000000000000000000082145b80611db057507f554120323030360000000000000000000000000000000000000000000000000082145b80611dda57507f554120323034340000000000000000000000000000000000000000000000000082145b80611e0457507f554120323036350000000000000000000000000000000000000000000000000082145b80611e2e57507f554120323038300000000000000000000000000000000000000000000000000082145b80611e5857507f554120323136300000000000000000000000000000000000000000000000000082145b80611e8257507f554120323233390000000000000000000000000000000000000000000000000082145b80611eac57507f554120323331390000000000000000000000000000000000000000000000000082145b15611eba5760019050612595565b7f4141203136000000000000000000000000000000000000000000000000000000821480611f0757507f414120373600000000000000000000000000000000000000000000000000000082145b80611f3157507f414120313634000000000000000000000000000000000000000000000000000082145b80611f5b57507f414120313636000000000000000000000000000000000000000000000000000082145b80611f8557507f414120313737000000000000000000000000000000000000000000000000000082145b80611faf57507f414120313739000000000000000000000000000000000000000000000000000082145b80611fd957507f414120323334000000000000000000000000000000000000000000000000000082145b8061200357507f414120323736000000000000000000000000000000000000000000000000000082145b8061202d57507f414120323330350000000000000000000000000000000000000000000000000082145b8061205757507f414120323635320000000000000000000000000000000000000000000000000082145b156120655760019050612595565b7f42362031350000000000000000000000000000000000000000000000000000008214806120b257507f423620313600000000000000000000000000000000000000000000000000000082145b806120dc57507f423620313637000000000000000000000000000000000000000000000000000082145b8061210657507f423620313638000000000000000000000000000000000000000000000000000082145b8061213057507f423620343135000000000000000000000000000000000000000000000000000082145b8061215a57507f423620343136000000000000000000000000000000000000000000000000000082145b8061218457507f423620353136000000000000000000000000000000000000000000000000000082145b806121ae57507f423620363135000000000000000000000000000000000000000000000000000082145b806121d857507f423620363136000000000000000000000000000000000000000000000000000082145b8061220257507f423620363639000000000000000000000000000000000000000000000000000082145b8061222c57507f423620363730000000000000000000000000000000000000000000000000000082145b8061225657507f423620393135000000000000000000000000000000000000000000000000000082145b8061228057507f423620393136000000000000000000000000000000000000000000000000000082145b806122aa57507f423620313431350000000000000000000000000000000000000000000000000082145b806122d457507f423620313531360000000000000000000000000000000000000000000000000082145b806122fe57507f423620313731350000000000000000000000000000000000000000000000000082145b1561230c5760019050612595565b7f444c20343236000000000000000000000000000000000000000000000000000082148061235957507f444c20343330000000000000000000000000000000000000000000000000000082145b8061238357507f444c20343930000000000000000000000000000000000000000000000000000082145b806123ad57507f444c20363130000000000000000000000000000000000000000000000000000082145b806123d757507f444c20363433000000000000000000000000000000000000000000000000000082145b8061240157507f444c20383638000000000000000000000000000000000000000000000000000082145b8061242b57507f444c20393336000000000000000000000000000000000000000000000000000082145b8061245557507f444c20313534380000000000000000000000000000000000000000000000000082145b8061247f57507f444c20313835390000000000000000000000000000000000000000000000000082145b806124a957507f444c20323637300000000000000000000000000000000000000000000000000082145b156124b75760019050612595565b7f484120323335330000000000000000000000000000000000000000000000000082148061250457507f484120323335320000000000000000000000000000000000000000000000000082145b8061252e57507f484120323336300000000000000000000000000000000000000000000000000082145b8061255857507f484120323336310000000000000000000000000000000000000000000000000082145b8061258257507f484120323336320000000000000000000000000000000000000000000000000082145b156125905760019050612595565b600090505b91905056fe6f6e6c7920726567697374726564206f7261636c652063616e206368616e676520726573756c742173686f756c64206368616e6765206265666f726520287363686564756c655f74616b655f6f66665f74696d65202b20656e645f646973707574655f696e74657276616c2973686f756c64207265706f727420616674657220287363686564756c655f74616b655f6f66665f74696d65202b2073746172745f7265706f72745f696e74657276616c2973686f756c64207265706f7274206265666f726520287363686564756c655f74616b655f6f66665f74696d65202b20656e645f7265706f72745f696e74657276616c29636f6e747261637420737461747573206e65656420746f2062652077616974696e6720666f72207265706f72746f6e6c79206f776e65722063616e2063616e63656c207468697320636f6e747261637473686f756c64206275792067657446726f7a656e506572696f64206265666f7265207363686564756c652074616b65206f66662074696d65636f6e747261637420737461747573206e65656420746f2062652077616974696e6720666f72206469737075746573686f756c64206368616e676520616674657220287363686564756c655f74616b655f6f66665f74696d65202b20656e645f7265706f72745f696e74657276616c296f6e6c7920726567697374726564206f7261636c652063616e207265706f727420726573756c742173686f756c6420736574746c6520616674657220287363686564756c655f74616b655f6f66665f74696d65202b20656e645f646973707574655f696e74657276616c29a265627a7a72315820612ed13838b4a948337cce571ea68c9fc0e970d7cf3fb2271f2606a6dc6b4f4264736f6c634300050b00327265616c206d61782062656e656669742073686f756c6420657175616c20746f206d61782062656e6566697421696e73756666696369656e7420696e697469616c206465706f7369742066756e64217363686564756c652074616b65206f66662074696d652073686f756c64206174206c65617374206d61782062656e656669742073686f756c64206c6172676572207468616e207072656d69756d00000000000000000000000000000000000000000000021e19e0c9bab2400000323031392d31302d313700000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000005da9004400000000000000000000000079f3574d440c21029eecb8e9ee66b23b26cbcb38") + _constantinopleOpCodeContract, _ = hex.DecodeString("608060405234801561001057600080fd5b506104d5806100206000396000f3fe608060405234801561001057600080fd5b50600436106100885760003560e01c806381ea44081161005b57806381ea440814610101578063a91b336214610159578063c2bc2efc14610177578063f5eacece146101cf57610088565b80635bec9e671461008d57806360fe47b1146100975780636bc8ecaa146100c5578063744f5f83146100e3575b600080fd5b6100956101ed565b005b6100c3600480360360208110156100ad57600080fd5b8101908080359060200190929190505050610239565b005b6100cd610270565b6040518082815260200191505060405180910390f35b6100eb6102b3565b6040518082815260200191505060405180910390f35b6101436004803603602081101561011757600080fd5b81019080803573ffffffffffffffffffffffffffffffffffffffff1690602001909291905050506102f6565b6040518082815260200191505060405180910390f35b61016161036a565b6040518082815260200191505060405180910390f35b6101b96004803603602081101561018d57600080fd5b81019080803573ffffffffffffffffffffffffffffffffffffffff1690602001909291905050506103ad565b6040518082815260200191505060405180910390f35b6101d761045f565b6040518082815260200191505060405180910390f35b5b60011561020b5760008081548092919060010191905055506101ee565b7f8bfaa460932ccf8751604dd60efa3eafa220ec358fccb32ef703f91c509bc3ea60405160405180910390a1565b80600081905550807fdf7a95aebff315db1b7716215d602ab537373cdb769232aae6055c06e798425b60405160405180910390a250565b6000805460081d905080600081905550807fdf7a95aebff315db1b7716215d602ab537373cdb769232aae6055c06e798425b60405160405180910390a280905090565b6000805460081c905080600081905550807fdf7a95aebff315db1b7716215d602ab537373cdb769232aae6055c06e798425b60405160405180910390a280905090565b60008073ffffffffffffffffffffffffffffffffffffffff168273ffffffffffffffffffffffffffffffffffffffff16141561033157600080fd5b813f9050807fdf7a95aebff315db1b7716215d602ab537373cdb769232aae6055c06e798425b60405160405180910390a2809050919050565b6000805460081b905080600081905550807fdf7a95aebff315db1b7716215d602ab537373cdb769232aae6055c06e798425b60405160405180910390a280905090565b60008073ffffffffffffffffffffffffffffffffffffffff168273ffffffffffffffffffffffffffffffffffffffff1614156103e857600080fd5b7fbde7a70c2261170a87678200113c8e12f82f63d0a1d1cfa45681cbac328e87e382600054604051808373ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff1681526020018281526020019250505060405180910390a16000549050919050565b60008080602060406000f59150817fdf7a95aebff315db1b7716215d602ab537373cdb769232aae6055c06e798425b60405160405180910390a2819150509056fea265627a7a72305820209a8ef04c4d621759f34878b27b238650e8605c8a71d6efc619a769a64aa9cc64736f6c634300050a0032") + _selfdestructContract, _ = hex.DecodeString("608060405234801561000f575f80fd5b50335f806101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055506102f88061005c5f395ff3fe608060405260043610610037575f3560e01c80630c08bf881461004257806312065fe0146100585780638da5cb5b146100825761003e565b3661003e57005b5f80fd5b34801561004d575f80fd5b506100566100ac565b005b348015610063575f80fd5b5061006c610171565b60405161007991906101b3565b60405180910390f35b34801561008d575f80fd5b50610096610178565b6040516100a3919061020b565b60405180910390f35b5f8054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff163373ffffffffffffffffffffffffffffffffffffffff1614610139576040517f08c379a0000000000000000000000000000000000000000000000000000000008152600401610130906102a4565b60405180910390fd5b5f8054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16ff5b5f47905090565b5f8054906101000a900473ffffffffffffffffffffffffffffffffffffffff1681565b5f819050919050565b6101ad8161019b565b82525050565b5f6020820190506101c65f8301846101a4565b92915050565b5f73ffffffffffffffffffffffffffffffffffffffff82169050919050565b5f6101f5826101cc565b9050919050565b610205816101eb565b82525050565b5f60208201905061021e5f8301846101fc565b92915050565b5f82825260208201905092915050565b7f4f6e6c7920746865206f776e65722063616e2064657374726f792074686520635f8201527f6f6e747261637400000000000000000000000000000000000000000000000000602082015250565b5f61028e602783610224565b915061029982610234565b604082019050919050565b5f6020820190508181035f8301526102bb81610282565b905091905056fea2646970667358221220b78ecca519bda7ff2a88d8d2e964b5eaa98e01e80322cffe2fc97d17b9dcb6e464736f6c63430008180033") + _selfdestructOnCreationContract, _ = hex.DecodeString("6080604052335f806101000a81548173ffffffffffffffffffffffffffffffffffffffff021916908373ffffffffffffffffffffffffffffffffffffffff1602179055505f8054906101000a900473ffffffffffffffffffffffffffffffffffffffff1673ffffffffffffffffffffffffffffffffffffffff16fffe") ) From 6ac9264c5f26444d180ffd055e3c5c0b35f0b65c Mon Sep 17 00:00:00 2001 From: dustinxie Date: Tue, 10 Dec 2024 17:02:40 -0800 Subject: [PATCH 06/21] [factory] refactore workingsetstore (#4519) --- state/factory/workingsetstore.go | 211 ++--------------------- state/factory/workingsetstore_factory.go | 138 +++++++++++++++ state/factory/workingsetstore_statedb.go | 69 ++++++++ 3 files changed, 221 insertions(+), 197 deletions(-) create mode 100644 state/factory/workingsetstore_factory.go create mode 100644 state/factory/workingsetstore_statedb.go diff --git a/state/factory/workingsetstore.go b/state/factory/workingsetstore.go index cec2d453e5..a6445df079 100644 --- a/state/factory/workingsetstore.go +++ b/state/factory/workingsetstore.go @@ -1,4 +1,4 @@ -// Copyright (c) 2022 IoTeX Foundation +// Copyright (c) 2024 IoTeX Foundation // This source code is provided 'as is' and no warranties are given as to title or non-infringement, merchantability // or fitness for purpose and, to the extent permitted by law, all liability for your use of the code is disclaimed. // This source code is governed by Apache License 2.0 that can be found in the LICENSE file. @@ -6,20 +6,10 @@ package factory import ( - "context" - "fmt" - "github.com/iotexproject/go-pkgs/hash" - "go.uber.org/zap" - - "github.com/pkg/errors" "github.com/iotexproject/iotex-core/v2/action/protocol" "github.com/iotexproject/iotex-core/v2/db" - "github.com/iotexproject/iotex-core/v2/db/trie" - "github.com/iotexproject/iotex-core/v2/pkg/log" - "github.com/iotexproject/iotex-core/v2/pkg/util/byteutil" - "github.com/iotexproject/iotex-core/v2/state" ) type ( @@ -35,220 +25,47 @@ type ( ReadView(string) (interface{}, error) WriteView(string, interface{}) error } - stateDBWorkingSetStore struct { - view protocol.View - flusher db.KVStoreFlusher - readBuffer bool - } - factoryWorkingSetStore struct { - view protocol.View - flusher db.KVStoreFlusher - tlt trie.TwoLayerTrie - trieRoots map[int][]byte + workingSetStoreCommon struct { + view protocol.View + flusher db.KVStoreFlusher } ) -func newStateDBWorkingSetStore(view protocol.View, flusher db.KVStoreFlusher, readBuffer bool) workingSetStore { - return &stateDBWorkingSetStore{ - flusher: flusher, - view: view, - readBuffer: readBuffer, - } -} - -func newFactoryWorkingSetStore(view protocol.View, flusher db.KVStoreFlusher) (workingSetStore, error) { - tlt, err := newTwoLayerTrie(ArchiveTrieNamespace, flusher.KVStoreWithBuffer(), ArchiveTrieRootKey, true) - if err != nil { - return nil, err - } - - return &factoryWorkingSetStore{ - flusher: flusher, - view: view, - tlt: tlt, - trieRoots: make(map[int][]byte), - }, nil -} - -func newFactoryWorkingSetStoreAtHeight(view protocol.View, flusher db.KVStoreFlusher, height uint64) (workingSetStore, error) { - rootKey := fmt.Sprintf("%s-%d", ArchiveTrieRootKey, height) - tlt, err := newTwoLayerTrie(ArchiveTrieNamespace, flusher.KVStoreWithBuffer(), rootKey, false) - if err != nil { - return nil, err - } - - return &factoryWorkingSetStore{ - flusher: flusher, - view: view, - tlt: tlt, - trieRoots: make(map[int][]byte), - }, nil -} - -func (store *stateDBWorkingSetStore) Start(context.Context) error { - return nil -} - -func (store *stateDBWorkingSetStore) Stop(context.Context) error { - return nil -} - -func (store *stateDBWorkingSetStore) ReadView(name string) (interface{}, error) { +func (store *workingSetStoreCommon) ReadView(name string) (interface{}, error) { return store.view.Read(name) } -func (store *stateDBWorkingSetStore) WriteView(name string, value interface{}) error { +func (store *workingSetStoreCommon) WriteView(name string, value interface{}) error { return store.view.Write(name, value) } -func (store *stateDBWorkingSetStore) Get(ns string, key []byte) ([]byte, error) { - data, err := store.flusher.KVStoreWithBuffer().Get(ns, key) - if err != nil { - if errors.Cause(err) == db.ErrNotExist { - return nil, errors.Wrapf(state.ErrStateNotExist, "failed to get state of ns = %x and key = %x", ns, key) - } - return nil, err - } - return data, nil -} - -func (store *stateDBWorkingSetStore) Put(ns string, key []byte, value []byte) error { +func (store *workingSetStoreCommon) Put(ns string, key []byte, value []byte) error { store.flusher.KVStoreWithBuffer().MustPut(ns, key, value) return nil } -func (store *stateDBWorkingSetStore) Delete(ns string, key []byte) error { +func (store *workingSetStoreCommon) Delete(ns string, key []byte) error { store.flusher.KVStoreWithBuffer().MustDelete(ns, key) return nil } -func (store *stateDBWorkingSetStore) States(ns string, keys [][]byte) ([][]byte, [][]byte, error) { - if store.readBuffer { - return readStates(store.flusher.KVStoreWithBuffer(), ns, keys) - } - return readStates(store.flusher.BaseKVStore(), ns, keys) -} - -func (store *stateDBWorkingSetStore) Digest() hash.Hash256 { +func (store *workingSetStoreCommon) Digest() hash.Hash256 { return hash.Hash256b(store.flusher.SerializeQueue()) } -func (store *stateDBWorkingSetStore) Finalize(height uint64) error { - // Persist current chain Height - store.flusher.KVStoreWithBuffer().MustPut( - AccountKVNamespace, - []byte(CurrentHeightKey), - byteutil.Uint64ToBytes(height), - ) - return nil -} - -func (store *stateDBWorkingSetStore) Commit() error { +func (store *workingSetStoreCommon) Commit() error { + _dbBatchSizelMtc.WithLabelValues().Set(float64(store.flusher.KVStoreWithBuffer().Size())) return store.flusher.Flush() } -func (store *stateDBWorkingSetStore) Snapshot() int { +func (store *workingSetStoreCommon) Snapshot() int { return store.flusher.KVStoreWithBuffer().Snapshot() } -func (store *stateDBWorkingSetStore) RevertSnapshot(snapshot int) error { +func (store *workingSetStoreCommon) RevertSnapshot(snapshot int) error { return store.flusher.KVStoreWithBuffer().RevertSnapshot(snapshot) } -func (store *stateDBWorkingSetStore) ResetSnapshots() { - store.flusher.KVStoreWithBuffer().ResetSnapshots() -} - -func (store *factoryWorkingSetStore) Start(ctx context.Context) error { - return store.tlt.Start(ctx) -} - -func (store *factoryWorkingSetStore) Stop(ctx context.Context) error { - return store.tlt.Stop(ctx) -} - -func (store *factoryWorkingSetStore) ReadView(name string) (interface{}, error) { - return store.view.Read(name) -} - -func (store *factoryWorkingSetStore) WriteView(name string, value interface{}) error { - return store.view.Write(name, value) -} - -func (store *factoryWorkingSetStore) Get(ns string, key []byte) ([]byte, error) { - return readStateFromTLT(store.tlt, ns, key) -} - -func (store *factoryWorkingSetStore) Put(ns string, key []byte, value []byte) error { - store.flusher.KVStoreWithBuffer().MustPut(ns, key, value) - nsHash := hash.Hash160b([]byte(ns)) - - return store.tlt.Upsert(nsHash[:], toLegacyKey(key), value) -} - -func (store *factoryWorkingSetStore) Delete(ns string, key []byte) error { - store.flusher.KVStoreWithBuffer().MustDelete(ns, key) - nsHash := hash.Hash160b([]byte(ns)) - - err := store.tlt.Delete(nsHash[:], toLegacyKey(key)) - if errors.Cause(err) == trie.ErrNotExist { - return errors.Wrapf(state.ErrStateNotExist, "key %x doesn't exist in namespace %x", key, nsHash) - } - return err -} - -func (store *factoryWorkingSetStore) States(ns string, keys [][]byte) ([][]byte, [][]byte, error) { - return readStatesFromTLT(store.tlt, ns, keys) -} - -func (store *factoryWorkingSetStore) Digest() hash.Hash256 { - return hash.Hash256b(store.flusher.SerializeQueue()) -} - -func (store *factoryWorkingSetStore) Finalize(h uint64) error { - rootHash, err := store.tlt.RootHash() - if err != nil { - return err - } - store.flusher.KVStoreWithBuffer().MustPut(AccountKVNamespace, []byte(CurrentHeightKey), byteutil.Uint64ToBytes(h)) - store.flusher.KVStoreWithBuffer().MustPut(ArchiveTrieNamespace, []byte(ArchiveTrieRootKey), rootHash) - // Persist the historical accountTrie's root hash - store.flusher.KVStoreWithBuffer().MustPut( - ArchiveTrieNamespace, - []byte(fmt.Sprintf("%s-%d", ArchiveTrieRootKey, h)), - rootHash, - ) - return nil -} - -func (store *factoryWorkingSetStore) Commit() error { - _dbBatchSizelMtc.WithLabelValues().Set(float64(store.flusher.KVStoreWithBuffer().Size())) - return store.flusher.Flush() -} - -func (store *factoryWorkingSetStore) Snapshot() int { - rh, err := store.tlt.RootHash() - if err != nil { - log.L().Panic("failed to do snapshot", zap.Error(err)) - } - s := store.flusher.KVStoreWithBuffer().Snapshot() - store.trieRoots[s] = rh - return s -} - -func (store *factoryWorkingSetStore) RevertSnapshot(snapshot int) error { - if err := store.flusher.KVStoreWithBuffer().RevertSnapshot(snapshot); err != nil { - return err - } - root, ok := store.trieRoots[snapshot] - if !ok { - // this should not happen, b/c we save the trie root on a successful return of Snapshot(), but check anyway - return errors.Wrapf(trie.ErrInvalidTrie, "failed to get trie root for snapshot = %d", snapshot) - } - return store.tlt.SetRootHash(root[:]) -} - -func (store *factoryWorkingSetStore) ResetSnapshots() { +func (store *workingSetStoreCommon) ResetSnapshots() { store.flusher.KVStoreWithBuffer().ResetSnapshots() - store.trieRoots = make(map[int][]byte) } diff --git a/state/factory/workingsetstore_factory.go b/state/factory/workingsetstore_factory.go new file mode 100644 index 0000000000..bd259f5861 --- /dev/null +++ b/state/factory/workingsetstore_factory.go @@ -0,0 +1,138 @@ +// Copyright (c) 2024 IoTeX Foundation +// This source code is provided 'as is' and no warranties are given as to title or non-infringement, merchantability +// or fitness for purpose and, to the extent permitted by law, all liability for your use of the code is disclaimed. +// This source code is governed by Apache License 2.0 that can be found in the LICENSE file. + +package factory + +import ( + "context" + "fmt" + + "github.com/iotexproject/go-pkgs/hash" + "github.com/pkg/errors" + "go.uber.org/zap" + + "github.com/iotexproject/iotex-core/v2/action/protocol" + "github.com/iotexproject/iotex-core/v2/db" + "github.com/iotexproject/iotex-core/v2/db/trie" + "github.com/iotexproject/iotex-core/v2/pkg/log" + "github.com/iotexproject/iotex-core/v2/pkg/util/byteutil" + "github.com/iotexproject/iotex-core/v2/state" +) + +type factoryWorkingSetStore struct { + *workingSetStoreCommon + tlt trie.TwoLayerTrie + trieRoots map[int][]byte +} + +func newFactoryWorkingSetStore(view protocol.View, flusher db.KVStoreFlusher) (workingSetStore, error) { + tlt, err := newTwoLayerTrie(ArchiveTrieNamespace, flusher.KVStoreWithBuffer(), ArchiveTrieRootKey, true) + if err != nil { + return nil, err + } + + return &factoryWorkingSetStore{ + workingSetStoreCommon: &workingSetStoreCommon{ + flusher: flusher, + view: view, + }, + tlt: tlt, + trieRoots: make(map[int][]byte), + }, nil +} + +func newFactoryWorkingSetStoreAtHeight(view protocol.View, flusher db.KVStoreFlusher, height uint64) (workingSetStore, error) { + rootKey := fmt.Sprintf("%s-%d", ArchiveTrieRootKey, height) + tlt, err := newTwoLayerTrie(ArchiveTrieNamespace, flusher.KVStoreWithBuffer(), rootKey, false) + if err != nil { + return nil, err + } + + return &factoryWorkingSetStore{ + workingSetStoreCommon: &workingSetStoreCommon{ + flusher: flusher, + view: view, + }, + tlt: tlt, + trieRoots: make(map[int][]byte), + }, nil +} + +func (store *factoryWorkingSetStore) Start(ctx context.Context) error { + return store.tlt.Start(ctx) +} + +func (store *factoryWorkingSetStore) Stop(ctx context.Context) error { + return store.tlt.Stop(ctx) +} + +func (store *factoryWorkingSetStore) Get(ns string, key []byte) ([]byte, error) { + return readStateFromTLT(store.tlt, ns, key) +} + +func (store *factoryWorkingSetStore) Put(ns string, key []byte, value []byte) error { + store.workingSetStoreCommon.Put(ns, key, value) + nsHash := hash.Hash160b([]byte(ns)) + + return store.tlt.Upsert(nsHash[:], toLegacyKey(key), value) +} + +func (store *factoryWorkingSetStore) Delete(ns string, key []byte) error { + store.workingSetStoreCommon.Delete(ns, key) + nsHash := hash.Hash160b([]byte(ns)) + + err := store.tlt.Delete(nsHash[:], toLegacyKey(key)) + if errors.Cause(err) == trie.ErrNotExist { + return errors.Wrapf(state.ErrStateNotExist, "key %x doesn't exist in namespace %x", key, nsHash) + } + return err +} + +func (store *factoryWorkingSetStore) States(ns string, keys [][]byte) ([][]byte, [][]byte, error) { + return readStatesFromTLT(store.tlt, ns, keys) +} + +func (store *factoryWorkingSetStore) Finalize(h uint64) error { + rootHash, err := store.tlt.RootHash() + if err != nil { + return err + } + store.flusher.KVStoreWithBuffer().MustPut(AccountKVNamespace, []byte(CurrentHeightKey), byteutil.Uint64ToBytes(h)) + store.flusher.KVStoreWithBuffer().MustPut(ArchiveTrieNamespace, []byte(ArchiveTrieRootKey), rootHash) + // Persist the historical accountTrie's root hash + store.flusher.KVStoreWithBuffer().MustPut( + ArchiveTrieNamespace, + []byte(fmt.Sprintf("%s-%d", ArchiveTrieRootKey, h)), + rootHash, + ) + return nil +} + +func (store *factoryWorkingSetStore) Snapshot() int { + rh, err := store.tlt.RootHash() + if err != nil { + log.L().Panic("failed to do snapshot", zap.Error(err)) + } + s := store.workingSetStoreCommon.Snapshot() + store.trieRoots[s] = rh + return s +} + +func (store *factoryWorkingSetStore) RevertSnapshot(snapshot int) error { + if err := store.workingSetStoreCommon.RevertSnapshot(snapshot); err != nil { + return err + } + root, ok := store.trieRoots[snapshot] + if !ok { + // this should not happen, b/c we save the trie root on a successful return of Snapshot(), but check anyway + return errors.Wrapf(trie.ErrInvalidTrie, "failed to get trie root for snapshot = %d", snapshot) + } + return store.tlt.SetRootHash(root[:]) +} + +func (store *factoryWorkingSetStore) ResetSnapshots() { + store.workingSetStoreCommon.ResetSnapshots() + store.trieRoots = make(map[int][]byte) +} diff --git a/state/factory/workingsetstore_statedb.go b/state/factory/workingsetstore_statedb.go new file mode 100644 index 0000000000..4f96237204 --- /dev/null +++ b/state/factory/workingsetstore_statedb.go @@ -0,0 +1,69 @@ +// Copyright (c) 2024 IoTeX Foundation +// This source code is provided 'as is' and no warranties are given as to title or non-infringement, merchantability +// or fitness for purpose and, to the extent permitted by law, all liability for your use of the code is disclaimed. +// This source code is governed by Apache License 2.0 that can be found in the LICENSE file. + +package factory + +import ( + "context" + + "github.com/pkg/errors" + + "github.com/iotexproject/iotex-core/v2/action/protocol" + "github.com/iotexproject/iotex-core/v2/db" + "github.com/iotexproject/iotex-core/v2/pkg/util/byteutil" + "github.com/iotexproject/iotex-core/v2/state" +) + +type stateDBWorkingSetStore struct { + *workingSetStoreCommon + readBuffer bool +} + +func newStateDBWorkingSetStore(view protocol.View, flusher db.KVStoreFlusher, readBuffer bool) workingSetStore { + return &stateDBWorkingSetStore{ + workingSetStoreCommon: &workingSetStoreCommon{ + flusher: flusher, + view: view, + }, + readBuffer: readBuffer, + } +} + +func (store *stateDBWorkingSetStore) Start(context.Context) error { + return nil +} + +func (store *stateDBWorkingSetStore) Stop(context.Context) error { + return nil +} + +func (store *stateDBWorkingSetStore) Get(ns string, key []byte) ([]byte, error) { + data, err := store.flusher.KVStoreWithBuffer().Get(ns, key) + if err != nil { + if errors.Cause(err) == db.ErrNotExist { + return nil, errors.Wrapf(state.ErrStateNotExist, "failed to get state of ns = %x and key = %x", ns, key) + } + return nil, err + } + return data, nil +} + +func (store *stateDBWorkingSetStore) States(ns string, keys [][]byte) ([][]byte, [][]byte, error) { + if store.readBuffer { + // TODO: after the 180 HF, we can revert readBuffer, and always go this case + return readStates(store.flusher.KVStoreWithBuffer(), ns, keys) + } + return readStates(store.flusher.BaseKVStore(), ns, keys) +} + +func (store *stateDBWorkingSetStore) Finalize(height uint64) error { + // Persist current chain Height + store.flusher.KVStoreWithBuffer().MustPut( + AccountKVNamespace, + []byte(CurrentHeightKey), + byteutil.Uint64ToBytes(height), + ) + return nil +} From 55e5e1ccbe916a3d891973de2e521104df099426 Mon Sep 17 00:00:00 2001 From: dustinxie Date: Sun, 15 Dec 2024 19:14:48 -0800 Subject: [PATCH 07/21] [factory] add daoRetrofitter interface (#4522) --- state/factory/daoretrofitter.go | 49 ++++++++++++++++++++++ state/factory/statedb.go | 74 +++++++++++++++------------------ 2 files changed, 83 insertions(+), 40 deletions(-) create mode 100644 state/factory/daoretrofitter.go diff --git a/state/factory/daoretrofitter.go b/state/factory/daoretrofitter.go new file mode 100644 index 0000000000..7170fcabed --- /dev/null +++ b/state/factory/daoretrofitter.go @@ -0,0 +1,49 @@ +// Copyright (c) 2024 IoTeX Foundation +// This source code is provided 'as is' and no warranties are given as to title or non-infringement, merchantability +// or fitness for purpose and, to the extent permitted by law, all liability for your use of the code is disclaimed. +// This source code is governed by Apache License 2.0 that can be found in the LICENSE file. + +package factory + +import ( + "context" + + "github.com/pkg/errors" + + "github.com/iotexproject/iotex-core/v2/db" + "github.com/iotexproject/iotex-core/v2/pkg/util/byteutil" +) + +type daoRTF struct { + dao db.KVStore +} + +func newDaoRetrofitter(dao db.KVStore) *daoRTF { + return &daoRTF{ + dao: dao, + } +} + +func (rtf *daoRTF) Start(ctx context.Context) error { + return rtf.dao.Start(ctx) +} + +func (rtf *daoRTF) Stop(ctx context.Context) error { + return rtf.dao.Stop(ctx) +} + +func (rtf *daoRTF) atHeight(uint64) db.KVStore { + return rtf.dao +} + +func (rtf *daoRTF) getHeight() (uint64, error) { + height, err := rtf.dao.Get(AccountKVNamespace, []byte(CurrentHeightKey)) + if err != nil { + return 0, errors.Wrap(err, "failed to get factory's height from underlying DB") + } + return byteutil.BytesToUint64(height), nil +} + +func (rtf *daoRTF) putHeight(h uint64) error { + return rtf.dao.Put(AccountKVNamespace, []byte(CurrentHeightKey), byteutil.Uint64ToBytes(h)) +} diff --git a/state/factory/statedb.go b/state/factory/statedb.go index cf5fc01f84..2e2be991aa 100644 --- a/state/factory/statedb.go +++ b/state/factory/statedb.go @@ -1,4 +1,4 @@ -// Copyright (c) 2020 IoTeX Foundation +// Copyright (c) 2024 IoTeX Foundation // This source code is provided 'as is' and no warranties are given as to title or non-infringement, merchantability // or fitness for purpose and, to the extent permitted by law, all liability for your use of the code is disclaimed. // This source code is governed by Apache License 2.0 that can be found in the LICENSE file. @@ -27,25 +27,34 @@ import ( "github.com/iotexproject/iotex-core/v2/blockchain/genesis" "github.com/iotexproject/iotex-core/v2/db" "github.com/iotexproject/iotex-core/v2/db/batch" + "github.com/iotexproject/iotex-core/v2/pkg/lifecycle" "github.com/iotexproject/iotex-core/v2/pkg/log" "github.com/iotexproject/iotex-core/v2/pkg/prometheustimer" - "github.com/iotexproject/iotex-core/v2/pkg/util/byteutil" "github.com/iotexproject/iotex-core/v2/state" ) -// stateDB implements StateFactory interface, tracks changes to account/contract and batch-commits to DB -type stateDB struct { - mutex sync.RWMutex - currentChainHeight uint64 - cfg Config - registry *protocol.Registry - dao db.KVStore // the underlying DB for account/contract storage - timerFactory *prometheustimer.TimerFactory - workingsets cache.LRUCache // lru cache for workingsets - protocolView protocol.View - skipBlockValidationOnPut bool - ps *patchStore -} +type ( + // daoRetrofitter represents the DAO-related methods to accommodate archive-mode + daoRetrofitter interface { + lifecycle.StartStopper + atHeight(uint64) db.KVStore + getHeight() (uint64, error) + putHeight(uint64) error + } + // stateDB implements StateFactory interface, tracks changes to account/contract and batch-commits to DB + stateDB struct { + mutex sync.RWMutex + currentChainHeight uint64 + cfg Config + registry *protocol.Registry + dao daoRetrofitter + timerFactory *prometheustimer.TimerFactory + workingsets cache.LRUCache // lru cache for workingsets + protocolView protocol.View + skipBlockValidationOnPut bool + ps *patchStore + } +) // StateDBOption sets stateDB construction parameter type StateDBOption func(*stateDB, *Config) error @@ -90,7 +99,6 @@ func NewStateDB(cfg Config, dao db.KVStore, opts ...StateDBOption) (Factory, err registry: protocol.NewRegistry(), protocolView: protocol.View{}, workingsets: cache.NewThreadSafeLruCache(int(cfg.Chain.WorkingSetCacheSize)), - dao: dao, } for _, opt := range opts { if err := opt(&sdb, &cfg); err != nil { @@ -98,6 +106,7 @@ func NewStateDB(cfg Config, dao db.KVStore, opts ...StateDBOption) (Factory, err return nil, err } } + sdb.dao = newDaoRetrofitter(dao) timerFactory, err := prometheustimer.New( "iotex_statefactory_perf", "Performance of state factory module", @@ -117,17 +126,17 @@ func (sdb *stateDB) Start(ctx context.Context) error { return err } // check factory height - h, err := sdb.dao.Get(AccountKVNamespace, []byte(CurrentHeightKey)) + h, err := sdb.dao.getHeight() switch errors.Cause(err) { case nil: - sdb.currentChainHeight = byteutil.BytesToUint64(h) + sdb.currentChainHeight = h // start all protocols if sdb.protocolView, err = sdb.registry.StartAll(ctx, sdb); err != nil { return err } case db.ErrNotExist: sdb.currentChainHeight = 0 - if err = sdb.dao.Put(AccountKVNamespace, []byte(CurrentHeightKey), byteutil.Uint64ToBytes(0)); err != nil { + if err = sdb.dao.putHeight(0); err != nil { return errors.Wrap(err, "failed to init statedb's height") } // start all protocols @@ -150,7 +159,6 @@ func (sdb *stateDB) Start(ctx context.Context) error { default: return err } - return nil } @@ -165,17 +173,13 @@ func (sdb *stateDB) Stop(ctx context.Context) error { func (sdb *stateDB) Height() (uint64, error) { sdb.mutex.RLock() defer sdb.mutex.RUnlock() - height, err := sdb.dao.Get(AccountKVNamespace, []byte(CurrentHeightKey)) - if err != nil { - return 0, errors.Wrap(err, "failed to get factory's height from underlying DB") - } - return byteutil.BytesToUint64(height), nil + return sdb.dao.getHeight() } func (sdb *stateDB) newWorkingSet(ctx context.Context, height uint64) (*workingSet, error) { g := genesis.MustExtractGenesisContext(ctx) flusher, err := db.NewKVStoreFlusher( - sdb.dao, + sdb.dao.atHeight(height), batch.NewCachedBatch(), sdb.flusherOptions(!g.IsEaster(height))..., ) @@ -332,7 +336,7 @@ func (sdb *stateDB) State(s interface{}, opts ...protocol.StateOption) (uint64, if cfg.Keys != nil { return 0, errors.Wrap(ErrNotSupported, "Read state with keys option has not been implemented yet") } - return sdb.currentChainHeight, sdb.state(cfg.Namespace, cfg.Key, s) + return sdb.currentChainHeight, sdb.state(sdb.currentChainHeight, cfg.Namespace, cfg.Key, s) } // State returns a set of states in the state factory @@ -346,7 +350,7 @@ func (sdb *stateDB) States(opts ...protocol.StateOption) (uint64, state.Iterator if cfg.Key != nil { return sdb.currentChainHeight, nil, errors.Wrap(ErrNotSupported, "Read states with key option has not been implemented yet") } - keys, values, err := readStates(sdb.dao, cfg.Namespace, cfg.Keys) + keys, values, err := readStates(sdb.dao.atHeight(sdb.currentChainHeight), cfg.Namespace, cfg.Keys) if err != nil { return 0, nil, err } @@ -358,16 +362,6 @@ func (sdb *stateDB) States(opts ...protocol.StateOption) (uint64, state.Iterator return sdb.currentChainHeight, iter, nil } -// StateAtHeight returns a confirmed state at height -- archive mode -func (sdb *stateDB) StateAtHeight(height uint64, s interface{}, opts ...protocol.StateOption) error { - return ErrNotSupported -} - -// StatesAtHeight returns a set states in the state factory at height -- archive mode -func (sdb *stateDB) StatesAtHeight(height uint64, opts ...protocol.StateOption) (state.Iterator, error) { - return nil, errors.Wrap(ErrNotSupported, "state db does not support archive mode") -} - // ReadView reads the view func (sdb *stateDB) ReadView(name string) (interface{}, error) { return sdb.protocolView.Read(name) @@ -397,8 +391,8 @@ func (sdb *stateDB) flusherOptions(preEaster bool) []db.KVStoreFlusherOption { ) } -func (sdb *stateDB) state(ns string, addr []byte, s interface{}) error { - data, err := sdb.dao.Get(ns, addr) +func (sdb *stateDB) state(h uint64, ns string, addr []byte, s interface{}) error { + data, err := sdb.dao.atHeight(h).Get(ns, addr) if err != nil { if errors.Cause(err) == db.ErrNotExist { return errors.Wrapf(state.ErrStateNotExist, "state of %x doesn't exist", addr) From caa1c8ce54a5a5bb21b3e4576ad75c02a90d3bf9 Mon Sep 17 00:00:00 2001 From: Chen Chen Date: Tue, 17 Dec 2024 08:40:43 +0800 Subject: [PATCH 08/21] Fix panic if account error (#4521) Co-authored-by: CoderZhi --- api/web3server_utils.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/api/web3server_utils.go b/api/web3server_utils.go index 54f94dcd41..6540ca1ee1 100644 --- a/api/web3server_utils.go +++ b/api/web3server_utils.go @@ -181,7 +181,10 @@ func (svr *web3Handler) checkContractAddr(to string) (bool, error) { return false, err } accountMeta, _, err := svr.coreService.Account(ioAddr) - return accountMeta.IsContract, err + if err != nil { + return false, err + } + return accountMeta.IsContract, nil } func (svr *web3Handler) getLogsWithFilter(from uint64, to uint64, addrs []string, topics [][]string) ([]*getLogsResult, error) { From a57eee38d41614b48d39566aac25ef58ec0ac1fc Mon Sep 17 00:00:00 2001 From: dustinxie Date: Mon, 16 Dec 2024 21:57:59 -0800 Subject: [PATCH 09/21] [blockchain] add ContextAtHeight() (#4524) --- blockchain/blockchain.go | 61 ++++++++++++-------- test/mock/mock_blockchain/mock_blockchain.go | 15 +++++ 2 files changed, 53 insertions(+), 23 deletions(-) diff --git a/blockchain/blockchain.go b/blockchain/blockchain.go index 80d509ca25..efc32b84b9 100644 --- a/blockchain/blockchain.go +++ b/blockchain/blockchain.go @@ -87,6 +87,8 @@ type ( Genesis() genesis.Genesis // Context returns current context Context(context.Context) (context.Context, error) + // ContextAtHeight returns context at given height + ContextAtHeight(context.Context, uint64) (context.Context, error) // For block operations // MintNewBlock creates a new block with given actions @@ -228,10 +230,14 @@ func (bc *blockchain) Start(ctx context.Context) error { defer bc.mu.Unlock() // pass registry to be used by state factory's initialization - ctx, err := bc.context(ctx, false) - if err != nil { - return err - } + ctx = protocol.WithFeatureWithHeightCtx(genesis.WithGenesisContext( + protocol.WithBlockchainCtx( + ctx, + protocol.BlockchainCtx{ + ChainID: bc.ChainID(), + EvmNetworkID: bc.EvmNetworkID(), + }, + ), bc.genesis)) return bc.lifecycle.OnStart(ctx) } @@ -281,7 +287,11 @@ func (bc *blockchain) ValidateBlock(blk *block.Block, opts ...BlockValidationOpt if blk == nil { return ErrInvalidBlock } - tip, err := bc.tipInfo() + tipHeight, err := bc.dao.Height() + if err != nil { + return err + } + tip, err := bc.tipInfo(tipHeight) if err != nil { return err } @@ -322,7 +332,7 @@ func (bc *blockchain) ValidateBlock(blk *block.Block, opts ...BlockValidationOpt if producerAddr == nil { return errors.New("failed to get address") } - ctx, err := bc.context(context.Background(), true) + ctx, err := bc.context(context.Background(), tipHeight) if err != nil { return err } @@ -351,8 +361,17 @@ func (bc *blockchain) ValidateBlock(blk *block.Block, opts ...BlockValidationOpt func (bc *blockchain) Context(ctx context.Context) (context.Context, error) { bc.mu.RLock() defer bc.mu.RUnlock() + tipHeight, err := bc.dao.Height() + if err != nil { + return nil, err + } + return bc.context(ctx, tipHeight) +} - return bc.context(ctx, true) +func (bc *blockchain) ContextAtHeight(ctx context.Context, height uint64) (context.Context, error) { + bc.mu.RLock() + defer bc.mu.RUnlock() + return bc.context(ctx, height) } func (bc *blockchain) contextWithBlock(ctx context.Context, producer address.Address, height uint64, timestamp time.Time, baseFee *big.Int, blobgas uint64) context.Context { @@ -368,21 +387,17 @@ func (bc *blockchain) contextWithBlock(ctx context.Context, producer address.Add }) } -func (bc *blockchain) context(ctx context.Context, tipInfoFlag bool) (context.Context, error) { - var tip protocol.TipInfo - if tipInfoFlag { - if tipInfoValue, err := bc.tipInfo(); err == nil { - tip = *tipInfoValue - } else { - return nil, err - } +func (bc *blockchain) context(ctx context.Context, height uint64) (context.Context, error) { + tip, err := bc.tipInfo(height) + if err != nil { + return nil, err } ctx = genesis.WithGenesisContext( protocol.WithBlockchainCtx( ctx, protocol.BlockchainCtx{ - Tip: tip, + Tip: *tip, ChainID: bc.ChainID(), EvmNetworkID: bc.EvmNetworkID(), }, @@ -402,7 +417,7 @@ func (bc *blockchain) MintNewBlock(timestamp time.Time) (*block.Block, error) { return nil, err } newblockHeight := tipHeight + 1 - ctx, err := bc.context(context.Background(), true) + ctx, err := bc.context(context.Background(), tipHeight) if err != nil { return nil, err } @@ -463,11 +478,7 @@ func (bc *blockchain) Genesis() genesis.Genesis { // private functions //===================================== -func (bc *blockchain) tipInfo() (*protocol.TipInfo, error) { - tipHeight, err := bc.dao.Height() - if err != nil { - return nil, err - } +func (bc *blockchain) tipInfo(tipHeight uint64) (*protocol.TipInfo, error) { if tipHeight == 0 { return &protocol.TipInfo{ Height: 0, @@ -493,7 +504,11 @@ func (bc *blockchain) tipInfo() (*protocol.TipInfo, error) { // commitBlock commits a block to the chain func (bc *blockchain) commitBlock(blk *block.Block) error { - ctx, err := bc.context(context.Background(), true) + tipHeight, err := bc.dao.Height() + if err != nil { + return err + } + ctx, err := bc.context(context.Background(), tipHeight) if err != nil { return err } diff --git a/test/mock/mock_blockchain/mock_blockchain.go b/test/mock/mock_blockchain/mock_blockchain.go index f1733e1671..42b5a75e15 100644 --- a/test/mock/mock_blockchain/mock_blockchain.go +++ b/test/mock/mock_blockchain/mock_blockchain.go @@ -141,6 +141,21 @@ func (mr *MockBlockchainMockRecorder) Context(arg0 interface{}) *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Context", reflect.TypeOf((*MockBlockchain)(nil).Context), arg0) } +// ContextAtHeight mocks base method. +func (m *MockBlockchain) ContextAtHeight(arg0 context.Context, arg1 uint64) (context.Context, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ContextAtHeight", arg0, arg1) + ret0, _ := ret[0].(context.Context) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ContextAtHeight indicates an expected call of ContextAtHeight. +func (mr *MockBlockchainMockRecorder) ContextAtHeight(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ContextAtHeight", reflect.TypeOf((*MockBlockchain)(nil).ContextAtHeight), arg0, arg1) +} + // EvmNetworkID mocks base method. func (m *MockBlockchain) EvmNetworkID() uint32 { m.ctrl.T.Helper() From 9bbb80e6a444dcca8faf27803d72ef2c4bd1e0d2 Mon Sep 17 00:00:00 2001 From: Chen Chen Date: Thu, 19 Dec 2024 14:05:51 +0800 Subject: [PATCH 10/21] [api] recovery api panic (#4523) --- .../execution/evm/evmstatedbadapter.go | 8 ++--- api/http.go | 8 +++-- api/web3server.go | 14 ++++++-- api/websocket.go | 34 +++++++++++-------- 4 files changed, 41 insertions(+), 23 deletions(-) diff --git a/action/protocol/execution/evm/evmstatedbadapter.go b/action/protocol/execution/evm/evmstatedbadapter.go index 7c38333ae4..3fbd1bd461 100644 --- a/action/protocol/execution/evm/evmstatedbadapter.go +++ b/action/protocol/execution/evm/evmstatedbadapter.go @@ -410,7 +410,7 @@ func (stateDB *StateDBAdapter) GetNonce(evmAddr common.Address) uint64 { } if stateDB.useConfirmedNonce { if pendingNonce == 0 { - panic("invalid pending nonce") + log.T(stateDB.ctx).Panic("invalid pending nonce") } pendingNonce-- } @@ -433,7 +433,7 @@ func (stateDB *StateDBAdapter) SetNonce(evmAddr common.Address, nonce uint64) { } if !stateDB.useConfirmedNonce { if nonce == 0 { - panic("invalid nonce zero") + log.T(stateDB.ctx).Panic("invalid nonce zero") } nonce-- } @@ -455,7 +455,7 @@ func (stateDB *StateDBAdapter) SubRefund(gas uint64) { log.T(stateDB.ctx).Debug("Called SubRefund.", zap.Uint64("gas", gas)) // stateDB.journal.append(refundChange{prev: self.refund}) if gas > stateDB.refund { - panic("Refund counter not enough!") + log.T(stateDB.ctx).Panic("Refund counter not enough!") } stateDB.refund -= gas } @@ -842,7 +842,7 @@ func (stateDB *StateDBAdapter) AddLog(evmLog *types.Log) { } if topics[0] == _inContractTransfer { if len(topics) != 3 { - panic("Invalid in contract transfer topics") + log.T(stateDB.ctx).Panic("Invalid in contract transfer topics") } if amount, zero := new(big.Int).SetBytes(evmLog.Data), big.NewInt(0); amount.Cmp(zero) == 1 { from, _ := address.FromBytes(topics[1][12:]) diff --git a/api/http.go b/api/http.go index 68858201fd..b2950051a2 100644 --- a/api/http.go +++ b/api/http.go @@ -11,6 +11,7 @@ import ( apitypes "github.com/iotexproject/iotex-core/v2/api/types" "github.com/iotexproject/iotex-core/v2/pkg/log" + "github.com/iotexproject/iotex-core/v2/pkg/tracer" "github.com/iotexproject/iotex-core/v2/pkg/util/httputil" ) @@ -68,7 +69,9 @@ func (handler *hTTPHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) return } - if err := handler.msgHandler.HandlePOSTReq(req.Context(), req.Body, + ctx, span := tracer.NewSpan(req.Context(), "http") + defer span.End() + if err := handler.msgHandler.HandlePOSTReq(ctx, req.Body, apitypes.NewResponseWriter( func(resp interface{}) (int, error) { w.Header().Set("Access-Control-Allow-Origin", "*") @@ -80,6 +83,7 @@ func (handler *hTTPHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) return w.Write(raw) }), ); err != nil { - log.Logger("api").Warn("fail to respond request.", zap.Error(err)) + w.WriteHeader(http.StatusInternalServerError) + log.T(ctx).Error("fail to respond request.", zap.Error(err)) } } diff --git a/api/web3server.go b/api/web3server.go index b9b65bf7f4..7f01078b8b 100644 --- a/api/web3server.go +++ b/api/web3server.go @@ -83,6 +83,7 @@ var ( errUnsupportedAction = errors.New("the type of action is not supported") errMsgBatchTooLarge = errors.New("batch too large") errHTTPNotSupported = errors.New("http not supported") + errPanic = errors.New("panic") _pendingBlockNumber = "pending" _latestBlockNumber = "latest" @@ -104,7 +105,7 @@ func NewWeb3Handler(core CoreService, cacheURL string, batchRequestLimit int) We } // HandlePOSTReq handles web3 request -func (svr *web3Handler) HandlePOSTReq(ctx context.Context, reader io.Reader, writer apitypes.Web3ResponseWriter) error { +func (svr *web3Handler) HandlePOSTReq(ctx context.Context, reader io.Reader, writer apitypes.Web3ResponseWriter) (err error) { ctx, span := tracer.NewSpan(ctx, "svr.HandlePOSTReq") defer span.End() web3Reqs, err := parseWeb3Reqs(reader) @@ -114,6 +115,15 @@ func (svr *web3Handler) HandlePOSTReq(ctx context.Context, reader io.Reader, wri _, err = writer.Write(&web3Response{err: err}) return err } + defer func() { + if r := recover(); r != nil { + err = errors.Wrapf(errPanic, "recovered from panic: %v, request params: %+v", r, web3Reqs) + return + } + if err != nil { + err = errors.Wrapf(err, "failed to handle web3 requests: %+v", web3Reqs) + } + }() if !web3Reqs.IsArray() { return svr.handleWeb3Req(ctx, &web3Reqs, writer) } @@ -147,7 +157,7 @@ func (svr *web3Handler) handleWeb3Req(ctx context.Context, web3Req *gjson.Result ) defer func(start time.Time) { svr.coreService.Track(ctx, start, method.(string), int64(size), err == nil) }(time.Now()) - log.T(ctx).Info("handleWeb3Req", zap.String("method", method.(string)), zap.String("requestParams", fmt.Sprintf("%+v", web3Req))) + log.T(ctx).Debug("handleWeb3Req", zap.String("method", method.(string)), zap.String("requestParams", fmt.Sprintf("%+v", web3Req))) _web3ServerMtc.WithLabelValues(method.(string)).Inc() _web3ServerMtc.WithLabelValues("requests_total").Inc() switch method { diff --git a/api/websocket.go b/api/websocket.go index ecc9eaa117..d160f64111 100644 --- a/api/websocket.go +++ b/api/websocket.go @@ -13,6 +13,7 @@ import ( apitypes "github.com/iotexproject/iotex-core/v2/api/types" "github.com/iotexproject/iotex-core/v2/pkg/log" + "github.com/iotexproject/iotex-core/v2/pkg/tracer" ) const ( @@ -141,21 +142,24 @@ func (wsSvr *WebsocketHandler) handleConnection(ctx context.Context, ws *websock cancel() return } - - err = wsSvr.msgHandler.HandlePOSTReq(ctx, reader, - apitypes.NewResponseWriter( - func(resp interface{}) (int, error) { - if err = safeWs.SetWriteDeadline(time.Now().Add(writeWait)); err != nil { - log.Logger("api").Warn("failed to set write deadline timeout.", zap.Error(err)) - } - return 0, safeWs.WriteJSON(resp) - }), - ) - if err != nil { - log.Logger("api").Warn("fail to respond request.", zap.Error(err)) - cancel() - return - } + func() { + wsCtx, span := tracer.NewSpan(ctx, "wss") + defer span.End() + err = wsSvr.msgHandler.HandlePOSTReq(wsCtx, reader, + apitypes.NewResponseWriter( + func(resp interface{}) (int, error) { + if err = safeWs.SetWriteDeadline(time.Now().Add(writeWait)); err != nil { + log.T(wsCtx).Warn("failed to set write deadline timeout.", zap.Error(err)) + } + return 0, safeWs.WriteJSON(resp) + }), + ) + if err != nil { + log.T(wsCtx).Warn("fail to respond request.", zap.Error(err)) + cancel() + return + } + }() } } } From 73b4526995c48248a207e062c25ffd7c95efa10e Mon Sep 17 00:00:00 2001 From: Chen Chen Date: Fri, 20 Dec 2024 06:02:58 +0800 Subject: [PATCH 11/21] [api] return data if estimategas reverted (#4459) Co-authored-by: dustinxie --- api/coreservice.go | 55 ++++++++++--------- api/coreservice_test.go | 37 +++++++------ api/grpcserver.go | 4 +- api/grpcserver_test.go | 2 +- api/web3server.go | 13 +++-- api/web3server_test.go | 2 +- .../mock_apicoreservice.go | 14 +++-- 7 files changed, 69 insertions(+), 58 deletions(-) diff --git a/api/coreservice.go b/api/coreservice.go index a65f126572..f18b261f23 100644 --- a/api/coreservice.go +++ b/api/coreservice.go @@ -138,11 +138,11 @@ type ( // UnconfirmedActionsByAddress returns all unconfirmed actions in actpool associated with an address UnconfirmedActionsByAddress(address string, start uint64, count uint64) ([]*iotexapi.ActionInfo, error) // EstimateMigrateStakeGasConsumption estimates gas for migrate stake - EstimateMigrateStakeGasConsumption(context.Context, *action.MigrateStake, address.Address) (uint64, error) + EstimateMigrateStakeGasConsumption(context.Context, *action.MigrateStake, address.Address) (uint64, []byte, error) // EstimateGasForNonExecution estimates action gas except execution EstimateGasForNonExecution(action.Action) (uint64, error) // EstimateExecutionGasConsumption estimate gas consumption for execution action - EstimateExecutionGasConsumption(ctx context.Context, sc action.Envelope, callerAddr address.Address, opts ...protocol.SimulateOption) (uint64, error) + EstimateExecutionGasConsumption(ctx context.Context, sc action.Envelope, callerAddr address.Address, opts ...protocol.SimulateOption) (uint64, []byte, error) // LogsInBlockByHash filter logs in the block by hash LogsInBlockByHash(filter *logfilter.LogFilter, blockHash hash.Hash256) ([]*action.Log, error) // LogsInRange filter logs among [start, end] blocks @@ -1569,19 +1569,19 @@ func (core *coreService) EstimateGasForNonExecution(actType action.Action) (uint } // EstimateMigrateStakeGasConsumption estimates gas consumption for migrate stake action -func (core *coreService) EstimateMigrateStakeGasConsumption(ctx context.Context, ms *action.MigrateStake, caller address.Address) (uint64, error) { +func (core *coreService) EstimateMigrateStakeGasConsumption(ctx context.Context, ms *action.MigrateStake, caller address.Address) (uint64, []byte, error) { g := core.bc.Genesis() header, err := core.bc.BlockHeaderByHeight(core.bc.TipHeight()) if err != nil { - return 0, err + return 0, nil, err } zeroAddr, err := address.FromString(address.ZeroAddress) if err != nil { - return 0, err + return 0, nil, err } ctx, err = core.bc.Context(ctx) if err != nil { - return 0, err + return 0, nil, err } tip := protocol.MustGetBlockchainCtx(ctx).Tip ctx = protocol.WithBlockCtx(ctx, protocol.BlockCtx{ @@ -1594,9 +1594,9 @@ func (core *coreService) EstimateMigrateStakeGasConsumption(ctx context.Context, }) exec, err := staking.FindProtocol(core.registry).ConstructExecution(ctx, ms, 0, 0, new(big.Int), core.sf) if err != nil { - return 0, err + return 0, nil, err } - gas, err := core.EstimateExecutionGasConsumption(ctx, exec, caller, protocol.WithSimulatePreOpt(func(sm protocol.StateManager) error { + gas, retval, err := core.EstimateExecutionGasConsumption(ctx, exec, caller, protocol.WithSimulatePreOpt(func(sm protocol.StateManager) error { // add amount to the sender account sender, err := accountutil.LoadAccount(sm, caller) if err != nil { @@ -1608,37 +1608,40 @@ func (core *coreService) EstimateMigrateStakeGasConsumption(ctx context.Context, return accountutil.StoreAccount(sm, caller, sender) })) if err != nil { - return 0, err + return 0, retval, err } intrinsicGas, err := ms.IntrinsicGas() if err != nil { - return 0, err + return 0, retval, err } - return gas + intrinsicGas, nil + return gas + intrinsicGas, retval, nil } // EstimateExecutionGasConsumption estimate gas consumption for execution action -func (core *coreService) EstimateExecutionGasConsumption(ctx context.Context, elp action.Envelope, callerAddr address.Address, opts ...protocol.SimulateOption) (uint64, error) { +func (core *coreService) EstimateExecutionGasConsumption(ctx context.Context, elp action.Envelope, callerAddr address.Address, opts ...protocol.SimulateOption) (uint64, []byte, error) { var ( g = core.bc.Genesis() blockGasLimit = g.BlockGasLimitByHeight(core.bc.TipHeight()) ) elp.SetGas(blockGasLimit) - enough, receipt, err := core.isGasLimitEnough(ctx, callerAddr, elp, opts...) + enough, receipt, retval, err := core.isGasLimitEnough(ctx, callerAddr, elp, opts...) if err != nil { - return 0, status.Error(codes.Internal, err.Error()) + return 0, nil, status.Error(codes.Internal, err.Error()) } if !enough { - if receipt.ExecutionRevertMsg() != "" { - return 0, status.Errorf(codes.Internal, fmt.Sprintf("execution simulation is reverted due to the reason: %s", receipt.ExecutionRevertMsg())) + if receipt.Status == uint64(iotextypes.ReceiptStatus_ErrExecutionReverted) { + if len(receipt.ExecutionRevertMsg()) > 0 { + return 0, retval, status.Errorf(codes.InvalidArgument, fmt.Sprintf("execution simulation is reverted due to the reason: %s", receipt.ExecutionRevertMsg())) + } + return 0, retval, status.Error(codes.InvalidArgument, "execution reverted") } - return 0, status.Error(codes.Internal, fmt.Sprintf("execution simulation failed: status = %d", receipt.Status)) + return 0, retval, status.Error(codes.Internal, fmt.Sprintf("execution simulation failed: status = %d", receipt.Status)) } estimatedGas := receipt.GasConsumed elp.SetGas(estimatedGas) - enough, _, err = core.isGasLimitEnough(ctx, callerAddr, elp, opts...) + enough, _, _, err = core.isGasLimitEnough(ctx, callerAddr, elp, opts...) if err != nil && err != action.ErrInsufficientFunds { - return 0, status.Error(codes.Internal, err.Error()) + return 0, nil, status.Error(codes.Internal, err.Error()) } if !enough { low, high := estimatedGas, blockGasLimit @@ -1646,9 +1649,9 @@ func (core *coreService) EstimateExecutionGasConsumption(ctx context.Context, el for low <= high { mid := (low + high) / 2 elp.SetGas(mid) - enough, _, err = core.isGasLimitEnough(ctx, callerAddr, elp, opts...) + enough, _, _, err = core.isGasLimitEnough(ctx, callerAddr, elp, opts...) if err != nil && err != action.ErrInsufficientFunds { - return 0, status.Error(codes.Internal, err.Error()) + return 0, nil, status.Error(codes.Internal, err.Error()) } if enough { estimatedGas = mid @@ -1659,7 +1662,7 @@ func (core *coreService) EstimateExecutionGasConsumption(ctx context.Context, el } } - return estimatedGas, nil + return estimatedGas, nil, nil } func (core *coreService) isGasLimitEnough( @@ -1667,14 +1670,14 @@ func (core *coreService) isGasLimitEnough( caller address.Address, elp action.Envelope, opts ...protocol.SimulateOption, -) (bool, *action.Receipt, error) { +) (bool, *action.Receipt, []byte, error) { ctx, span := tracer.NewSpan(ctx, "Server.isGasLimitEnough") defer span.End() - _, receipt, err := core.simulateExecution(ctx, caller, elp, opts...) + ret, receipt, err := core.simulateExecution(ctx, caller, elp, opts...) if err != nil { - return false, nil, err + return false, nil, nil, err } - return receipt.Status == uint64(iotextypes.ReceiptStatus_Success), receipt, nil + return receipt.Status == uint64(iotextypes.ReceiptStatus_Success), receipt, ret, nil } func (core *coreService) getProductivityByEpoch( diff --git a/api/coreservice_test.go b/api/coreservice_test.go index 5ba15fa53c..0940834314 100644 --- a/api/coreservice_test.go +++ b/api/coreservice_test.go @@ -24,6 +24,11 @@ import ( "github.com/iotexproject/go-pkgs/hash" "github.com/iotexproject/iotex-address/address" + "github.com/iotexproject/iotex-election/test/mock/mock_committee" + "github.com/iotexproject/iotex-election/types" + "github.com/iotexproject/iotex-proto/golang/iotexapi" + "github.com/iotexproject/iotex-proto/golang/iotextypes" + "github.com/iotexproject/iotex-core/v2/action" "github.com/iotexproject/iotex-core/v2/action/protocol" accountutil "github.com/iotexproject/iotex-core/v2/action/protocol/account/util" @@ -48,10 +53,6 @@ import ( "github.com/iotexproject/iotex-core/v2/test/mock/mock_envelope" "github.com/iotexproject/iotex-core/v2/test/mock/mock_factory" "github.com/iotexproject/iotex-core/v2/testutil" - "github.com/iotexproject/iotex-election/test/mock/mock_committee" - "github.com/iotexproject/iotex-election/types" - "github.com/iotexproject/iotex-proto/golang/iotexapi" - "github.com/iotexproject/iotex-proto/golang/iotextypes" ) func TestLogsInRange(t *testing.T) { @@ -493,7 +494,7 @@ func TestEstimateExecutionGasConsumption(t *testing.T) { bc.EXPECT().TipHeight().Return(uint64(1)).Times(1) bc.EXPECT().Context(gomock.Any()).Return(ctx, nil).Times(1) elp := (&action.EnvelopeBuilder{}).SetAction(&action.Execution{}).Build() - _, err := cs.EstimateExecutionGasConsumption(ctx, elp, &address.AddrV1{}) + _, _, err := cs.EstimateExecutionGasConsumption(ctx, elp, &address.AddrV1{}) require.ErrorContains(err, t.Name()) }) @@ -510,15 +511,15 @@ func TestEstimateExecutionGasConsumption(t *testing.T) { ctx context.Context, caller address.Address, sc *action.Execution, - ) (bool, *action.Receipt, error) { - return false, nil, errors.New(t.Name()) + ) (bool, *action.Receipt, []byte, error) { + return false, nil, nil, errors.New(t.Name()) }, ) bc.EXPECT().Genesis().Return(genesis.Genesis{}).Times(1) bc.EXPECT().TipHeight().Return(uint64(0)).Times(1) elp := (&action.EnvelopeBuilder{}).SetAction(&action.Execution{}).Build() - _, err := cs.EstimateExecutionGasConsumption(ctx, elp, &address.AddrV1{}) + _, _, err := cs.EstimateExecutionGasConsumption(ctx, elp, &address.AddrV1{}) require.ErrorContains(err, t.Name()) }) @@ -537,8 +538,8 @@ func TestEstimateExecutionGasConsumption(t *testing.T) { ctx context.Context, caller address.Address, sc *action.Execution, - ) (bool, *action.Receipt, error) { - return false, receipt, nil + ) (bool, *action.Receipt, []byte, error) { + return false, receipt, nil, nil }, ) p = p.ApplyMethodReturn(receipt, "ExecutionRevertMsg", "") @@ -546,7 +547,7 @@ func TestEstimateExecutionGasConsumption(t *testing.T) { bc.EXPECT().Genesis().Return(genesis.Genesis{}).Times(1) bc.EXPECT().TipHeight().Return(uint64(0)).Times(1) elp := (&action.EnvelopeBuilder{}).SetAction(&action.Execution{}).Build() - _, err := cs.EstimateExecutionGasConsumption(ctx, elp, &address.AddrV1{}) + _, _, err := cs.EstimateExecutionGasConsumption(ctx, elp, &address.AddrV1{}) require.ErrorContains(err, "execution simulation failed:") }) @@ -554,7 +555,9 @@ func TestEstimateExecutionGasConsumption(t *testing.T) { p := NewPatches() defer p.Reset() - receipt := &action.Receipt{} + receipt := &action.Receipt{ + Status: uint64(iotextypes.ReceiptStatus_ErrExecutionReverted), + } p = p.ApplyFuncReturn(accountutil.AccountState, &state.Account{}, nil) p = p.ApplyMethodReturn(&genesis.Blockchain{}, "BlockGasLimitByHeight", uint64(0)) p = p.ApplyPrivateMethod( @@ -564,8 +567,8 @@ func TestEstimateExecutionGasConsumption(t *testing.T) { ctx context.Context, caller address.Address, sc *action.Execution, - ) (bool, *action.Receipt, error) { - return false, receipt, nil + ) (bool, *action.Receipt, []byte, error) { + return false, receipt, nil, nil }, ) p = p.ApplyMethodReturn(receipt, "ExecutionRevertMsg", "TestRevertMsg") @@ -573,7 +576,7 @@ func TestEstimateExecutionGasConsumption(t *testing.T) { bc.EXPECT().Genesis().Return(genesis.Genesis{}).Times(1) bc.EXPECT().TipHeight().Return(uint64(0)).Times(1) elp := (&action.EnvelopeBuilder{}).SetAction(&action.Execution{}).Build() - _, err := cs.EstimateExecutionGasConsumption(ctx, elp, &address.AddrV1{}) + _, _, err := cs.EstimateExecutionGasConsumption(ctx, elp, &address.AddrV1{}) require.ErrorContains(err, "execution simulation is reverted due to the reason:") }) }) @@ -587,13 +590,13 @@ func TestEstimateExecutionGasConsumption(t *testing.T) { //gasprice is zero elp := (&action.EnvelopeBuilder{}).SetAction(sc).Build() - estimatedGas, err := svr.EstimateExecutionGasConsumption(context.Background(), elp, callAddr) + estimatedGas, _, err := svr.EstimateExecutionGasConsumption(context.Background(), elp, callAddr) require.NoError(err) require.Equal(uint64(10000), estimatedGas) //gasprice no zero, should return error elp = (&action.EnvelopeBuilder{}).SetGasPrice(big.NewInt(100)).SetAction(sc).Build() - estimatedGas, err = svr.EstimateExecutionGasConsumption(context.Background(), elp, callAddr) + estimatedGas, _, err = svr.EstimateExecutionGasConsumption(context.Background(), elp, callAddr) require.ErrorContains(err, "rpc error: code = Internal desc = insufficient funds for gas * price + value") require.Zero(estimatedGas) }) diff --git a/api/grpcserver.go b/api/grpcserver.go index 4d5de352bd..3856a08fbd 100644 --- a/api/grpcserver.go +++ b/api/grpcserver.go @@ -408,7 +408,7 @@ func (svr *gRPCHandler) EstimateActionGasConsumption(ctx context.Context, in *io return nil, status.Error(codes.InvalidArgument, err.Error()) } elp := (&action.EnvelopeBuilder{}).SetAction(sc).Build() - ret, err := svr.coreService.EstimateExecutionGasConsumption(ctx, elp, callerAddr) + ret, _, err := svr.coreService.EstimateExecutionGasConsumption(ctx, elp, callerAddr) if err != nil { return nil, err } @@ -423,7 +423,7 @@ func (svr *gRPCHandler) EstimateActionGasConsumption(ctx context.Context, in *io if err := ms.LoadProto(in.GetStakeMigrate()); err != nil { return nil, status.Error(codes.InvalidArgument, err.Error()) } - ret, err := svr.coreService.EstimateMigrateStakeGasConsumption(ctx, ms, callerAddr) + ret, _, err := svr.coreService.EstimateMigrateStakeGasConsumption(ctx, ms, callerAddr) if err != nil { return nil, status.Error(codes.InvalidArgument, err.Error()) } diff --git a/api/grpcserver_test.go b/api/grpcserver_test.go index f1d13ffb27..ff6d987982 100644 --- a/api/grpcserver_test.go +++ b/api/grpcserver_test.go @@ -596,7 +596,7 @@ func TestGrpcServer_EstimateActionGasConsumption(t *testing.T) { } t.Run("Execution is not nil", func(t *testing.T) { - core.EXPECT().EstimateExecutionGasConsumption(gomock.Any(), gomock.Any(), gomock.Any()).Return(uint64(10100), nil) + core.EXPECT().EstimateExecutionGasConsumption(gomock.Any(), gomock.Any(), gomock.Any()).Return(uint64(10100), nil, nil) request.Action = &iotexapi.EstimateActionGasConsumptionRequest_Execution{ Execution: &iotextypes.Execution{ Data: _executionHash1[:], diff --git a/api/web3server.go b/api/web3server.go index 7f01078b8b..fe7fffa8c6 100644 --- a/api/web3server.go +++ b/api/web3server.go @@ -456,18 +456,21 @@ func (svr *web3Handler) estimateGas(ctx context.Context, in *gjson.Result) (inte return nil, err } - var estimatedGas uint64 - from := callMsg.From + var ( + estimatedGas uint64 + retval []byte + from = callMsg.From + ) switch act := elp.Action().(type) { case *action.Execution: - estimatedGas, err = svr.coreService.EstimateExecutionGasConsumption(ctx, elp, from) + estimatedGas, retval, err = svr.coreService.EstimateExecutionGasConsumption(ctx, elp, from) case *action.MigrateStake: - estimatedGas, err = svr.coreService.EstimateMigrateStakeGasConsumption(ctx, act, from) + estimatedGas, retval, err = svr.coreService.EstimateMigrateStakeGasConsumption(ctx, act, from) default: estimatedGas, err = svr.coreService.EstimateGasForNonExecution(act) } if err != nil { - return nil, err + return "0x" + hex.EncodeToString(retval), err } if estimatedGas < 21000 { estimatedGas = 21000 diff --git a/api/web3server_test.go b/api/web3server_test.go index 5bf0a15477..d320000e26 100644 --- a/api/web3server_test.go +++ b/api/web3server_test.go @@ -394,7 +394,7 @@ func TestEstimateGas(t *testing.T) { t.Run("estimate execution", func(t *testing.T) { core.EXPECT().EVMNetworkID().Return(uint32(0)).AnyTimes() core.EXPECT().Account(gomock.Any()).Return(&iotextypes.AccountMeta{IsContract: true}, nil, nil) - core.EXPECT().EstimateExecutionGasConsumption(gomock.Any(), gomock.Any(), gomock.Any()).Return(uint64(11000), nil) + core.EXPECT().EstimateExecutionGasConsumption(gomock.Any(), gomock.Any(), gomock.Any()).Return(uint64(11000), nil, nil) in := gjson.Parse(`{"params":[{ "from": "", diff --git a/test/mock/mock_apicoreservice/mock_apicoreservice.go b/test/mock/mock_apicoreservice/mock_apicoreservice.go index 9985598d58..7777a1b9c3 100644 --- a/test/mock/mock_apicoreservice/mock_apicoreservice.go +++ b/test/mock/mock_apicoreservice/mock_apicoreservice.go @@ -306,7 +306,7 @@ func (mr *MockCoreServiceMockRecorder) EpochMeta(epochNum interface{}) *gomock.C } // EstimateExecutionGasConsumption mocks base method. -func (m *MockCoreService) EstimateExecutionGasConsumption(ctx context.Context, sc action.Envelope, callerAddr address.Address, opts ...protocol.SimulateOption) (uint64, error) { +func (m *MockCoreService) EstimateExecutionGasConsumption(ctx context.Context, sc action.Envelope, callerAddr address.Address, opts ...protocol.SimulateOption) (uint64, []byte, error) { m.ctrl.T.Helper() varargs := []interface{}{ctx, sc, callerAddr} for _, a := range opts { @@ -314,8 +314,9 @@ func (m *MockCoreService) EstimateExecutionGasConsumption(ctx context.Context, s } ret := m.ctrl.Call(m, "EstimateExecutionGasConsumption", varargs...) ret0, _ := ret[0].(uint64) - ret1, _ := ret[1].(error) - return ret0, ret1 + ret1, _ := ret[1].([]byte) + ret2, _ := ret[2].(error) + return ret0, ret1, ret2 } // EstimateExecutionGasConsumption indicates an expected call of EstimateExecutionGasConsumption. @@ -356,12 +357,13 @@ func (mr *MockCoreServiceMockRecorder) EstimateGasForNonExecution(arg0 interface } // EstimateMigrateStakeGasConsumption mocks base method. -func (m *MockCoreService) EstimateMigrateStakeGasConsumption(arg0 context.Context, arg1 *action.MigrateStake, arg2 address.Address) (uint64, error) { +func (m *MockCoreService) EstimateMigrateStakeGasConsumption(arg0 context.Context, arg1 *action.MigrateStake, arg2 address.Address) (uint64, []byte, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "EstimateMigrateStakeGasConsumption", arg0, arg1, arg2) ret0, _ := ret[0].(uint64) - ret1, _ := ret[1].(error) - return ret0, ret1 + ret1, _ := ret[1].([]byte) + ret2, _ := ret[2].(error) + return ret0, ret1, ret2 } // EstimateMigrateStakeGasConsumption indicates an expected call of EstimateMigrateStakeGasConsumption. From 991f1601c63742d25201c0e5ac634e09203b3073 Mon Sep 17 00:00:00 2001 From: dustinxie Date: Thu, 19 Dec 2024 16:22:28 -0800 Subject: [PATCH 12/21] [api] parse block number from request (#4526) --- api/coreservice_test.go | 21 +++++++------ api/web3server.go | 11 ++++--- api/web3server_integrity_test.go | 20 +++++------- api/web3server_test.go | 28 +++++++---------- api/web3server_utils.go | 52 +++++++++++++++++++------------- api/web3server_utils_test.go | 9 ++---- 6 files changed, 71 insertions(+), 70 deletions(-) diff --git a/api/coreservice_test.go b/api/coreservice_test.go index 0940834314..1c3323ae57 100644 --- a/api/coreservice_test.go +++ b/api/coreservice_test.go @@ -508,9 +508,10 @@ func TestEstimateExecutionGasConsumption(t *testing.T) { cs, "isGasLimitEnough", func( - ctx context.Context, - caller address.Address, - sc *action.Execution, + context.Context, + address.Address, + *action.Envelope, + ...protocol.SimulateOption, ) (bool, *action.Receipt, []byte, error) { return false, nil, nil, errors.New(t.Name()) }, @@ -535,9 +536,10 @@ func TestEstimateExecutionGasConsumption(t *testing.T) { cs, "isGasLimitEnough", func( - ctx context.Context, - caller address.Address, - sc *action.Execution, + context.Context, + address.Address, + *action.Envelope, + ...protocol.SimulateOption, ) (bool, *action.Receipt, []byte, error) { return false, receipt, nil, nil }, @@ -564,9 +566,10 @@ func TestEstimateExecutionGasConsumption(t *testing.T) { cs, "isGasLimitEnough", func( - ctx context.Context, - caller address.Address, - sc *action.Execution, + context.Context, + address.Address, + *action.Envelope, + ...protocol.SimulateOption, ) (bool, *action.Receipt, []byte, error) { return false, receipt, nil, nil }, diff --git a/api/web3server.go b/api/web3server.go index fe7fffa8c6..5307bcb578 100644 --- a/api/web3server.go +++ b/api/web3server.go @@ -393,7 +393,10 @@ func (svr *web3Handler) call(ctx context.Context, in *gjson.Result) (interface{} if err != nil { return nil, err } - callerAddr, to, gasLimit, value, data := callMsg.From, callMsg.To, callMsg.Gas, callMsg.Value, callMsg.Data + var ( + to = callMsg.To + data = callMsg.Data + ) if to == _metamaskBalanceContractAddr { return nil, nil } @@ -427,9 +430,9 @@ func (svr *web3Handler) call(ctx context.Context, in *gjson.Result) (interface{} } return "0x" + ret, nil } - elp := (&action.EnvelopeBuilder{}).SetAction(action.NewExecution(to, value, data)). - SetGasLimit(gasLimit).Build() - ret, receipt, err := svr.coreService.ReadContract(ctx, callerAddr, elp) + elp := (&action.EnvelopeBuilder{}).SetAction(action.NewExecution(to, callMsg.Value, data)). + SetGasLimit(callMsg.Gas).Build() + ret, receipt, err := svr.coreService.ReadContract(ctx, callMsg.From, elp) if err != nil { return nil, err } diff --git a/api/web3server_integrity_test.go b/api/web3server_integrity_test.go index caffb3d412..3eaeb286dd 100644 --- a/api/web3server_integrity_test.go +++ b/api/web3server_integrity_test.go @@ -243,7 +243,7 @@ func getBlockByNumber(t *testing.T, handler *hTTPHandler) { func getBalance(t *testing.T, handler *hTTPHandler) { require := require.New(t) - result := serveTestHTTP(require, handler, "eth_getBalance", `["0xDa7e12Ef57c236a06117c5e0d04a228e7181CF36", 1]`) + result := serveTestHTTP(require, handler, "eth_getBalance", `["0xDa7e12Ef57c236a06117c5e0d04a228e7181CF36"]`) ans, ok := new(big.Int).SetString("9999999999999999999999999991", 10) require.True(ok) actual, ok := result.(string) @@ -283,8 +283,7 @@ func ethCall(t *testing.T, handler *hTTPHandler) { "gasPrice": "0xe8d4a51000", "value": "0x1", "data": "0x1" - }, - 1]`, + }]`, 1, }, { @@ -295,8 +294,7 @@ func ethCall(t *testing.T, handler *hTTPHandler) { "gasPrice": "0xe8d4a51000", "value": "0x1", "data": "0x1" - }, - 1]`, + }]`, 0, }, } { @@ -681,8 +679,7 @@ func web3Staking(t *testing.T, handler *hTTPHandler) { "gas": "0x0", "gasPrice": "0x0", "value": "0x0", - "data": "%s"}, - 1]`, identityset.Address(28).Hex(), toAddr, hex.EncodeToString(test.data))) + "data": "%s"}]`, identityset.Address(28).Hex(), toAddr, hex.EncodeToString(test.data))) actual, ok := result.(string) require.True(ok) gasLimit, err := hexStringToNumber(actual) @@ -746,8 +743,7 @@ func estimateGas(t *testing.T, handler *hTTPHandler, bc blockchain.Blockchain, d "gas": "0x0", "gasPrice": "0x0", "value": "0x0", - "data": "0x1123123c"}, - 1]`, fromAddr, toAddr), + "data": "0x1123123c"}]`, fromAddr, toAddr), 21000, }, { @@ -757,8 +753,7 @@ func estimateGas(t *testing.T, handler *hTTPHandler, bc blockchain.Blockchain, d "gas": "0x0", "gasPrice": "0x0", "value": "0x0", - "data": "344933be000000000000000000000000000000000000000000000000000be497a92e9f3300000000000000000000000000000000000000000000000000000000000000a0000000000000000000000000f8be4046fd89199906ca348bcd3822c4b250e246000000000000000000000000000000000000000000000000000000006173a15400000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002000000000000000000000000a00744882684c3e4747faefd68d283ea44099d030000000000000000000000000258866edaf84d6081df17660357ab20a07d0c80"}, - 1]`, fromAddr, toAddr), + "data": "344933be000000000000000000000000000000000000000000000000000be497a92e9f3300000000000000000000000000000000000000000000000000000000000000a0000000000000000000000000f8be4046fd89199906ca348bcd3822c4b250e246000000000000000000000000000000000000000000000000000000006173a15400000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000002000000000000000000000000a00744882684c3e4747faefd68d283ea44099d030000000000000000000000000258866edaf84d6081df17660357ab20a07d0c80"}]`, fromAddr, toAddr), 36000, }, { @@ -768,8 +763,7 @@ func estimateGas(t *testing.T, handler *hTTPHandler, bc blockchain.Blockchain, d "gas": "0x0", "gasPrice": "0x1000000000000000000", "value": "0x0", - "data": "0x6d4ce63c"}, - 1]`, fromAddr, contractAddr), + "data": "0x6d4ce63c"}]`, fromAddr, contractAddr), 21000, }, } { diff --git a/api/web3server_test.go b/api/web3server_test.go index d320000e26..d967653de9 100644 --- a/api/web3server_test.go +++ b/api/web3server_test.go @@ -266,7 +266,7 @@ func TestGetBalance(t *testing.T) { balance := "111111111111111111" core.EXPECT().Account(gomock.Any()).Return(&iotextypes.AccountMeta{Balance: balance}, nil, nil) - in := gjson.Parse(`{"params":["0xDa7e12Ef57c236a06117c5e0d04a228e7181CF36", 1]}`) + in := gjson.Parse(`{"params":["0xDa7e12Ef57c236a06117c5e0d04a228e7181CF36", "0x1"]}`) ret, err := web3svr.getBalance(&in) require.NoError(err) ans, ok := new(big.Int).SetString(balance, 10) @@ -283,11 +283,11 @@ func TestGetTransactionCount(t *testing.T) { core.EXPECT().PendingNonce(gomock.Any()).Return(uint64(2), nil) inNil := gjson.Parse(`{"params":[]}`) - ret, err := web3svr.getTransactionCount(&inNil) + _, err := web3svr.getTransactionCount(&inNil) require.EqualError(err, errInvalidFormat.Error()) in := gjson.Parse(`{"params":["0xDa7e12Ef57c236a06117c5e0d04a228e7181CF36", 1]}`) - ret, err = web3svr.getTransactionCount(&in) + ret, err := web3svr.getTransactionCount(&in) require.NoError(err) require.Equal("0x2", ret.(string)) } @@ -315,8 +315,7 @@ func TestCall(t *testing.T) { "gasPrice": "0xe8d4a51000", "value": "0x1", "data": "d201114a" - }, - 1]}`) + }, "0x1"]}`) ret, err := web3svr.call(context.Background(), &in) require.NoError(err) require.Equal("0x0000000000000000000000000000000000000000000000056bc75e2d63100000", ret.(string)) @@ -334,8 +333,7 @@ func TestCall(t *testing.T) { "gasPrice": "0xe8d4a51000", "value": "0x1", "data": "ad7a672f" - }, - 1]}`) + }, "0x1"]}`) ret, err := web3svr.call(context.Background(), &in) require.NoError(err) require.Equal("0x0000000000000000000000000000000000000000000000000000000000002710", ret.(string)) @@ -350,8 +348,7 @@ func TestCall(t *testing.T) { "gasPrice": "0xe8d4a51000", "value": "0x1", "data": "0x1" - }, - 1]}`) + }, "0x1"]}`) ret, err := web3svr.call(context.Background(), &in) require.NoError(err) require.Equal("0x111111", ret.(string)) @@ -376,8 +373,7 @@ func TestCall(t *testing.T) { "gasPrice": "0xe8d4a51000", "value": "0x1", "data": "0x1" - }, - 1]}`) + }, "0x1"]}`) _, err := web3svr.call(context.Background(), &in) require.EqualError(err, "rpc error: code = InvalidArgument desc = execution reverted: "+receipt.GetExecutionRevertMsg()) }) @@ -390,9 +386,9 @@ func TestEstimateGas(t *testing.T) { core := mock_apicoreservice.NewMockCoreService(ctrl) web3svr := &web3Handler{core, nil, _defaultBatchRequestLimit} core.EXPECT().ChainID().Return(uint32(1)).Times(2) + core.EXPECT().EVMNetworkID().Return(uint32(0)).Times(2) t.Run("estimate execution", func(t *testing.T) { - core.EXPECT().EVMNetworkID().Return(uint32(0)).AnyTimes() core.EXPECT().Account(gomock.Any()).Return(&iotextypes.AccountMeta{IsContract: true}, nil, nil) core.EXPECT().EstimateExecutionGasConsumption(gomock.Any(), gomock.Any(), gomock.Any()).Return(uint64(11000), nil, nil) @@ -403,8 +399,7 @@ func TestEstimateGas(t *testing.T) { "gasPrice": "0xe8d4a51000", "value": "0x1", "data": "0x6d4ce63c" - }, - 1]}`) + }, "0x1"]}`) ret, err := web3svr.estimateGas(context.Background(), &in) require.NoError(err) require.Equal(uint64ToHex(uint64(21000)), ret.(string)) @@ -421,8 +416,7 @@ func TestEstimateGas(t *testing.T) { "gasPrice": "0xe8d4a51000", "value": "0x1", "data": "0x1123123c" - }, - 1]}`) + }, "0x1"]}`) ret, err := web3svr.estimateGas(context.Background(), &in) require.NoError(err) require.Equal(uint64ToHex(uint64(36000)), ret.(string)) @@ -1261,7 +1255,7 @@ func TestDebugTraceCall(t *testing.T) { core.EXPECT().TraceCall(ctx, gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).AnyTimes().Return([]byte{0x01}, receipt, structLogger, nil) - in := gjson.Parse(`{"method":"debug_traceCall","params":[{"from":null,"to":"0x6b175474e89094c44da98b954eedeac495271d0f","data":"0x70a082310000000000000000000000006E0d01A76C3Cf4288372a29124A26D4353EE51BE"}, {"blockNumber":1}],"id":1,"jsonrpc":"2.0"}`) + in := gjson.Parse(`{"method":"debug_traceCall","params":[{"from":null,"to":"0x6b175474e89094c44da98b954eedeac495271d0f","data":"0x70a082310000000000000000000000006E0d01A76C3Cf4288372a29124A26D4353EE51BE"}],"id":1,"jsonrpc":"2.0"}`) ret, err := web3svr.traceCall(ctx, &in) require.NoError(err) rlt, ok := ret.(*debugTraceTransactionResult) diff --git a/api/web3server_utils.go b/api/web3server_utils.go index 6540ca1ee1..dd264b1404 100644 --- a/api/web3server_utils.go +++ b/api/web3server_utils.go @@ -13,6 +13,7 @@ import ( "github.com/ethereum/go-ethereum/common/math" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/eth/tracers/logger" + "github.com/ethereum/go-ethereum/rpc" "github.com/go-redis/redis/v8" "github.com/iotexproject/go-pkgs/cache/ttl" "github.com/iotexproject/go-pkgs/hash" @@ -267,16 +268,16 @@ func parseLogRequest(in gjson.Result) (*filterObject, error) { } type callMsg struct { - From address.Address // the sender of the 'transaction' - To string // the destination contract (empty for contract creation) - Gas uint64 // if 0, the call executes with near-infinite gas - GasPrice *big.Int // wei <-> gas exchange ratio - GasFeeCap *big.Int // EIP-1559 fee cap per gas. - GasTipCap *big.Int // EIP-1559 tip per gas. - Value *big.Int // amount of wei sent along with the call - Data []byte // input data, usually an ABI-encoded contract method invocation - - AccessList types.AccessList // EIP-2930 access list. + From address.Address // the sender of the 'transaction' + To string // the destination contract (empty for contract creation) + Gas uint64 // if 0, the call executes with near-infinite gas + GasPrice *big.Int // wei <-> gas exchange ratio + GasFeeCap *big.Int // EIP-1559 fee cap per gas. + GasTipCap *big.Int // EIP-1559 tip per gas. + Value *big.Int // amount of wei sent along with the call + Data []byte // input data, usually an ABI-encoded contract method invocation + AccessList types.AccessList // EIP-2930 access list. + BlockNumber rpc.BlockNumber } func parseCallObject(in *gjson.Result) (*callMsg, error) { @@ -290,6 +291,7 @@ func parseCallObject(in *gjson.Result) (*callMsg, error) { value *big.Int = big.NewInt(0) data []byte acl types.AccessList + bn = rpc.LatestBlockNumber err error ) fromStr := in.Get("params.0.from").String() @@ -346,8 +348,7 @@ func parseCallObject(in *gjson.Result) (*callMsg, error) { } } - input := in.Get("params.0.input") - if input.Exists() { + if input := in.Get("params.0.input"); input.Exists() { data = common.FromHex(input.String()) } else { data = common.FromHex(in.Get("params.0.data").String()) @@ -360,16 +361,25 @@ func parseCallObject(in *gjson.Result) (*callMsg, error) { return nil, errors.Wrapf(err, "failed to unmarshal access list %s", accessList.Raw) } } + if bnParam := in.Get("params.1"); bnParam.Exists() { + if err = bn.UnmarshalJSON([]byte(bnParam.String())); err != nil { + return nil, errors.Wrapf(err, "failed to unmarshal height %s", bnParam.String()) + } + if bn == rpc.PendingBlockNumber { + return nil, errors.Wrap(errNotImplemented, "pending block number is not supported") + } + } return &callMsg{ - From: from, - To: to, - Gas: gasLimit, - GasPrice: gasPrice, - GasFeeCap: gasFeeCap, - GasTipCap: gasTipCap, - Value: value, - Data: data, - AccessList: acl, + From: from, + To: to, + Gas: gasLimit, + GasPrice: gasPrice, + GasFeeCap: gasFeeCap, + GasTipCap: gasTipCap, + Value: value, + Data: data, + AccessList: acl, + BlockNumber: bn, }, nil } diff --git a/api/web3server_utils_test.go b/api/web3server_utils_test.go index 9870e2412e..685874e203 100644 --- a/api/web3server_utils_test.go +++ b/api/web3server_utils_test.go @@ -41,8 +41,7 @@ func TestParseCallObject(t *testing.T) { "gasPrice": "0xe8d4a51000", "value": "0x1", "data": "0x6d4ce63c" - }, - 1]}`, + }, "0x1"]}`, from: address.ZeroAddress, to: "io10sfcvmuj2000083qqd8d6qg7r457vll9gly090", gasLimit: 20000, @@ -59,8 +58,7 @@ func TestParseCallObject(t *testing.T) { "gasPrice": "0xe8d4a51000", "value": "0x1", "input": "0x6d4ce63c" - }, - 1]}`, + }, "0x1"]}`, from: address.ZeroAddress, to: "io10sfcvmuj2000083qqd8d6qg7r457vll9gly090", gasLimit: 20000, @@ -84,8 +82,7 @@ func TestParseCallObject(t *testing.T) { "storageKeys": ["0x0000000000000000000000001a2f3b98e2f5a0f9f9f3f3f3f3f3f3f3f3f3f3f3"] } ] - }, - 1]}`, + }, "0x1"]}`, from: "io1rghnhx8z7ks0n70n70el8uln70el8ulnp8hq9l", to: "io10sfcvmuj2000083qqd8d6qg7r457vll9gly090", gasLimit: 20000, From e53ef04a76abca883e0430464c0c16291df22a5c Mon Sep 17 00:00:00 2001 From: Chen Chen Date: Fri, 20 Dec 2024 11:05:24 +0800 Subject: [PATCH 13/21] grpcdao secure (#4525) Co-authored-by: dustinxie --- blockchain/blockdao/grpcblockdao.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/blockchain/blockdao/grpcblockdao.go b/blockchain/blockdao/grpcblockdao.go index 30e8dad625..1209c8041d 100644 --- a/blockchain/blockdao/grpcblockdao.go +++ b/blockchain/blockdao/grpcblockdao.go @@ -7,6 +7,7 @@ package blockdao import ( "context" + "crypto/tls" "encoding/hex" "fmt" "sync/atomic" @@ -15,6 +16,7 @@ import ( "github.com/iotexproject/iotex-proto/golang/iotextypes" "github.com/pkg/errors" "google.golang.org/grpc" + "google.golang.org/grpc/credentials" "google.golang.org/grpc/credentials/insecure" "google.golang.org/protobuf/types/known/emptypb" @@ -57,8 +59,10 @@ func (gbd *GrpcBlockDAO) Start(ctx context.Context) error { opts := []grpc.DialOption{} if gbd.insecure { opts = append(opts, grpc.WithTransportCredentials(insecure.NewCredentials())) + } else { + opts = append(opts, grpc.WithTransportCredentials(credentials.NewTLS(&tls.Config{MinVersion: tls.VersionTLS12}))) } - gbd.conn, err = grpc.Dial(gbd.url, opts...) + gbd.conn, err = grpc.NewClient(gbd.url, opts...) if err != nil { return err } From 3580fd0dd8648a73b120fe1b7d91c4739ccda86d Mon Sep 17 00:00:00 2001 From: wanxiangchwng Date: Tue, 24 Dec 2024 10:29:02 +0800 Subject: [PATCH 14/21] fix some function names in comment (#4502) --- api/coreservice.go | 2 +- ioctl/cmd/bc/bcbucketlist.go | 2 +- server/itx/nodestats/systemstats.go | 2 +- state/factory/factory.go | 2 +- state/factory/statedb.go | 2 +- state/factory/workingset.go | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/api/coreservice.go b/api/coreservice.go index f18b261f23..80710080d6 100644 --- a/api/coreservice.go +++ b/api/coreservice.go @@ -127,7 +127,7 @@ type ( ActionByActionHash(h hash.Hash256) (*action.SealedEnvelope, *block.Block, uint32, error) // PendingActionByActionHash returns action by action hash PendingActionByActionHash(h hash.Hash256) (*action.SealedEnvelope, error) - // ActPoolActions returns the all Transaction Identifiers in the actpool + // ActionsInActPool returns the all Transaction Identifiers in the actpool ActionsInActPool(actHashes []string) ([]*action.SealedEnvelope, error) // BlockByHeightRange returns blocks within the height range BlockByHeightRange(uint64, uint64) ([]*apitypes.BlockWithReceipts, error) diff --git a/ioctl/cmd/bc/bcbucketlist.go b/ioctl/cmd/bc/bcbucketlist.go index 9eee2531db..f5788d4680 100644 --- a/ioctl/cmd/bc/bcbucketlist.go +++ b/ioctl/cmd/bc/bcbucketlist.go @@ -110,7 +110,7 @@ func getBucketList(method, addr string, args ...string) (err error) { return output.NewError(output.InputError, "unknown ", nil) } -// getBucketList get bucket list from chain by voter address +// getBucketListByVoter get bucket list from chain by voter address func getBucketListByVoter(addr string, offset, limit uint32) error { address, err := util.GetAddress(addr) if err != nil { diff --git a/server/itx/nodestats/systemstats.go b/server/itx/nodestats/systemstats.go index 30d66cc5a9..c9f18615d2 100644 --- a/server/itx/nodestats/systemstats.go +++ b/server/itx/nodestats/systemstats.go @@ -17,7 +17,7 @@ type DiskStatus struct { Free uint64 `json:"Free"` } -// diskusage of path/disk +// diskUsage of path/disk func diskUsage(path string) (disk DiskStatus) { fs := syscall.Statfs_t{} err := syscall.Statfs(path, &fs) diff --git a/state/factory/factory.go b/state/factory/factory.go index 228f5ca268..37cd5a7b62 100644 --- a/state/factory/factory.go +++ b/state/factory/factory.go @@ -499,7 +499,7 @@ func (sf *factory) State(s interface{}, opts ...protocol.StateOption) (uint64, e return sf.currentChainHeight, state.Deserialize(s, value) } -// State returns a set states in the state factory +// States returns a set states in the state factory func (sf *factory) States(opts ...protocol.StateOption) (uint64, state.Iterator, error) { sf.mutex.RLock() defer sf.mutex.RUnlock() diff --git a/state/factory/statedb.go b/state/factory/statedb.go index 2e2be991aa..638debf7ab 100644 --- a/state/factory/statedb.go +++ b/state/factory/statedb.go @@ -339,7 +339,7 @@ func (sdb *stateDB) State(s interface{}, opts ...protocol.StateOption) (uint64, return sdb.currentChainHeight, sdb.state(sdb.currentChainHeight, cfg.Namespace, cfg.Key, s) } -// State returns a set of states in the state factory +// States returns a set of states in the state factory func (sdb *stateDB) States(opts ...protocol.StateOption) (uint64, state.Iterator, error) { cfg, err := processOptions(opts...) if err != nil { diff --git a/state/factory/workingset.go b/state/factory/workingset.go index cb28cf880a..890ae6d0f4 100644 --- a/state/factory/workingset.go +++ b/state/factory/workingset.go @@ -422,7 +422,7 @@ func (ws *workingSet) Reset() { ws.dock.Reset() } -// createGenesisStates initialize the genesis states +// CreateGenesisStates initialize the genesis states func (ws *workingSet) CreateGenesisStates(ctx context.Context) error { if reg, ok := protocol.GetRegistry(ctx); ok { for _, p := range reg.All() { From c8c7a849c350d33b50c783b89c120660f90fba54 Mon Sep 17 00:00:00 2001 From: Chen Chen Date: Wed, 25 Dec 2024 14:26:50 +0800 Subject: [PATCH 15/21] [api] impl eth_feeHistory (#4527) --- api/coreservice.go | 7 + api/web3server.go | 39 +++++ api/web3server_integrity_test.go | 36 +++++ api/web3server_marshal.go | 9 ++ api/web3server_utils.go | 19 +++ gasstation/config.go | 14 +- gasstation/gasstattion.go | 148 +++++++++++++++++- .../mock_apicoreservice.go | 20 +++ 8 files changed, 280 insertions(+), 12 deletions(-) diff --git a/api/coreservice.go b/api/coreservice.go index 80710080d6..66ace25a13 100644 --- a/api/coreservice.go +++ b/api/coreservice.go @@ -97,6 +97,8 @@ type ( SuggestGasPrice() (uint64, error) // SuggestGasTipCap suggests gas tip cap SuggestGasTipCap() (*big.Int, error) + // FeeHistory returns the fee history + FeeHistory(ctx context.Context, blocks, lastBlock uint64, rewardPercentiles []float64) (uint64, [][]*big.Int, []*big.Int, []float64, []*big.Int, []float64, error) // EstimateGasForAction estimates gas for action EstimateGasForAction(ctx context.Context, in *iotextypes.Action) (uint64, error) // EpochMeta gets epoch metadata @@ -600,6 +602,11 @@ func (core *coreService) SuggestGasTipCap() (*big.Int, error) { return fee, nil } +// FeeHistory returns the fee history +func (core *coreService) FeeHistory(ctx context.Context, blocks, lastBlock uint64, rewardPercentiles []float64) (uint64, [][]*big.Int, []*big.Int, []float64, []*big.Int, []float64, error) { + return core.gs.FeeHistory(ctx, blocks, lastBlock, rewardPercentiles) +} + // EstimateGasForAction estimates gas for action func (core *coreService) EstimateGasForAction(ctx context.Context, in *iotextypes.Action) (uint64, error) { selp, err := (&action.Deserializer{}).SetEvmNetworkID(core.EVMNetworkID()).ActionToSealedEnvelope(in) diff --git a/api/web3server.go b/api/web3server.go index 5307bcb578..414e1d009e 100644 --- a/api/web3server.go +++ b/api/web3server.go @@ -6,6 +6,7 @@ import ( "encoding/json" "fmt" "io" + "math/big" "strconv" "time" @@ -167,6 +168,8 @@ func (svr *web3Handler) handleWeb3Req(ctx context.Context, web3Req *gjson.Result res, err = svr.gasPrice() case "eth_maxPriorityFeePerGas": res, err = svr.maxPriorityFee() + case "eth_feeHistory": + res, err = svr.feeHistory(ctx, web3Req) case "eth_getBlockByHash": res, err = svr.getBlockByHash(web3Req) case "eth_chainId": @@ -325,6 +328,42 @@ func (svr *web3Handler) maxPriorityFee() (interface{}, error) { return uint64ToHex(ret.Uint64()), nil } +func (svr *web3Handler) feeHistory(ctx context.Context, in *gjson.Result) (interface{}, error) { + blkCnt, newestBlk, rewardPercentiles := in.Get("params.0"), in.Get("params.1"), in.Get("params.2") + if !blkCnt.Exists() || !newestBlk.Exists() { + return nil, errInvalidFormat + } + blocks, err := strconv.ParseUint(blkCnt.String(), 10, 64) + if err != nil { + return nil, err + } + lastBlock, err := svr.parseBlockNumber(newestBlk.String()) + if err != nil { + return nil, err + } + rewardPercents := []float64{} + if rewardPercentiles.Exists() { + for _, p := range rewardPercentiles.Array() { + rewardPercents = append(rewardPercents, p.Float()) + } + } + oldest, reward, baseFee, gasRatio, blobBaseFee, blobGasRatio, err := svr.coreService.FeeHistory(ctx, blocks, lastBlock, rewardPercents) + if err != nil { + return nil, err + } + + return &feeHistoryResult{ + OldestBlock: uint64ToHex(oldest), + BaseFeePerGas: mapper(baseFee, bigIntToHex), + GasUsedRatio: gasRatio, + BaseFeePerBlobGas: mapper(blobBaseFee, bigIntToHex), + BlobGasUsedRatio: blobGasRatio, + Reward: mapper(reward, func(a []*big.Int) []string { + return mapper(a, bigIntToHex) + }), + }, nil +} + func (svr *web3Handler) getChainID() (interface{}, error) { return uint64ToHex(uint64(svr.coreService.EVMNetworkID())), nil } diff --git a/api/web3server_integrity_test.go b/api/web3server_integrity_test.go index 3eaeb286dd..8c3dc89938 100644 --- a/api/web3server_integrity_test.go +++ b/api/web3server_integrity_test.go @@ -157,6 +157,10 @@ func TestWeb3ServerIntegrity(t *testing.T) { t.Run("eth_getStorageAt", func(t *testing.T) { getStorageAt(t, handler, bc, dao, actPool) }) + + t.Run("eth_feeHistory", func(t *testing.T) { + feeHistory(t, handler, bc, dao, actPool) + }) } func setupTestServer() (*ServerV2, blockchain.Blockchain, blockdao.BlockDAO, actpool.ActPool, func()) { @@ -813,3 +817,35 @@ func getStorageAt(t *testing.T, handler *hTTPHandler, bc blockchain.Blockchain, require.Equal("0x0000000000000000000000000000000000000000000000000000000000000000", actual) } } + +func feeHistory(t *testing.T, handler *hTTPHandler, bc blockchain.Blockchain, dao blockdao.BlockDAO, actPool actpool.ActPool) { + require := require.New(t) + for _, test := range []struct { + params string + expected int + }{ + {`[4, "latest", [25,75]]`, 1}, + } { + oldnest := max(bc.TipHeight()-4+1, 1) + result := serveTestHTTP(require, handler, "eth_feeHistory", test.params) + if test.expected == 0 { + require.Nil(result) + continue + } + actual, err := json.Marshal(result) + require.NoError(err) + require.JSONEq(fmt.Sprintf(`{ + "oldestBlock": "0x%0x", + "reward": [ + ["0x0", "0x0"], + ["0x0", "0x0"], + ["0x0", "0x0"], + ["0x0", "0x0"] + ], + "baseFeePerGas": ["0x0","0x0","0x0","0x0","0x0"], + "gasUsedRatio": [0,0,0,0], + "baseFeePerBlobGas": ["0x1", "0x1", "0x1", "0x1", "0x1"], + "blobGasUsedRatio": [0, 0, 0, 0] + }`, oldnest), string(actual)) + } +} diff --git a/api/web3server_marshal.go b/api/web3server_marshal.go index 945d92261b..f5cd5f5855 100644 --- a/api/web3server_marshal.go +++ b/api/web3server_marshal.go @@ -87,6 +87,15 @@ type ( Gas uint64 `json:"gas"` StructLogs []apitypes.StructLog `json:"structLogs"` } + + feeHistoryResult struct { + OldestBlock string `json:"oldestBlock"` + BaseFeePerGas []string `json:"baseFeePerGas"` + GasUsedRatio []float64 `json:"gasUsedRatio"` + BaseFeePerBlobGas []string `json:"baseFeePerBlobGas"` + BlobGasUsedRatio []float64 `json:"blobGasUsedRatio"` + Reward [][]string `json:"reward,omitempty"` + } ) var ( diff --git a/api/web3server_utils.go b/api/web3server_utils.go index dd264b1404..8e1ad917d5 100644 --- a/api/web3server_utils.go +++ b/api/web3server_utils.go @@ -59,6 +59,25 @@ func uint64ToHex(val uint64) string { return "0x" + strconv.FormatUint(val, 16) } +func bigIntToHex(b *big.Int) string { + if b == nil { + return "0x0" + } + if b.Sign() == 0 { + return "0x0" + } + return "0x" + b.Text(16) +} + +// mapper maps a slice of S to a slice of T +func mapper[S, T any](arr []S, fn func(S) T) []T { + ret := make([]T, len(arr)) + for i, v := range arr { + ret[i] = fn(v) + } + return ret +} + func intStrToHex(str string) (string, error) { amount, ok := new(big.Int).SetString(str, 10) if !ok { diff --git a/gasstation/config.go b/gasstation/config.go index 40e54d3918..b1afb88fc9 100644 --- a/gasstation/config.go +++ b/gasstation/config.go @@ -9,14 +9,16 @@ import "github.com/iotexproject/iotex-core/v2/pkg/unit" // Config is the gas station config type Config struct { - SuggestBlockWindow int `yaml:"suggestBlockWindow"` - DefaultGas uint64 `yaml:"defaultGas"` - Percentile int `yaml:"Percentile"` + SuggestBlockWindow int `yaml:"suggestBlockWindow"` + DefaultGas uint64 `yaml:"defaultGas"` + Percentile int `yaml:"Percentile"` + FeeHistoryCacheSize int `yaml:"feeHistoryCacheSize"` } // DefaultConfig is the default config var DefaultConfig = Config{ - SuggestBlockWindow: 20, - DefaultGas: uint64(unit.Qev), - Percentile: 60, + SuggestBlockWindow: 20, + DefaultGas: uint64(unit.Qev), + Percentile: 60, + FeeHistoryCacheSize: 1024, } diff --git a/gasstation/gasstattion.go b/gasstation/gasstattion.go index 6853c698fa..b21ffdbdf8 100644 --- a/gasstation/gasstattion.go +++ b/gasstation/gasstattion.go @@ -10,19 +10,27 @@ import ( "math/big" "sort" + "github.com/ethereum/go-ethereum/params" + "github.com/iotexproject/go-pkgs/cache" "github.com/iotexproject/go-pkgs/hash" "github.com/iotexproject/iotex-address/address" + "go.uber.org/zap" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" "github.com/iotexproject/iotex-core/v2/action" + "github.com/iotexproject/iotex-core/v2/action/protocol" "github.com/iotexproject/iotex-core/v2/action/protocol/execution/evm" "github.com/iotexproject/iotex-core/v2/blockchain" "github.com/iotexproject/iotex-core/v2/blockchain/block" + "github.com/iotexproject/iotex-core/v2/pkg/log" ) // BlockDAO represents the block data access object type BlockDAO interface { GetBlockHash(uint64) (hash.Hash256, error) GetBlockByHeight(uint64) (*block.Block, error) + GetReceipts(uint64) ([]*action.Receipt, error) } // SimulateFunc is function that simulate execution @@ -30,17 +38,21 @@ type SimulateFunc func(context.Context, address.Address, *action.Execution, evm. // GasStation provide gas related api type GasStation struct { - bc blockchain.Blockchain - dao BlockDAO - cfg Config + bc blockchain.Blockchain + dao BlockDAO + cfg Config + feeCache cache.LRUCache + percentileCache cache.LRUCache } // NewGasStation creates a new gas station func NewGasStation(bc blockchain.Blockchain, dao BlockDAO, cfg Config) *GasStation { return &GasStation{ - bc: bc, - dao: dao, - cfg: cfg, + bc: bc, + dao: dao, + cfg: cfg, + feeCache: cache.NewThreadSafeLruCache(cfg.FeeHistoryCacheSize), + percentileCache: cache.NewThreadSafeLruCache(cfg.FeeHistoryCacheSize), } } @@ -103,3 +115,127 @@ func (gs *GasStation) SuggestGasPrice() (uint64, error) { } return gasPrice, nil } + +type blockFee struct { + baseFee *big.Int + gasUsedRatio float64 + blobBaseFee *big.Int + blobGasRatio float64 +} + +type blockPercents struct { + ascEffectivePriorityFees []*big.Int +} + +// FeeHistory returns fee history over a series of blocks +func (gs *GasStation) FeeHistory(ctx context.Context, blocks, lastBlock uint64, rewardPercentiles []float64) (uint64, [][]*big.Int, []*big.Int, []float64, []*big.Int, []float64, error) { + if blocks < 1 { + return 0, nil, nil, nil, nil, nil, nil + } + maxFeeHistory := uint64(1024) + if blocks > maxFeeHistory { + log.T(ctx).Warn("Sanitizing fee history length", zap.Uint64("requested", blocks), zap.Uint64("truncated", maxFeeHistory)) + blocks = maxFeeHistory + } + for i, p := range rewardPercentiles { + if p < 0 || p > 100 { + return 0, nil, nil, nil, nil, nil, status.Error(codes.InvalidArgument, "percentile must be in [0, 100]") + } + if i > 0 && p <= rewardPercentiles[i-1] { + return 0, nil, nil, nil, nil, nil, status.Error(codes.InvalidArgument, "percentiles must be in ascending order") + } + } + + var ( + rewards = make([][]*big.Int, 0, blocks) + baseFees = make([]*big.Int, blocks+1) + gasUsedRatios = make([]float64, blocks) + blobBaseFees = make([]*big.Int, blocks+1) + blobGasUsedRatios = make([]float64, blocks) + g = gs.bc.Genesis() + lastBlk *block.Block + ) + for i := uint64(0); i < blocks; i++ { + height := lastBlock - i + if blkFee, ok := gs.feeCache.Get(height); ok { + // cache hit + log.T(ctx).Debug("fee cache hit", zap.Uint64("height", height)) + bf := blkFee.(*blockFee) + baseFees[i] = bf.baseFee + gasUsedRatios[i] = bf.gasUsedRatio + blobBaseFees[i] = bf.blobBaseFee + blobGasUsedRatios[i] = bf.blobGasRatio + } else { + // read block fee from dao + log.T(ctx).Debug("fee cache miss", zap.Uint64("height", height)) + blk, err := gs.dao.GetBlockByHeight(height) + if err != nil { + return 0, nil, nil, nil, nil, nil, status.Error(codes.NotFound, err.Error()) + } + if i == 0 { + lastBlk = blk + } + baseFees[i] = blk.BaseFee() + gasUsedRatios[i] = float64(blk.GasUsed()) / float64(g.BlockGasLimitByHeight(blk.Height())) + blobBaseFees[i] = protocol.CalcBlobFee(blk.ExcessBlobGas()) + blobGasUsedRatios[i] = float64(blk.BlobGasUsed()) / float64(params.MaxBlobGasPerBlock) + gs.feeCache.Add(height, &blockFee{ + baseFee: baseFees[i], + gasUsedRatio: gasUsedRatios[i], + blobBaseFee: blobBaseFees[i], + blobGasRatio: blobGasUsedRatios[i], + }) + } + // block priority fee percentiles + if len(rewardPercentiles) > 0 { + if blkPercents, ok := gs.percentileCache.Get(height); ok { + log.T(ctx).Debug("percentile cache hit", zap.Uint64("height", height)) + rewards = append(rewards, feesPercentiles(blkPercents.(*blockPercents).ascEffectivePriorityFees, rewardPercentiles)) + } else { + log.T(ctx).Debug("percentile cache miss", zap.Uint64("height", height)) + receipts, err := gs.dao.GetReceipts(height) + if err != nil { + return 0, nil, nil, nil, nil, nil, status.Error(codes.NotFound, err.Error()) + } + fees := make([]*big.Int, 0, len(receipts)) + for _, r := range receipts { + fees = append(fees, r.PriorityFee()) + } + sort.Slice(fees, func(i, j int) bool { + return fees[i].Cmp(fees[j]) < 0 + }) + rewards = append(rewards, feesPercentiles(fees, rewardPercentiles)) + gs.percentileCache.Add(height, &blockPercents{ + ascEffectivePriorityFees: fees, + }) + } + } + } + // fill next block base fee + if lastBlk == nil { + blk, err := gs.dao.GetBlockByHeight(lastBlock) + if err != nil { + return 0, nil, nil, nil, nil, nil, status.Error(codes.NotFound, err.Error()) + } + lastBlk = blk + } + baseFees[blocks] = protocol.CalcBaseFee(g.Blockchain, &protocol.TipInfo{ + Height: lastBlock, + GasUsed: lastBlk.GasUsed(), + BaseFee: lastBlk.BaseFee(), + }) + blobBaseFees[blocks] = protocol.CalcBlobFee(protocol.CalcExcessBlobGas(lastBlk.ExcessBlobGas(), lastBlk.BlobGasUsed())) + return lastBlock - blocks + 1, rewards, baseFees, gasUsedRatios, blobBaseFees, blobGasUsedRatios, nil +} + +func feesPercentiles(ascFees []*big.Int, percentiles []float64) []*big.Int { + res := make([]*big.Int, len(percentiles)) + for i, p := range percentiles { + idx := int(float64(len(ascFees)) * p) + if idx >= len(ascFees) { + idx = len(ascFees) - 1 + } + res[i] = ascFees[idx] + } + return res +} diff --git a/test/mock/mock_apicoreservice/mock_apicoreservice.go b/test/mock/mock_apicoreservice/mock_apicoreservice.go index 7777a1b9c3..49890a041d 100644 --- a/test/mock/mock_apicoreservice/mock_apicoreservice.go +++ b/test/mock/mock_apicoreservice/mock_apicoreservice.go @@ -372,6 +372,26 @@ func (mr *MockCoreServiceMockRecorder) EstimateMigrateStakeGasConsumption(arg0, return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EstimateMigrateStakeGasConsumption", reflect.TypeOf((*MockCoreService)(nil).EstimateMigrateStakeGasConsumption), arg0, arg1, arg2) } +// FeeHistory mocks base method. +func (m *MockCoreService) FeeHistory(ctx context.Context, blocks, lastBlock uint64, rewardPercentiles []float64) (uint64, [][]*big.Int, []*big.Int, []float64, []*big.Int, []float64, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "FeeHistory", ctx, blocks, lastBlock, rewardPercentiles) + ret0, _ := ret[0].(uint64) + ret1, _ := ret[1].([][]*big.Int) + ret2, _ := ret[2].([]*big.Int) + ret3, _ := ret[3].([]float64) + ret4, _ := ret[4].([]*big.Int) + ret5, _ := ret[5].([]float64) + ret6, _ := ret[6].(error) + return ret0, ret1, ret2, ret3, ret4, ret5, ret6 +} + +// FeeHistory indicates an expected call of FeeHistory. +func (mr *MockCoreServiceMockRecorder) FeeHistory(ctx, blocks, lastBlock, rewardPercentiles interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "FeeHistory", reflect.TypeOf((*MockCoreService)(nil).FeeHistory), ctx, blocks, lastBlock, rewardPercentiles) +} + // Genesis mocks base method. func (m *MockCoreService) Genesis() genesis.Genesis { m.ctrl.T.Helper() From 36a8913529eb8d6901fe7e2fc15611770bc88c01 Mon Sep 17 00:00:00 2001 From: dustinxie Date: Wed, 25 Dec 2024 16:58:38 -0800 Subject: [PATCH 16/21] [blockindex] reduce indexbuilder PutBlocks() size to 100 (#4531) --- blockindex/indexbuilder.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/blockindex/indexbuilder.go b/blockindex/indexbuilder.go index cbdf607bda..2a873516d0 100644 --- a/blockindex/indexbuilder.go +++ b/blockindex/indexbuilder.go @@ -144,8 +144,8 @@ func (ib *IndexBuilder) init(ctx context.Context) error { return err } blks = append(blks, blk) - // commit once every 5000 blocks - if startHeight%5000 == 0 || startHeight == tipHeight { + // commit once every 100 blocks + if startHeight%100 == 0 || startHeight == tipHeight { if err := ib.indexer.PutBlocks(gCtx, blks); err != nil { return err } From 40838e99ae9fc2de5ee8cc09799b2b4659670da3 Mon Sep 17 00:00:00 2001 From: dustinxie Date: Mon, 30 Dec 2024 22:54:19 -0800 Subject: [PATCH 17/21] [staking] fix alias for non-stop node (#4503) --- action/protocol/staking/builder.go | 1 + action/protocol/staking/protocol.go | 24 +++++++++++++++++++----- blockchain/config.go | 3 +++ chainservice/builder.go | 1 + 4 files changed, 24 insertions(+), 5 deletions(-) diff --git a/action/protocol/staking/builder.go b/action/protocol/staking/builder.go index e3a57adc1d..ba213f052c 100644 --- a/action/protocol/staking/builder.go +++ b/action/protocol/staking/builder.go @@ -8,6 +8,7 @@ type ( BuilderConfig struct { Staking genesis.Staking PersistStakingPatchBlock uint64 + FixAliasForNonStopHeight uint64 StakingPatchDir string Revise ReviseConfig } diff --git a/action/protocol/staking/protocol.go b/action/protocol/staking/protocol.go index 0932a79065..9a42173248 100644 --- a/action/protocol/staking/protocol.go +++ b/action/protocol/staking/protocol.go @@ -94,6 +94,7 @@ type ( MinStakeAmount *big.Int BootstrapCandidates []genesis.BootstrapCandidate PersistStakingPatchBlock uint64 + FixAliasForNonStopHeight uint64 EndorsementWithdrawWaitingBlocks uint64 MigrateContractAddress string } @@ -167,6 +168,7 @@ func NewProtocol( MinStakeAmount: minStakeAmount, BootstrapCandidates: cfg.Staking.BootstrapCandidates, PersistStakingPatchBlock: cfg.PersistStakingPatchBlock, + FixAliasForNonStopHeight: cfg.FixAliasForNonStopHeight, EndorsementWithdrawWaitingBlocks: cfg.Staking.EndorsementWithdrawWaitingBlocks, MigrateContractAddress: migrateContractAddress, }, @@ -277,10 +279,12 @@ func (p *Protocol) CreateGenesisStates( // CreatePreStates updates state manager func (p *Protocol) CreatePreStates(ctx context.Context, sm protocol.StateManager) error { - g := genesis.MustExtractGenesisContext(ctx) - blkCtx := protocol.MustGetBlockCtx(ctx) - featureCtx := protocol.MustGetFeatureCtx(ctx) - featureWithHeightCtx := protocol.MustGetFeatureWithHeightCtx(ctx) + var ( + g = genesis.MustExtractGenesisContext(ctx) + blkCtx = protocol.MustGetBlockCtx(ctx) + featureCtx = protocol.MustGetFeatureCtx(ctx) + featureWithHeightCtx = protocol.MustGetFeatureWithHeightCtx(ctx) + ) if blkCtx.BlockHeight == g.GreenlandBlockHeight { csr, err := ConstructBaseView(sm) if err != nil { @@ -290,7 +294,17 @@ func (p *Protocol) CreatePreStates(ctx context.Context, sm protocol.StateManager return err } } - + if blkCtx.BlockHeight == p.config.FixAliasForNonStopHeight { + csm, err := NewCandidateStateManager(sm, featureWithHeightCtx.ReadStateFromDB(blkCtx.BlockHeight)) + if err != nil { + return err + } + base := csm.DirtyView().candCenter.base + owners := base.all() + if err := base.loadNameOperatorMapOwnerList(owners, owners, nil); err != nil { + return err + } + } if p.voteReviser.NeedRevise(blkCtx.BlockHeight) { csm, err := NewCandidateStateManager(sm, featureWithHeightCtx.ReadStateFromDB(blkCtx.BlockHeight)) if err != nil { diff --git a/blockchain/config.go b/blockchain/config.go index 8993e92adc..32336e8fbc 100644 --- a/blockchain/config.go +++ b/blockchain/config.go @@ -74,6 +74,8 @@ type ( StreamingBlockBufferSize uint64 `yaml:"streamingBlockBufferSize"` // PersistStakingPatchBlock is the block to persist staking patch PersistStakingPatchBlock uint64 `yaml:"persistStakingPatchBlock"` + // FixAliasForNonStopHeight is the height to fix candidate alias for a non-stopping node + FixAliasForNonStopHeight uint64 `yaml:"fixAliasForNonStopHeight"` // FactoryDBType is the type of factory db FactoryDBType string `yaml:"factoryDBType"` // MintTimeout is the timeout for minting @@ -120,6 +122,7 @@ var ( WorkingSetCacheSize: 20, StreamingBlockBufferSize: 200, PersistStakingPatchBlock: 19778037, + FixAliasForNonStopHeight: 19778036, FactoryDBType: db.DBBolt, MintTimeout: 1500 * time.Millisecond, // valued with block accept ttl - 500ms(tolerate network delay) } diff --git a/chainservice/builder.go b/chainservice/builder.go index 69664e37f9..19cbeaaf66 100644 --- a/chainservice/builder.go +++ b/chainservice/builder.go @@ -667,6 +667,7 @@ func (builder *Builder) registerStakingProtocol() error { &staking.BuilderConfig{ Staking: builder.cfg.Genesis.Staking, PersistStakingPatchBlock: builder.cfg.Chain.PersistStakingPatchBlock, + FixAliasForNonStopHeight: builder.cfg.Chain.FixAliasForNonStopHeight, StakingPatchDir: builder.cfg.Chain.StakingPatchDir, Revise: staking.ReviseConfig{ VoteWeight: builder.cfg.Genesis.VoteWeightCalConsts, From bf606c0d1ec07c8317e7f8847429409773104c41 Mon Sep 17 00:00:00 2001 From: Chen Chen Date: Wed, 1 Jan 2025 02:02:35 +0800 Subject: [PATCH 18/21] [api] impl eth_blobBaseFee (#4528) --- api/web3server.go | 11 +++++++++++ api/web3server_integrity_test.go | 23 +++++++++++++++++++++++ api/web3server_utils.go | 5 +---- 3 files changed, 35 insertions(+), 4 deletions(-) diff --git a/api/web3server.go b/api/web3server.go index 414e1d009e..df14d98b45 100644 --- a/api/web3server.go +++ b/api/web3server.go @@ -26,6 +26,7 @@ import ( "google.golang.org/grpc/status" "github.com/iotexproject/iotex-core/v2/action" + "github.com/iotexproject/iotex-core/v2/action/protocol" rewardingabi "github.com/iotexproject/iotex-core/v2/action/protocol/rewarding/ethabi" stakingabi "github.com/iotexproject/iotex-core/v2/action/protocol/staking/ethabi" apitypes "github.com/iotexproject/iotex-core/v2/api/types" @@ -170,6 +171,8 @@ func (svr *web3Handler) handleWeb3Req(ctx context.Context, web3Req *gjson.Result res, err = svr.maxPriorityFee() case "eth_feeHistory": res, err = svr.feeHistory(ctx, web3Req) + case "eth_blobBaseFee": + res, err = svr.blobBaseFee() case "eth_getBlockByHash": res, err = svr.getBlockByHash(web3Req) case "eth_chainId": @@ -368,6 +371,14 @@ func (svr *web3Handler) getChainID() (interface{}, error) { return uint64ToHex(uint64(svr.coreService.EVMNetworkID())), nil } +func (svr *web3Handler) blobBaseFee() (interface{}, error) { + blk, err := svr.coreService.BlockByHeight(svr.coreService.TipHeight()) + if err != nil { + return nil, err + } + return bigIntToHex(protocol.CalcBlobFee(protocol.CalcExcessBlobGas(blk.Block.ExcessBlobGas(), blk.Block.BlobGasUsed()))), nil +} + func (svr *web3Handler) getBlockNumber() (interface{}, error) { return uint64ToHex(svr.coreService.TipHeight()), nil } diff --git a/api/web3server_integrity_test.go b/api/web3server_integrity_test.go index 8c3dc89938..65dab79027 100644 --- a/api/web3server_integrity_test.go +++ b/api/web3server_integrity_test.go @@ -161,6 +161,10 @@ func TestWeb3ServerIntegrity(t *testing.T) { t.Run("eth_feeHistory", func(t *testing.T) { feeHistory(t, handler, bc, dao, actPool) }) + + t.Run("eth_blobBaseFee", func(t *testing.T) { + blobBaseFee(t, handler, bc, dao, actPool) + }) } func setupTestServer() (*ServerV2, blockchain.Blockchain, blockdao.BlockDAO, actpool.ActPool, func()) { @@ -849,3 +853,22 @@ func feeHistory(t *testing.T, handler *hTTPHandler, bc blockchain.Blockchain, da }`, oldnest), string(actual)) } } + +func blobBaseFee(t *testing.T, handler *hTTPHandler, bc blockchain.Blockchain, dao blockdao.BlockDAO, actPool actpool.ActPool) { + require := require.New(t) + for _, test := range []struct { + params string + expected int + }{ + {`[]`, 1}, + } { + result := serveTestHTTP(require, handler, "eth_blobBaseFee", test.params) + if test.expected == 0 { + require.Nil(result) + continue + } + actual, ok := result.(string) + require.True(ok) + require.Equal("0x1", actual) + } +} diff --git a/api/web3server_utils.go b/api/web3server_utils.go index 8e1ad917d5..6b07df3c63 100644 --- a/api/web3server_utils.go +++ b/api/web3server_utils.go @@ -60,10 +60,7 @@ func uint64ToHex(val uint64) string { } func bigIntToHex(b *big.Int) string { - if b == nil { - return "0x0" - } - if b.Sign() == 0 { + if b == nil || b.Sign() == 0 { return "0x0" } return "0x" + b.Text(16) From 1c18414877fc37e68d784ca6d86a5c7fae101755 Mon Sep 17 00:00:00 2001 From: dustinxie Date: Tue, 31 Dec 2024 23:33:59 -0800 Subject: [PATCH 19/21] add start history to README (#4535) --- README.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/README.md b/README.md index 38dc637ea3..af73073e29 100644 --- a/README.md +++ b/README.md @@ -119,6 +119,10 @@ ioctl [command] Refer to [CLI document](https://docs.iotex.io/developer/ioctl/install.html) for more details. +## Star History + +[![Star History Chart](https://api.star-history.com/svg?repos=iotexproject/iotex-core&type=Date)](https://star-history.com/#iotexproject/iotex-core&Date) + ## Contact - Mailing list: [iotex-dev](iotex-dev@iotex.io) From 0d7bb4046ec653bce3cd9f627d464032fba621f2 Mon Sep 17 00:00:00 2001 From: dustinxie Date: Mon, 6 Jan 2025 12:12:20 -0800 Subject: [PATCH 20/21] [factory] WorkingSetAtHeight() to take pre-actions (#4530) --- state/factory/factory.go | 21 +++++++++++++++++---- state/factory/statedb.go | 17 ++++++++++++++--- test/mock/mock_factory/mock_factory.go | 13 +++++++++---- 3 files changed, 40 insertions(+), 11 deletions(-) diff --git a/state/factory/factory.go b/state/factory/factory.go index 37cd5a7b62..274cc41b7f 100644 --- a/state/factory/factory.go +++ b/state/factory/factory.go @@ -87,7 +87,7 @@ type ( NewBlockBuilder(context.Context, actpool.ActPool, func(action.Envelope) (*action.SealedEnvelope, error)) (*block.Builder, error) PutBlock(context.Context, *block.Block) error WorkingSet(context.Context) (protocol.StateManager, error) - WorkingSetAtHeight(context.Context, uint64) (protocol.StateManager, error) + WorkingSetAtHeight(context.Context, uint64, ...*action.SealedEnvelope) (protocol.StateManager, error) } // factory implements StateFactory interface, tracks changes to account/contract and batch-commits to DB @@ -408,16 +408,29 @@ func (sf *factory) WorkingSet(ctx context.Context) (protocol.StateManager, error return sf.newWorkingSet(ctx, sf.currentChainHeight+1) } -func (sf *factory) WorkingSetAtHeight(ctx context.Context, height uint64) (protocol.StateManager, error) { +func (sf *factory) WorkingSetAtHeight(ctx context.Context, height uint64, preacts ...*action.SealedEnvelope) (protocol.StateManager, error) { if !sf.saveHistory { return nil, ErrNoArchiveData } sf.mutex.Lock() - defer sf.mutex.Unlock() if height > sf.currentChainHeight { + sf.mutex.Unlock() return nil, errors.Errorf("query height %d is higher than tip height %d", height, sf.currentChainHeight) } - return sf.newWorkingSetAtHeight(ctx, height) + ws, err := sf.newWorkingSetAtHeight(ctx, height) + sf.mutex.Unlock() + if err != nil { + return nil, errors.Wrap(err, "failed to obtain working set from state factory") + } + if len(preacts) == 0 { + return ws, nil + } + // prepare workingset at height, and run acts + ws.height++ + if err := ws.Process(ctx, preacts); err != nil { + return nil, err + } + return ws, nil } // PutBlock persists all changes in RunActions() into the DB diff --git a/state/factory/statedb.go b/state/factory/statedb.go index 638debf7ab..03634eb060 100644 --- a/state/factory/statedb.go +++ b/state/factory/statedb.go @@ -270,9 +270,20 @@ func (sdb *stateDB) WorkingSet(ctx context.Context) (protocol.StateManager, erro return sdb.newWorkingSet(ctx, height+1) } -func (sdb *stateDB) WorkingSetAtHeight(ctx context.Context, height uint64) (protocol.StateManager, error) { - // TODO: implement archive mode - return sdb.newWorkingSet(ctx, height) +func (sdb *stateDB) WorkingSetAtHeight(ctx context.Context, height uint64, preacts ...*action.SealedEnvelope) (protocol.StateManager, error) { + ws, err := sdb.newWorkingSet(ctx, height) + if err != nil { + return nil, errors.Wrap(err, "failed to obtain working set from state db") + } + if len(preacts) == 0 { + return ws, nil + } + // prepare workingset at height, and run acts + ws.height++ + if err := ws.Process(ctx, preacts); err != nil { + return nil, err + } + return ws, nil } // PutBlock persists all changes in RunActions() into the DB diff --git a/test/mock/mock_factory/mock_factory.go b/test/mock/mock_factory/mock_factory.go index 475f831700..c383989d17 100644 --- a/test/mock/mock_factory/mock_factory.go +++ b/test/mock/mock_factory/mock_factory.go @@ -210,16 +210,21 @@ func (mr *MockFactoryMockRecorder) WorkingSet(arg0 interface{}) *gomock.Call { } // WorkingSetAtHeight mocks base method. -func (m *MockFactory) WorkingSetAtHeight(arg0 context.Context, arg1 uint64) (protocol.StateManager, error) { +func (m *MockFactory) WorkingSetAtHeight(arg0 context.Context, arg1 uint64, arg2 ...*action.SealedEnvelope) (protocol.StateManager, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "WorkingSetAtHeight", arg0, arg1) + varargs := []interface{}{arg0, arg1} + for _, a := range arg2 { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "WorkingSetAtHeight", varargs...) ret0, _ := ret[0].(protocol.StateManager) ret1, _ := ret[1].(error) return ret0, ret1 } // WorkingSetAtHeight indicates an expected call of WorkingSetAtHeight. -func (mr *MockFactoryMockRecorder) WorkingSetAtHeight(arg0, arg1 interface{}) *gomock.Call { +func (mr *MockFactoryMockRecorder) WorkingSetAtHeight(arg0, arg1 interface{}, arg2 ...interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WorkingSetAtHeight", reflect.TypeOf((*MockFactory)(nil).WorkingSetAtHeight), arg0, arg1) + varargs := append([]interface{}{arg0, arg1}, arg2...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WorkingSetAtHeight", reflect.TypeOf((*MockFactory)(nil).WorkingSetAtHeight), varargs...) } From d6886f10bef26dfaf93d9f905c9a4dc8ee7737cb Mon Sep 17 00:00:00 2001 From: dustinxie Date: Tue, 7 Jan 2025 09:59:24 -0800 Subject: [PATCH 21/21] [workingset] use processWithCorrectOrder() always (#4534) --- action/protocol/context.go | 2 -- state/factory/workingset.go | 48 ++----------------------------------- 2 files changed, 2 insertions(+), 48 deletions(-) diff --git a/action/protocol/context.go b/action/protocol/context.go index 6b8d3cd350..adbe659314 100644 --- a/action/protocol/context.go +++ b/action/protocol/context.go @@ -146,7 +146,6 @@ type ( SufficentBalanceGuarantee bool EnableCancunEVM bool UnfoldContainerBeforeValidate bool - CorrectValidationOrder bool UnstakedButNotClearSelfStakeAmount bool EnableNewTxTypes bool VerifyNotContainerBeforeRun bool @@ -310,7 +309,6 @@ func WithFeatureCtx(ctx context.Context) context.Context { SufficentBalanceGuarantee: g.IsVanuatu(height), EnableCancunEVM: g.IsVanuatu(height), UnfoldContainerBeforeValidate: g.IsVanuatu(height), - CorrectValidationOrder: g.IsVanuatu(height), UnstakedButNotClearSelfStakeAmount: !g.IsVanuatu(height), EnableNewTxTypes: g.IsVanuatu(height), VerifyNotContainerBeforeRun: g.IsVanuatu(height), diff --git a/state/factory/workingset.go b/state/factory/workingset.go index 890ae6d0f4..3dbbbcc0ba 100644 --- a/state/factory/workingset.go +++ b/state/factory/workingset.go @@ -504,10 +504,7 @@ func (ws *workingSet) checkNonceContinuity(ctx context.Context, accountNonceMap } func (ws *workingSet) Process(ctx context.Context, actions []*action.SealedEnvelope) error { - if protocol.MustGetFeatureCtx(ctx).CorrectValidationOrder { - return ws.processWithCorrectOrder(ctx, actions) - } - return ws.process(ctx, actions) + return ws.processWithCorrectOrder(ctx, actions) } func (ws *workingSet) processWithCorrectOrder(ctx context.Context, actions []*action.SealedEnvelope) error { @@ -565,41 +562,6 @@ func (ws *workingSet) processWithCorrectOrder(ctx context.Context, actions []*ac return ws.finalize() } -func (ws *workingSet) process(ctx context.Context, actions []*action.SealedEnvelope) error { - if err := ws.validate(ctx); err != nil { - return err - } - - reg := protocol.MustGetRegistry(ctx) - for _, act := range actions { - ctxWithActionContext, err := withActionCtx(ctx, act) - if err != nil { - return err - } - for _, p := range reg.All() { - if validator, ok := p.(protocol.ActionValidator); ok { - if err := validator.Validate(ctxWithActionContext, act.Envelope, ws); err != nil { - return err - } - } - } - } - for _, p := range reg.All() { - if pp, ok := p.(protocol.PreStatesCreator); ok { - if err := pp.CreatePreStates(ctx, ws); err != nil { - return err - } - } - } - - receipts, err := ws.runActions(ctx, actions) - if err != nil { - return err - } - ws.receipts = receipts - return ws.finalize() -} - func (ws *workingSet) generateSystemActions(ctx context.Context) ([]action.Envelope, error) { reg := protocol.MustGetRegistry(ctx) postSystemActions := []action.Envelope{} @@ -841,13 +803,7 @@ func (ws *workingSet) ValidateBlock(ctx context.Context, blk *block.Block) error return err } } - var err error - if protocol.MustGetFeatureCtx(ctx).CorrectValidationOrder { - err = ws.processWithCorrectOrder(ctx, blk.RunnableActions().Actions()) - } else { - err = ws.process(ctx, blk.RunnableActions().Actions()) - } - if err != nil { + if err := ws.processWithCorrectOrder(ctx, blk.RunnableActions().Actions()); err != nil { log.L().Error("Failed to update state.", zap.Uint64("height", ws.height), zap.Error(err)) return err }