evm: various fixes (#319)
* evm: use prefix stores, move SetBlockBloom to EndBlock, bug fixes * add logs to genesis state * log tests * commit and finalize on InitGenesis * validate TransactionLogs * changelog * fix test-import * fix lint * add more tests
This commit is contained in:
parent
446eb0a3b7
commit
427e96c1de
@ -56,6 +56,11 @@ Ref: https://keepachangelog.com/en/1.0.0/
|
||||
* (`x/evm`) [\#255](https://github.com/ChainSafe/ethermint/pull/255) Add missing `GenesisState` fields and support `ExportGenesis` functionality.
|
||||
* [\#272](https://github.com/ChainSafe/ethermint/pull/272) Add `Logger` for evm module.
|
||||
* [\#317](https://github.com/ChainSafe/ethermint/pull/317) `GenesisAccount` validation.
|
||||
* (`x/evm`) [\#319](https://github.com/ChainSafe/ethermint/pull/319) Verious evm improvements:
|
||||
* Add transaction `[]*ethtypes.Logs` to evm's `GenesisState` to persist logs after an upgrade.
|
||||
* Remove evm `CodeKey` and `BlockKey`in favor of a prefix `Store`.
|
||||
* Set `BlockBloom` during `EndBlock` instead of `BeginBlock`.
|
||||
* `Commit` state object and `Finalize` storage after `InitGenesis` setup.
|
||||
|
||||
### Features
|
||||
|
||||
@ -73,5 +78,6 @@ Ref: https://keepachangelog.com/en/1.0.0/
|
||||
|
||||
### Bug Fixes
|
||||
|
||||
* (`x/evm`) [\#319](https://github.com/ChainSafe/ethermint/pull/319) Fix `SetBlockHash` that was setting the incorrect height during `BeginBlock`.
|
||||
* (x/evm) [\#176](https://github.com/ChainSafe/ethermint/issues/176) Updated Web3 transaction hash from using RLP hash. Now all transaction hashes exposed are amino hashes.
|
||||
* Removes `Hash()` (RLP) function from `MsgEthereumTx` to avoid confusion or misuse in future.
|
||||
|
2
Makefile
2
Makefile
@ -163,7 +163,7 @@ test-rpc:
|
||||
@${GO_MOD} go test -v --vet=off ./tests/rpc_test
|
||||
|
||||
it-tests:
|
||||
./scripts/integration-test-all.sh -q 1 -z 1 -s 10
|
||||
./scripts/integration-test-all.sh -q 1 -z 1 -s 2
|
||||
|
||||
godocs:
|
||||
@echo "--> Wait a few seconds and visit http://localhost:6060/pkg/github.com/cosmos/ethermint"
|
||||
|
@ -147,10 +147,9 @@ func NewEthermintApp(
|
||||
keys := sdk.NewKVStoreKeys(
|
||||
bam.MainStoreKey, auth.StoreKey, bank.StoreKey, staking.StoreKey,
|
||||
supply.StoreKey, mint.StoreKey, distr.StoreKey, slashing.StoreKey,
|
||||
gov.StoreKey, params.StoreKey, evidence.StoreKey, evm.CodeKey, evm.StoreKey,
|
||||
gov.StoreKey, params.StoreKey, evidence.StoreKey, evm.StoreKey,
|
||||
faucet.StoreKey,
|
||||
)
|
||||
blockKey := sdk.NewKVStoreKey(evm.BlockKey)
|
||||
|
||||
tkeys := sdk.NewTransientStoreKeys(params.TStoreKey)
|
||||
|
||||
@ -203,7 +202,7 @@ func NewEthermintApp(
|
||||
app.subspaces[crisis.ModuleName], invCheckPeriod, app.SupplyKeeper, auth.FeeCollectorName,
|
||||
)
|
||||
app.EvmKeeper = evm.NewKeeper(
|
||||
app.cdc, blockKey, keys[evm.CodeKey], keys[evm.StoreKey], app.AccountKeeper,
|
||||
app.cdc, keys[evm.StoreKey], app.AccountKeeper,
|
||||
app.BankKeeper,
|
||||
)
|
||||
// TODO: use protobuf
|
||||
@ -299,10 +298,6 @@ func NewEthermintApp(
|
||||
app.MountKVStores(keys)
|
||||
app.MountTransientStores(tkeys)
|
||||
|
||||
// Mount block hash mapping key as DB (no need for historical queries)
|
||||
// TODO: why does this need to be always StoreTypeDB?
|
||||
app.MountStore(blockKey, sdk.StoreTypeDB)
|
||||
|
||||
// initialize BaseApp
|
||||
app.SetInitChainer(app.InitChainer)
|
||||
app.SetBeginBlocker(app.BeginBlocker)
|
||||
|
@ -51,7 +51,6 @@ var (
|
||||
|
||||
accKey = sdk.NewKVStoreKey(auth.StoreKey)
|
||||
storeKey = sdk.NewKVStoreKey(evmtypes.StoreKey)
|
||||
codeKey = sdk.NewKVStoreKey(evmtypes.CodeKey)
|
||||
|
||||
logger = tmlog.NewNopLogger()
|
||||
|
||||
@ -107,7 +106,7 @@ func createAndTestGenesis(t *testing.T, cms sdk.CommitMultiStore, ak auth.Accoun
|
||||
ms := cms.CacheMultiStore()
|
||||
ctx := sdk.NewContext(ms, abci.Header{}, false, logger)
|
||||
|
||||
stateDB := evmtypes.NewCommitStateDB(ctx, codeKey, storeKey, ak, bk)
|
||||
stateDB := evmtypes.NewCommitStateDB(ctx, storeKey, ak, bk)
|
||||
|
||||
// sort the addresses and insertion of key/value pairs matters
|
||||
genAddrs := make([]string, len(genBlock.Alloc))
|
||||
@ -189,7 +188,7 @@ func TestImportBlocks(t *testing.T) {
|
||||
bk := bank.NewBaseKeeper(appCodec, bankKey, ak, bankSubspace, nil)
|
||||
|
||||
// mount stores
|
||||
keys := []*sdk.KVStoreKey{accKey, bankKey, storeKey, codeKey}
|
||||
keys := []*sdk.KVStoreKey{accKey, bankKey, storeKey}
|
||||
for _, key := range keys {
|
||||
cms.MountStoreWithDB(key, sdk.StoreTypeIAVL, nil)
|
||||
}
|
||||
@ -280,7 +279,7 @@ func TestImportBlocks(t *testing.T) {
|
||||
|
||||
// nolint: interfacer
|
||||
func createStateDB(ctx sdk.Context, ak auth.AccountKeeper, bk bank.Keeper) *evmtypes.CommitStateDB {
|
||||
return evmtypes.NewCommitStateDB(ctx, codeKey, storeKey, ak, bk)
|
||||
return evmtypes.NewCommitStateDB(ctx, storeKey, ak, bk)
|
||||
}
|
||||
|
||||
// accumulateRewards credits the coinbase of the given block with the mining
|
||||
|
@ -118,7 +118,7 @@ func (e *EthermintBackend) getEthBlockByNumber(height int64, fullTx bool) (map[s
|
||||
}
|
||||
}
|
||||
|
||||
res, _, err := e.cliCtx.Query(fmt.Sprintf("custom/%s/%s/%s", types.ModuleName, evm.QueryLogsBloom, strconv.FormatInt(block.Block.Height, 10)))
|
||||
res, _, err := e.cliCtx.Query(fmt.Sprintf("custom/%s/%s/%s", types.ModuleName, evm.QueryBloom, strconv.FormatInt(block.Block.Height, 10)))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -301,10 +301,12 @@ func TestEth_GetTransactionReceipt(t *testing.T) {
|
||||
|
||||
param := []string{hash.String()}
|
||||
rpcRes := call(t, "eth_getTransactionReceipt", param)
|
||||
require.Nil(t, rpcRes.Error)
|
||||
|
||||
receipt := make(map[string]interface{})
|
||||
err := json.Unmarshal(rpcRes.Result, &receipt)
|
||||
require.NoError(t, err)
|
||||
require.NotEmpty(t, receipt)
|
||||
require.Equal(t, "0x1", receipt["status"].(string))
|
||||
require.Equal(t, []interface{}{}, receipt["logs"].([]interface{}))
|
||||
}
|
||||
|
@ -10,23 +10,23 @@ import (
|
||||
ethtypes "github.com/ethereum/go-ethereum/core/types"
|
||||
)
|
||||
|
||||
// BeginBlock sets the Bloom and Hash mappings and resets the Bloom filter and
|
||||
// BeginBlock sets the block hash -> block height map and resets the Bloom filter and
|
||||
// the transaction count to 0.
|
||||
func BeginBlock(k Keeper, ctx sdk.Context, req abci.RequestBeginBlock) {
|
||||
if req.Header.LastBlockId.GetHash() == nil || req.Header.GetHeight() < 1 {
|
||||
return
|
||||
}
|
||||
|
||||
// Consider removing this when using evm as module without web3 API
|
||||
bloom := ethtypes.BytesToBloom(k.Bloom.Bytes())
|
||||
k.SetBlockBloomMapping(ctx, bloom, req.Header.GetHeight())
|
||||
k.SetBlockHashMapping(ctx, req.Header.LastBlockId.GetHash(), req.Header.GetHeight())
|
||||
k.SetBlockHash(ctx, req.Header.LastBlockId.GetHash(), req.Header.GetHeight()-1)
|
||||
|
||||
// reset counters
|
||||
k.Bloom = big.NewInt(0)
|
||||
k.TxCount = 0
|
||||
}
|
||||
|
||||
// EndBlock updates the accounts and commits states objects to the KV Store
|
||||
func EndBlock(k Keeper, ctx sdk.Context, _ abci.RequestEndBlock) []abci.ValidatorUpdate {
|
||||
// EndBlock updates the accounts and commits states objects to the KV Store.
|
||||
//
|
||||
func EndBlock(k Keeper, ctx sdk.Context, req abci.RequestEndBlock) []abci.ValidatorUpdate {
|
||||
// Gas costs are handled within msg handler so costs should be ignored
|
||||
ctx = ctx.WithBlockGasMeter(sdk.NewInfiniteGasMeter())
|
||||
|
||||
@ -42,5 +42,8 @@ func EndBlock(k Keeper, ctx sdk.Context, _ abci.RequestEndBlock) []abci.Validato
|
||||
// Clear accounts cache after account data has been committed
|
||||
k.CommitStateDB.ClearStateObjects()
|
||||
|
||||
bloom := ethtypes.BytesToBloom(k.Bloom.Bytes())
|
||||
k.SetBlockBloom(ctx, ctx.BlockHeight(), bloom)
|
||||
|
||||
return []abci.ValidatorUpdate{}
|
||||
}
|
||||
|
@ -9,8 +9,6 @@ import (
|
||||
const (
|
||||
ModuleName = types.ModuleName
|
||||
StoreKey = types.StoreKey
|
||||
CodeKey = types.StoreKey
|
||||
BlockKey = types.BlockKey
|
||||
RouterKey = types.RouterKey
|
||||
QueryProtocolVersion = types.QueryProtocolVersion
|
||||
QueryBalance = types.QueryBalance
|
||||
@ -20,7 +18,7 @@ const (
|
||||
QueryNonce = types.QueryNonce
|
||||
QueryHashToHeight = types.QueryHashToHeight
|
||||
QueryTransactionLogs = types.QueryTransactionLogs
|
||||
QueryLogsBloom = types.QueryLogsBloom
|
||||
QueryBloom = types.QueryBloom
|
||||
QueryLogs = types.QueryLogs
|
||||
QueryAccount = types.QueryAccount
|
||||
)
|
||||
|
@ -13,15 +13,33 @@ import (
|
||||
// InitGenesis initializes genesis state based on exported genesis
|
||||
func InitGenesis(ctx sdk.Context, k Keeper, data GenesisState) []abci.ValidatorUpdate {
|
||||
for _, account := range data.Accounts {
|
||||
csdb := k.CommitStateDB.WithContext(ctx)
|
||||
// FIXME: this will override bank InitGenesis balance!
|
||||
csdb.SetBalance(account.Address, account.Balance)
|
||||
csdb.SetCode(account.Address, account.Code)
|
||||
k.SetBalance(ctx, account.Address, account.Balance)
|
||||
k.SetCode(ctx, account.Address, account.Code)
|
||||
for _, storage := range account.Storage {
|
||||
csdb.SetState(account.Address, storage.Key, storage.Value)
|
||||
k.SetState(ctx, account.Address, storage.Key, storage.Value)
|
||||
}
|
||||
}
|
||||
// TODO: Commit?
|
||||
|
||||
var err error
|
||||
for _, txLog := range data.TxsLogs {
|
||||
err = k.SetLogs(ctx, txLog.Hash, txLog.Logs)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
// set state objects and code to store
|
||||
_, err = k.Commit(ctx, false)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
// set storage to store
|
||||
err = k.Finalise(ctx, true)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return []abci.ValidatorUpdate{}
|
||||
}
|
||||
|
||||
@ -59,5 +77,8 @@ func ExportGenesis(ctx sdk.Context, k Keeper, ak types.AccountKeeper) GenesisSta
|
||||
ethGenAccounts = append(ethGenAccounts, genAccount)
|
||||
}
|
||||
|
||||
return GenesisState{Accounts: ethGenAccounts}
|
||||
return GenesisState{
|
||||
Accounts: ethGenAccounts,
|
||||
TxsLogs: k.GetAllTxLogs(ctx),
|
||||
}
|
||||
}
|
||||
|
@ -74,7 +74,10 @@ func handleMsgEthereumTx(ctx sdk.Context, k Keeper, msg types.MsgEthereumTx) (*s
|
||||
k.Bloom.Or(k.Bloom, executionResult.Bloom)
|
||||
|
||||
// update transaction logs in KVStore
|
||||
k.SetTransactionLogs(ctx, txHash, executionResult.Logs)
|
||||
err = k.SetLogs(ctx, common.BytesToHash(txHash), executionResult.Logs)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
// log successful execution
|
||||
k.Logger(ctx).Info(executionResult.Result.Log)
|
||||
@ -147,7 +150,10 @@ func handleMsgEthermint(ctx sdk.Context, k Keeper, msg types.MsgEthermint) (*sdk
|
||||
k.Bloom.Or(k.Bloom, executionResult.Bloom)
|
||||
|
||||
// update transaction logs in KVStore
|
||||
k.SetTransactionLogs(ctx, txHash, executionResult.Logs)
|
||||
err = k.SetLogs(ctx, common.BytesToHash(txHash), executionResult.Logs)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
// log successful execution
|
||||
k.Logger(ctx).Info(executionResult.Result.Log)
|
||||
|
@ -236,9 +236,10 @@ func (suite *EvmTestSuite) TestHandlerLogs() {
|
||||
suite.Require().Equal(len(resultData.Logs[0].Topics), 2)
|
||||
|
||||
hash := []byte{1}
|
||||
suite.app.EvmKeeper.SetTransactionLogs(suite.ctx, hash, resultData.Logs)
|
||||
err = suite.app.EvmKeeper.SetLogs(suite.ctx, ethcmn.BytesToHash(hash), resultData.Logs)
|
||||
suite.Require().NoError(err)
|
||||
|
||||
logs, err := suite.app.EvmKeeper.GetTransactionLogs(suite.ctx, hash)
|
||||
logs, err := suite.app.EvmKeeper.GetLogs(suite.ctx, ethcmn.BytesToHash(hash))
|
||||
suite.Require().NoError(err, "failed to get logs")
|
||||
|
||||
suite.Require().Equal(logs, resultData.Logs)
|
||||
@ -273,7 +274,7 @@ func (suite *EvmTestSuite) TestQueryTxLogs() {
|
||||
// get logs by tx hash
|
||||
hash := resultData.TxHash.Bytes()
|
||||
|
||||
logs, err := suite.app.EvmKeeper.GetTransactionLogs(suite.ctx, hash)
|
||||
logs, err := suite.app.EvmKeeper.GetLogs(suite.ctx, ethcmn.BytesToHash(hash))
|
||||
suite.Require().NoError(err, "failed to get logs")
|
||||
|
||||
suite.Require().Equal(logs, resultData.Logs)
|
||||
|
@ -2,20 +2,19 @@ package keeper
|
||||
|
||||
import (
|
||||
"encoding/binary"
|
||||
"errors"
|
||||
"fmt"
|
||||
"math/big"
|
||||
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
|
||||
"github.com/cosmos/cosmos-sdk/codec"
|
||||
"github.com/cosmos/cosmos-sdk/store/prefix"
|
||||
sdk "github.com/cosmos/cosmos-sdk/types"
|
||||
|
||||
"github.com/cosmos/ethermint/x/evm/types"
|
||||
|
||||
ethcmn "github.com/ethereum/go-ethereum/common"
|
||||
ethtypes "github.com/ethereum/go-ethereum/core/types"
|
||||
|
||||
"math/big"
|
||||
)
|
||||
|
||||
// Keeper wraps the CommitStateDB, allowing us to pass in SDK context while adhering
|
||||
@ -23,9 +22,13 @@ import (
|
||||
type Keeper struct {
|
||||
// Amino codec
|
||||
cdc *codec.Codec
|
||||
// Store key required to update the block bloom filter mappings needed for the
|
||||
// Web3 API
|
||||
blockKey sdk.StoreKey
|
||||
// Store key required for the EVM Prefix KVStore. It is required by:
|
||||
// - storing Account's Storage State
|
||||
// - storing Account's Code
|
||||
// - storing transaction Logs
|
||||
// - storing block height -> bloom filter map. Needed for the Web3 API.
|
||||
// - storing block hash -> block height map. Needed for the Web3 API.
|
||||
storeKey sdk.StoreKey
|
||||
CommitStateDB *types.CommitStateDB
|
||||
// Transaction counter in a block. Used on StateSB's Prepare function.
|
||||
// It is reset to 0 every block on BeginBlock so there's no point in storing the counter
|
||||
@ -36,13 +39,12 @@ type Keeper struct {
|
||||
|
||||
// NewKeeper generates new evm module keeper
|
||||
func NewKeeper(
|
||||
cdc *codec.Codec, blockKey, codeKey, storeKey sdk.StoreKey,
|
||||
ak types.AccountKeeper, bk types.BankKeeper,
|
||||
cdc *codec.Codec, storeKey sdk.StoreKey, ak types.AccountKeeper, bk types.BankKeeper,
|
||||
) Keeper {
|
||||
return Keeper{
|
||||
cdc: cdc,
|
||||
blockKey: blockKey,
|
||||
CommitStateDB: types.NewCommitStateDB(sdk.Context{}, codeKey, storeKey, ak, bk),
|
||||
storeKey: storeKey,
|
||||
CommitStateDB: types.NewCommitStateDB(sdk.Context{}, storeKey, ak, bk),
|
||||
TxCount: 0,
|
||||
Bloom: big.NewInt(0),
|
||||
}
|
||||
@ -55,68 +57,66 @@ func (k Keeper) Logger(ctx sdk.Context) log.Logger {
|
||||
|
||||
// ----------------------------------------------------------------------------
|
||||
// Block hash mapping functions
|
||||
// May be removed when using only as module (only required by rpc api)
|
||||
// Required by Web3 API.
|
||||
// TODO: remove once tendermint support block queries by hash.
|
||||
// ----------------------------------------------------------------------------
|
||||
|
||||
// GetBlockHashMapping gets block height from block consensus hash
|
||||
func (k Keeper) GetBlockHashMapping(ctx sdk.Context, hash []byte) (int64, error) {
|
||||
store := ctx.KVStore(k.blockKey)
|
||||
// GetBlockHash gets block height from block consensus hash
|
||||
func (k Keeper) GetBlockHash(ctx sdk.Context, hash []byte) (int64, bool) {
|
||||
store := prefix.NewStore(ctx.KVStore(k.storeKey), types.KeyPrefixBlockHash)
|
||||
bz := store.Get(hash)
|
||||
if len(bz) == 0 {
|
||||
return 0, fmt.Errorf("block with hash '%s' not found", ethcmn.BytesToHash(hash).Hex())
|
||||
return 0, false
|
||||
}
|
||||
|
||||
height := binary.BigEndian.Uint64(bz)
|
||||
return int64(height), nil
|
||||
return int64(height), true
|
||||
}
|
||||
|
||||
// SetBlockHashMapping sets the mapping from block consensus hash to block height
|
||||
func (k Keeper) SetBlockHashMapping(ctx sdk.Context, hash []byte, height int64) {
|
||||
store := ctx.KVStore(k.blockKey)
|
||||
// SetBlockHash sets the mapping from block consensus hash to block height
|
||||
func (k Keeper) SetBlockHash(ctx sdk.Context, hash []byte, height int64) {
|
||||
store := prefix.NewStore(ctx.KVStore(k.storeKey), types.KeyPrefixBlockHash)
|
||||
bz := sdk.Uint64ToBigEndian(uint64(height))
|
||||
store.Set(hash, bz)
|
||||
}
|
||||
|
||||
// ----------------------------------------------------------------------------
|
||||
// Block bloom bits mapping functions
|
||||
// May be removed when using only as module (only required by rpc api)
|
||||
// Required by Web3 API.
|
||||
// ----------------------------------------------------------------------------
|
||||
|
||||
// GetBlockBloomMapping gets bloombits from block height
|
||||
func (k Keeper) GetBlockBloomMapping(ctx sdk.Context, height int64) (ethtypes.Bloom, error) {
|
||||
store := ctx.KVStore(k.blockKey)
|
||||
heightBz := sdk.Uint64ToBigEndian(uint64(height))
|
||||
bz := store.Get(types.BloomKey(heightBz))
|
||||
// GetBlockBloom gets bloombits from block height
|
||||
func (k Keeper) GetBlockBloom(ctx sdk.Context, height int64) (ethtypes.Bloom, bool) {
|
||||
store := prefix.NewStore(ctx.KVStore(k.storeKey), types.KeyPrefixBloom)
|
||||
bz := store.Get(types.BloomKey(height))
|
||||
if len(bz) == 0 {
|
||||
return ethtypes.Bloom{}, fmt.Errorf("block at height %d not found", height)
|
||||
return ethtypes.Bloom{}, false
|
||||
}
|
||||
|
||||
return ethtypes.BytesToBloom(bz), nil
|
||||
return ethtypes.BytesToBloom(bz), true
|
||||
}
|
||||
|
||||
// SetBlockBloomMapping sets the mapping from block height to bloom bits
|
||||
func (k Keeper) SetBlockBloomMapping(ctx sdk.Context, bloom ethtypes.Bloom, height int64) {
|
||||
store := ctx.KVStore(k.blockKey)
|
||||
heightBz := sdk.Uint64ToBigEndian(uint64(height))
|
||||
store.Set(types.BloomKey(heightBz), bloom.Bytes())
|
||||
// SetBlockBloom sets the mapping from block height to bloom bits
|
||||
func (k Keeper) SetBlockBloom(ctx sdk.Context, height int64, bloom ethtypes.Bloom) {
|
||||
store := prefix.NewStore(ctx.KVStore(k.storeKey), types.KeyPrefixBloom)
|
||||
store.Set(types.BloomKey(height), bloom.Bytes())
|
||||
}
|
||||
|
||||
// SetTransactionLogs sets the transaction's logs in the KVStore
|
||||
func (k *Keeper) SetTransactionLogs(ctx sdk.Context, hash []byte, logs []*ethtypes.Log) {
|
||||
store := ctx.KVStore(k.blockKey)
|
||||
bz := k.cdc.MustMarshalBinaryLengthPrefixed(logs)
|
||||
store.Set(types.LogsKey(hash), bz)
|
||||
}
|
||||
// GetAllTxLogs return all the transaction logs from the store.
|
||||
func (k Keeper) GetAllTxLogs(ctx sdk.Context) []types.TransactionLogs {
|
||||
store := ctx.KVStore(k.storeKey)
|
||||
iterator := sdk.KVStorePrefixIterator(store, types.KeyPrefixLogs)
|
||||
defer iterator.Close()
|
||||
|
||||
// GetTransactionLogs gets the logs for a transaction from the KVStore
|
||||
func (k *Keeper) GetTransactionLogs(ctx sdk.Context, hash []byte) ([]*ethtypes.Log, error) {
|
||||
store := ctx.KVStore(k.blockKey)
|
||||
bz := store.Get(types.LogsKey(hash))
|
||||
if len(bz) == 0 {
|
||||
return nil, errors.New("cannot get transaction logs")
|
||||
txsLogs := []types.TransactionLogs{}
|
||||
for ; iterator.Valid(); iterator.Next() {
|
||||
hash := ethcmn.BytesToHash(iterator.Key())
|
||||
var logs []*ethtypes.Log
|
||||
k.cdc.MustUnmarshalBinaryLengthPrefixed(iterator.Value(), &logs)
|
||||
|
||||
// add a new entry
|
||||
txLog := types.NewTransactionLogs(hash, logs)
|
||||
txsLogs = append(txsLogs, txLog)
|
||||
}
|
||||
|
||||
var logs []*ethtypes.Log
|
||||
k.cdc.MustUnmarshalBinaryLengthPrefixed(bz, &logs)
|
||||
return logs, nil
|
||||
return txsLogs
|
||||
}
|
||||
|
@ -46,22 +46,49 @@ func TestKeeperTestSuite(t *testing.T) {
|
||||
}
|
||||
|
||||
func (suite *KeeperTestSuite) TestTransactionLogs() {
|
||||
ethHash := ethcmn.BytesToHash(hash)
|
||||
log := ðtypes.Log{
|
||||
Address: address,
|
||||
Data: []byte("log"),
|
||||
BlockNumber: 10,
|
||||
}
|
||||
log2 := ðtypes.Log{
|
||||
Address: address,
|
||||
Data: []byte("log2"),
|
||||
BlockNumber: 11,
|
||||
}
|
||||
expLogs := []*ethtypes.Log{log}
|
||||
|
||||
suite.app.EvmKeeper.SetTransactionLogs(suite.ctx, hash, expLogs)
|
||||
suite.app.EvmKeeper.AddLog(suite.ctx, expLogs[0])
|
||||
err := suite.app.EvmKeeper.SetLogs(suite.ctx, ethHash, expLogs)
|
||||
suite.Require().NoError(err)
|
||||
|
||||
logs, err := suite.app.EvmKeeper.GetTransactionLogs(suite.ctx, hash)
|
||||
logs, err := suite.app.EvmKeeper.GetLogs(suite.ctx, ethHash)
|
||||
suite.Require().NoError(err)
|
||||
suite.Require().Equal(expLogs, logs)
|
||||
|
||||
expLogs = []*ethtypes.Log{log2, log}
|
||||
|
||||
// add another log under the zero hash
|
||||
suite.app.EvmKeeper.AddLog(suite.ctx, log2)
|
||||
logs = suite.app.EvmKeeper.AllLogs(suite.ctx)
|
||||
suite.Require().Equal(expLogs, logs)
|
||||
|
||||
// add another log under the zero hash
|
||||
log3 := ðtypes.Log{
|
||||
Address: address,
|
||||
Data: []byte("log3"),
|
||||
BlockNumber: 10,
|
||||
}
|
||||
suite.app.EvmKeeper.AddLog(suite.ctx, log3)
|
||||
|
||||
txLogs := suite.app.EvmKeeper.GetAllTxLogs(suite.ctx)
|
||||
suite.Require().Equal(2, len(txLogs))
|
||||
|
||||
suite.Require().Equal(ethcmn.Hash{}.String(), txLogs[0].Hash.String())
|
||||
suite.Require().Equal([]*ethtypes.Log{log2, log3}, txLogs[0].Logs)
|
||||
|
||||
suite.Require().Equal(ethHash.String(), txLogs[1].Hash.String())
|
||||
suite.Require().Equal([]*ethtypes.Log{log}, txLogs[1].Logs)
|
||||
}
|
||||
|
||||
func (suite *KeeperTestSuite) TestDBStorage() {
|
||||
@ -73,16 +100,16 @@ func (suite *KeeperTestSuite) TestDBStorage() {
|
||||
suite.app.EvmKeeper.SetCode(suite.ctx, address, []byte{0x1})
|
||||
|
||||
// Test block hash mapping functionality
|
||||
suite.app.EvmKeeper.SetBlockHashMapping(suite.ctx, hash, 7)
|
||||
height, err := suite.app.EvmKeeper.GetBlockHashMapping(suite.ctx, hash)
|
||||
suite.Require().NoError(err)
|
||||
suite.app.EvmKeeper.SetBlockHash(suite.ctx, hash, 7)
|
||||
height, found := suite.app.EvmKeeper.GetBlockHash(suite.ctx, hash)
|
||||
suite.Require().True(found)
|
||||
suite.Require().Equal(int64(7), height)
|
||||
|
||||
suite.app.EvmKeeper.SetBlockHashMapping(suite.ctx, []byte{0x43, 0x32}, 8)
|
||||
suite.app.EvmKeeper.SetBlockHash(suite.ctx, []byte{0x43, 0x32}, 8)
|
||||
|
||||
// Test block height mapping functionality
|
||||
testBloom := ethtypes.BytesToBloom([]byte{0x1, 0x3})
|
||||
suite.app.EvmKeeper.SetBlockBloomMapping(suite.ctx, testBloom, 4)
|
||||
suite.app.EvmKeeper.SetBlockBloom(suite.ctx, 4, testBloom)
|
||||
|
||||
// Get those state transitions
|
||||
suite.Require().Equal(suite.app.EvmKeeper.GetBalance(suite.ctx, address).Cmp(big.NewInt(5)), 0)
|
||||
@ -90,19 +117,19 @@ func (suite *KeeperTestSuite) TestDBStorage() {
|
||||
suite.Require().Equal(suite.app.EvmKeeper.GetState(suite.ctx, address, ethcmn.HexToHash("0x2")), ethcmn.HexToHash("0x3"))
|
||||
suite.Require().Equal(suite.app.EvmKeeper.GetCode(suite.ctx, address), []byte{0x1})
|
||||
|
||||
height, err = suite.app.EvmKeeper.GetBlockHashMapping(suite.ctx, hash)
|
||||
suite.Require().NoError(err)
|
||||
height, found = suite.app.EvmKeeper.GetBlockHash(suite.ctx, hash)
|
||||
suite.Require().True(found)
|
||||
suite.Require().Equal(height, int64(7))
|
||||
height, err = suite.app.EvmKeeper.GetBlockHashMapping(suite.ctx, []byte{0x43, 0x32})
|
||||
suite.Require().NoError(err)
|
||||
height, found = suite.app.EvmKeeper.GetBlockHash(suite.ctx, []byte{0x43, 0x32})
|
||||
suite.Require().True(found)
|
||||
suite.Require().Equal(height, int64(8))
|
||||
|
||||
bloom, err := suite.app.EvmKeeper.GetBlockBloomMapping(suite.ctx, 4)
|
||||
suite.Require().NoError(err)
|
||||
bloom, found := suite.app.EvmKeeper.GetBlockBloom(suite.ctx, 4)
|
||||
suite.Require().True(found)
|
||||
suite.Require().Equal(bloom, testBloom)
|
||||
|
||||
// commit stateDB
|
||||
_, err = suite.app.EvmKeeper.Commit(suite.ctx, false)
|
||||
_, err := suite.app.EvmKeeper.Commit(suite.ctx, false)
|
||||
suite.Require().NoError(err, "failed to commit StateDB")
|
||||
|
||||
// simulate BaseApp EndBlocker commitment
|
||||
|
@ -36,8 +36,8 @@ func NewQuerier(keeper Keeper) sdk.Querier {
|
||||
return queryHashToHeight(ctx, path, keeper)
|
||||
case types.QueryTransactionLogs:
|
||||
return queryTransactionLogs(ctx, path, keeper)
|
||||
case types.QueryLogsBloom:
|
||||
return queryBlockLogsBloom(ctx, path, keeper)
|
||||
case types.QueryBloom:
|
||||
return queryBlockBloom(ctx, path, keeper)
|
||||
case types.QueryLogs:
|
||||
return queryLogs(ctx, keeper)
|
||||
case types.QueryAccount:
|
||||
@ -113,9 +113,9 @@ func queryCode(ctx sdk.Context, path []string, keeper Keeper) ([]byte, error) {
|
||||
|
||||
func queryHashToHeight(ctx sdk.Context, path []string, keeper Keeper) ([]byte, error) {
|
||||
blockHash := ethcmn.FromHex(path[1])
|
||||
blockNumber, err := keeper.GetBlockHashMapping(ctx, blockHash)
|
||||
if err != nil {
|
||||
return []byte{}, err
|
||||
blockNumber, found := keeper.GetBlockHash(ctx, blockHash)
|
||||
if !found {
|
||||
return []byte{}, fmt.Errorf("block height not found for hash %s", path[1])
|
||||
}
|
||||
|
||||
res := types.QueryResBlockNumber{Number: blockNumber}
|
||||
@ -127,15 +127,15 @@ func queryHashToHeight(ctx sdk.Context, path []string, keeper Keeper) ([]byte, e
|
||||
return bz, nil
|
||||
}
|
||||
|
||||
func queryBlockLogsBloom(ctx sdk.Context, path []string, keeper Keeper) ([]byte, error) {
|
||||
func queryBlockBloom(ctx sdk.Context, path []string, keeper Keeper) ([]byte, error) {
|
||||
num, err := strconv.ParseInt(path[1], 10, 64)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("could not unmarshal block number: %w", err)
|
||||
return nil, fmt.Errorf("could not unmarshal block height: %w", err)
|
||||
}
|
||||
|
||||
bloom, err := keeper.GetBlockBloomMapping(ctx, num)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to get block bloom mapping: %w", err)
|
||||
bloom, found := keeper.GetBlockBloom(ctx, num)
|
||||
if !found {
|
||||
return nil, fmt.Errorf("block bloom not found for height %d", num)
|
||||
}
|
||||
|
||||
res := types.QueryBloomFilter{Bloom: bloom}
|
||||
|
@ -46,6 +46,11 @@ func (k *Keeper) SetCode(ctx sdk.Context, addr ethcmn.Address, code []byte) {
|
||||
k.CommitStateDB.WithContext(ctx).SetCode(addr, code)
|
||||
}
|
||||
|
||||
// SetLogs calls CommitStateDB.SetLogs using the passed in context
|
||||
func (k *Keeper) SetLogs(ctx sdk.Context, hash ethcmn.Hash, logs []*ethtypes.Log) error {
|
||||
return k.CommitStateDB.WithContext(ctx).SetLogs(hash, logs)
|
||||
}
|
||||
|
||||
// AddLog calls CommitStateDB.AddLog using the passed in context
|
||||
func (k *Keeper) AddLog(ctx sdk.Context, log *ethtypes.Log) {
|
||||
k.CommitStateDB.WithContext(ctx).AddLog(log)
|
||||
@ -149,7 +154,7 @@ func (k *Keeper) StorageTrie(ctx sdk.Context, addr ethcmn.Address) ethstate.Trie
|
||||
// Persistence
|
||||
// ----------------------------------------------------------------------------
|
||||
|
||||
// Commit calls CommitStateDB.Commit using the passed { in context
|
||||
// Commit calls CommitStateDB.Commit using the passed in context
|
||||
func (k *Keeper) Commit(ctx sdk.Context, deleteEmptyObjects bool) (root ethcmn.Hash, err error) {
|
||||
return k.CommitStateDB.WithContext(ctx).Commit(deleteEmptyObjects)
|
||||
}
|
||||
|
@ -11,10 +11,10 @@ import (
|
||||
)
|
||||
|
||||
type (
|
||||
// GenesisState defines the application's genesis state. It contains all the
|
||||
// information required and accounts to initialize the blockchain.
|
||||
// GenesisState defines the evm module genesis state
|
||||
GenesisState struct {
|
||||
Accounts []GenesisAccount `json:"accounts"`
|
||||
Accounts []GenesisAccount `json:"accounts"`
|
||||
TxsLogs []TransactionLogs `json:"txs_logs"`
|
||||
}
|
||||
|
||||
// GenesisStorage represents the GenesisAccount Storage map as single key value
|
||||
@ -76,6 +76,7 @@ func NewGenesisStorage(key, value ethcmn.Hash) GenesisStorage {
|
||||
func DefaultGenesisState() GenesisState {
|
||||
return GenesisState{
|
||||
Accounts: []GenesisAccount{},
|
||||
TxsLogs: []TransactionLogs{},
|
||||
}
|
||||
}
|
||||
|
||||
@ -83,6 +84,7 @@ func DefaultGenesisState() GenesisState {
|
||||
// failure.
|
||||
func (gs GenesisState) Validate() error {
|
||||
seenAccounts := make(map[string]bool)
|
||||
seenTxs := make(map[string]bool)
|
||||
for _, acc := range gs.Accounts {
|
||||
if seenAccounts[acc.Address.String()] {
|
||||
return fmt.Errorf("duplicated genesis account %s", acc.Address.String())
|
||||
@ -92,5 +94,17 @@ func (gs GenesisState) Validate() error {
|
||||
}
|
||||
seenAccounts[acc.Address.String()] = true
|
||||
}
|
||||
for _, tx := range gs.TxsLogs {
|
||||
if seenTxs[tx.Hash.String()] {
|
||||
return fmt.Errorf("duplicated logs from transaction %s", tx.Hash.String())
|
||||
}
|
||||
|
||||
if err := tx.Validate(); err != nil {
|
||||
return fmt.Errorf("invalid logs from transaction %s: %w", tx.Hash.String(), err)
|
||||
}
|
||||
|
||||
seenTxs[tx.Hash.String()] = true
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
@ -204,11 +204,16 @@ func (ch refundChange) dirtied() *ethcmn.Address {
|
||||
}
|
||||
|
||||
func (ch addLogChange) revert(s *CommitStateDB) {
|
||||
logs := s.logs[ch.txhash]
|
||||
logs, err := s.GetLogs(ch.txhash)
|
||||
if err != nil {
|
||||
// panic on unmarshal error
|
||||
panic(err)
|
||||
}
|
||||
|
||||
if len(logs) == 1 {
|
||||
delete(s.logs, ch.txhash)
|
||||
} else {
|
||||
s.logs[ch.txhash] = logs[:len(logs)-1]
|
||||
s.DeleteLogs(ch.txhash)
|
||||
} else if err := s.SetLogs(ch.txhash, logs[:len(logs)-1]); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
s.logSize--
|
||||
|
@ -1,27 +1,38 @@
|
||||
package types
|
||||
|
||||
import (
|
||||
sdk "github.com/cosmos/cosmos-sdk/types"
|
||||
ethcmn "github.com/ethereum/go-ethereum/common"
|
||||
)
|
||||
|
||||
const (
|
||||
// ModuleName string name of module
|
||||
ModuleName = "evm"
|
||||
|
||||
// StoreKey key for ethereum storage data (StateDB)
|
||||
// StoreKey key for ethereum storage data, account code (StateDB) or block
|
||||
// related data for Web3.
|
||||
// The EVM module should use a prefix store.
|
||||
StoreKey = ModuleName
|
||||
// CodeKey key for ethereum code data
|
||||
CodeKey = ModuleName + "code"
|
||||
// BlockKey key
|
||||
BlockKey = ModuleName + "block"
|
||||
|
||||
// RouterKey uses module name for routing
|
||||
RouterKey = ModuleName
|
||||
)
|
||||
|
||||
var bloomPrefix = []byte("bloom")
|
||||
var logsPrefix = []byte("logs")
|
||||
// KVStore key prefixes
|
||||
var (
|
||||
KeyPrefixBlockHash = []byte{0x01}
|
||||
KeyPrefixBloom = []byte{0x02}
|
||||
KeyPrefixLogs = []byte{0x03}
|
||||
KeyPrefixCode = []byte{0x04}
|
||||
KeyPrefixStorage = []byte{0x05}
|
||||
)
|
||||
|
||||
func BloomKey(key []byte) []byte {
|
||||
return append(bloomPrefix, key...)
|
||||
// BloomKey defines the store key for a block Bloom
|
||||
func BloomKey(height int64) []byte {
|
||||
return sdk.Uint64ToBigEndian(uint64(height))
|
||||
}
|
||||
|
||||
func LogsKey(key []byte) []byte {
|
||||
return append(logsPrefix, key...)
|
||||
// AddressStoragePrefix returns a prefix to iterate over a given account storage.
|
||||
func AddressStoragePrefix(address ethcmn.Address) []byte {
|
||||
return append(KeyPrefixStorage, address.Bytes()...)
|
||||
}
|
||||
|
75
x/evm/types/logs.go
Normal file
75
x/evm/types/logs.go
Normal file
@ -0,0 +1,75 @@
|
||||
package types
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"errors"
|
||||
"fmt"
|
||||
|
||||
ethcmn "github.com/ethereum/go-ethereum/common"
|
||||
ethtypes "github.com/ethereum/go-ethereum/core/types"
|
||||
)
|
||||
|
||||
// TransactionLogs define the logs generated from a transaction execution
|
||||
// with a given hash. It it used for import/export data as transactions are not persisted
|
||||
// on blockchain state after an upgrade.
|
||||
type TransactionLogs struct {
|
||||
Hash ethcmn.Hash `json:"hash"`
|
||||
Logs []*ethtypes.Log `json:"logs"`
|
||||
}
|
||||
|
||||
// NewTransactionLogs creates a new NewTransactionLogs instance.
|
||||
func NewTransactionLogs(hash ethcmn.Hash, logs []*ethtypes.Log) TransactionLogs {
|
||||
return TransactionLogs{
|
||||
Hash: hash,
|
||||
Logs: logs,
|
||||
}
|
||||
}
|
||||
|
||||
// MarshalLogs encodes an array of logs using amino
|
||||
func MarshalLogs(logs []*ethtypes.Log) ([]byte, error) {
|
||||
return ModuleCdc.MarshalBinaryLengthPrefixed(logs)
|
||||
}
|
||||
|
||||
// UnmarshalLogs decodes an amino-encoded byte array into an array of logs
|
||||
func UnmarshalLogs(in []byte) ([]*ethtypes.Log, error) {
|
||||
logs := []*ethtypes.Log{}
|
||||
err := ModuleCdc.UnmarshalBinaryLengthPrefixed(in, &logs)
|
||||
return logs, err
|
||||
}
|
||||
|
||||
// Validate performs a basic validation of a GenesisAccount fields.
|
||||
func (tx TransactionLogs) Validate() error {
|
||||
if bytes.Equal(tx.Hash.Bytes(), ethcmn.Hash{}.Bytes()) {
|
||||
return fmt.Errorf("hash cannot be the empty %s", tx.Hash.String())
|
||||
}
|
||||
|
||||
for i, log := range tx.Logs {
|
||||
if err := ValidateLog(log); err != nil {
|
||||
return fmt.Errorf("invalid log %d: %w", i, err)
|
||||
}
|
||||
if bytes.Equal(log.TxHash.Bytes(), tx.Hash.Bytes()) {
|
||||
return fmt.Errorf("log tx hash mismatch (%s ≠ %s)", log.TxHash.String(), tx.Hash.String())
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// ValidateLog performs a basic validation of an ethereum Log fields.
|
||||
func ValidateLog(log *ethtypes.Log) error {
|
||||
if log == nil {
|
||||
return errors.New("log cannot be nil")
|
||||
}
|
||||
if bytes.Equal(log.Address.Bytes(), ethcmn.Address{}.Bytes()) {
|
||||
return fmt.Errorf("log address cannot be empty %s", log.Address.String())
|
||||
}
|
||||
if bytes.Equal(log.BlockHash.Bytes(), ethcmn.Hash{}.Bytes()) {
|
||||
return fmt.Errorf("block hash cannot be the empty %s", log.BlockHash.String())
|
||||
}
|
||||
if log.BlockNumber == 0 {
|
||||
return errors.New("block number cannot be zero")
|
||||
}
|
||||
if bytes.Equal(log.TxHash.Bytes(), ethcmn.Hash{}.Bytes()) {
|
||||
return fmt.Errorf("tx hash cannot be the empty %s", log.TxHash.String())
|
||||
}
|
||||
return nil
|
||||
}
|
@ -16,7 +16,7 @@ const (
|
||||
QueryNonce = "nonce"
|
||||
QueryHashToHeight = "hashToHeight"
|
||||
QueryTransactionLogs = "transactionLogs"
|
||||
QueryLogsBloom = "logsBloom"
|
||||
QueryBloom = "bloom"
|
||||
QueryLogs = "logs"
|
||||
QueryAccount = "account"
|
||||
)
|
||||
|
@ -6,6 +6,7 @@ import (
|
||||
"io"
|
||||
"math/big"
|
||||
|
||||
"github.com/cosmos/cosmos-sdk/store/prefix"
|
||||
sdk "github.com/cosmos/cosmos-sdk/types"
|
||||
authexported "github.com/cosmos/cosmos-sdk/x/auth/exported"
|
||||
|
||||
@ -220,7 +221,7 @@ func (so *stateObject) markSuicided() {
|
||||
// commitState commits all dirty storage to a KVStore.
|
||||
func (so *stateObject) commitState() {
|
||||
ctx := so.stateDB.ctx
|
||||
store := ctx.KVStore(so.stateDB.storeKey)
|
||||
store := prefix.NewStore(ctx.KVStore(so.stateDB.storeKey), KeyPrefixStorage)
|
||||
|
||||
for key, value := range so.dirtyStorage {
|
||||
delete(so.dirtyStorage, key)
|
||||
@ -240,14 +241,12 @@ func (so *stateObject) commitState() {
|
||||
|
||||
store.Set(key.Bytes(), value.Bytes())
|
||||
}
|
||||
|
||||
// TODO: Set the account (storage) root (but we probably don't need this)
|
||||
}
|
||||
|
||||
// commitCode persists the state object's code to the KVStore.
|
||||
func (so *stateObject) commitCode() {
|
||||
ctx := so.stateDB.ctx
|
||||
store := ctx.KVStore(so.stateDB.codeKey)
|
||||
store := prefix.NewStore(ctx.KVStore(so.stateDB.storeKey), KeyPrefixCode)
|
||||
store.Set(so.CodeHash(), so.code)
|
||||
}
|
||||
|
||||
@ -296,7 +295,7 @@ func (so *stateObject) Code(_ ethstate.Database) []byte {
|
||||
}
|
||||
|
||||
ctx := so.stateDB.ctx
|
||||
store := ctx.KVStore(so.stateDB.codeKey)
|
||||
store := prefix.NewStore(ctx.KVStore(so.stateDB.storeKey), KeyPrefixCode)
|
||||
code := store.Get(so.CodeHash())
|
||||
|
||||
if len(code) == 0 {
|
||||
@ -334,7 +333,7 @@ func (so *stateObject) GetCommittedState(_ ethstate.Database, key ethcmn.Hash) e
|
||||
|
||||
// otherwise load the value from the KVStore
|
||||
ctx := so.stateDB.ctx
|
||||
store := ctx.KVStore(so.stateDB.storeKey)
|
||||
store := prefix.NewStore(ctx.KVStore(so.stateDB.storeKey), KeyPrefixStorage)
|
||||
rawValue := store.Get(prefixKey.Bytes())
|
||||
|
||||
if len(rawValue) > 0 {
|
||||
|
@ -6,6 +6,7 @@ import (
|
||||
"sort"
|
||||
"sync"
|
||||
|
||||
"github.com/cosmos/cosmos-sdk/store/prefix"
|
||||
sdk "github.com/cosmos/cosmos-sdk/types"
|
||||
|
||||
emint "github.com/cosmos/ethermint/types"
|
||||
@ -40,8 +41,7 @@ type CommitStateDB struct {
|
||||
// StateDB interface. Perhaps there is a better way.
|
||||
ctx sdk.Context
|
||||
|
||||
codeKey sdk.StoreKey
|
||||
storeKey sdk.StoreKey // i.e storage key
|
||||
storeKey sdk.StoreKey
|
||||
accountKeeper AccountKeeper
|
||||
bankKeeper BankKeeper
|
||||
|
||||
@ -55,7 +55,6 @@ type CommitStateDB struct {
|
||||
|
||||
thash, bhash ethcmn.Hash
|
||||
txIndex int
|
||||
logs map[ethcmn.Hash][]*ethtypes.Log
|
||||
logSize uint
|
||||
|
||||
// TODO: Determine if we actually need this as we do not need preimages in
|
||||
@ -85,17 +84,15 @@ type CommitStateDB struct {
|
||||
// CONTRACT: Stores used for state must be cache-wrapped as the ordering of the
|
||||
// key/value space matters in determining the merkle root.
|
||||
func NewCommitStateDB(
|
||||
ctx sdk.Context, codeKey, storeKey sdk.StoreKey, ak AccountKeeper, bk BankKeeper,
|
||||
ctx sdk.Context, storeKey sdk.StoreKey, ak AccountKeeper, bk BankKeeper,
|
||||
) *CommitStateDB {
|
||||
return &CommitStateDB{
|
||||
ctx: ctx,
|
||||
codeKey: codeKey,
|
||||
storeKey: storeKey,
|
||||
accountKeeper: ak,
|
||||
bankKeeper: bk,
|
||||
stateObjects: make(map[ethcmn.Address]*stateObject),
|
||||
stateObjectsDirty: make(map[ethcmn.Address]struct{}),
|
||||
logs: make(map[ethcmn.Hash][]*ethtypes.Log),
|
||||
preimages: make(map[ethcmn.Hash][]byte),
|
||||
journal: newJournal(),
|
||||
}
|
||||
@ -159,22 +156,32 @@ func (csdb *CommitStateDB) SetCode(addr ethcmn.Address, code []byte) {
|
||||
}
|
||||
}
|
||||
|
||||
// ----------------------------------------------------------------------------
|
||||
// Transaction logs
|
||||
// Required for upgrade logic or ease of querying.
|
||||
// NOTE: we use BinaryLengthPrefixed since the tx logs are also included on Result data,
|
||||
// which can't use BinaryBare.
|
||||
// ----------------------------------------------------------------------------
|
||||
|
||||
// SetLogs sets the logs for a transaction in the KVStore.
|
||||
func (csdb *CommitStateDB) SetLogs(hash ethcmn.Hash, logs []*ethtypes.Log) error {
|
||||
store := csdb.ctx.KVStore(csdb.storeKey)
|
||||
enc, err := EncodeLogs(logs)
|
||||
store := prefix.NewStore(csdb.ctx.KVStore(csdb.storeKey), KeyPrefixLogs)
|
||||
bz, err := MarshalLogs(logs)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if len(enc) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
store.Set(LogsKey(hash[:]), enc)
|
||||
store.Set(hash.Bytes(), bz)
|
||||
csdb.logSize = uint(len(logs))
|
||||
return nil
|
||||
}
|
||||
|
||||
// DeleteLogs removes the logs from the KVStore. It is used during journal.Revert.
|
||||
func (csdb *CommitStateDB) DeleteLogs(hash ethcmn.Hash) {
|
||||
store := prefix.NewStore(csdb.ctx.KVStore(csdb.storeKey), KeyPrefixLogs)
|
||||
store.Delete(hash.Bytes())
|
||||
}
|
||||
|
||||
// AddLog adds a new log to the state and sets the log metadata from the state.
|
||||
func (csdb *CommitStateDB) AddLog(log *ethtypes.Log) {
|
||||
csdb.journal.append(addLogChange{txhash: csdb.thash})
|
||||
@ -183,8 +190,17 @@ func (csdb *CommitStateDB) AddLog(log *ethtypes.Log) {
|
||||
log.BlockHash = csdb.bhash
|
||||
log.TxIndex = uint(csdb.txIndex)
|
||||
log.Index = csdb.logSize
|
||||
csdb.logs[csdb.thash] = append(csdb.logs[csdb.thash], log)
|
||||
csdb.logSize++
|
||||
|
||||
logs, err := csdb.GetLogs(csdb.thash)
|
||||
if err != nil {
|
||||
// panic on unmarshal error
|
||||
panic(err)
|
||||
}
|
||||
|
||||
if err = csdb.SetLogs(csdb.thash, append(logs, log)); err != nil {
|
||||
// panic on marshal error
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
// AddPreimage records a SHA3 preimage seen by the VM.
|
||||
@ -308,30 +324,30 @@ func (csdb *CommitStateDB) GetCommittedState(addr ethcmn.Address, hash ethcmn.Ha
|
||||
|
||||
// GetLogs returns the current logs for a given transaction hash from the KVStore.
|
||||
func (csdb *CommitStateDB) GetLogs(hash ethcmn.Hash) ([]*ethtypes.Log, error) {
|
||||
if csdb.logs[hash] != nil {
|
||||
return csdb.logs[hash], nil
|
||||
}
|
||||
|
||||
store := csdb.ctx.KVStore(csdb.storeKey)
|
||||
|
||||
encLogs := store.Get(LogsKey(hash[:]))
|
||||
if len(encLogs) == 0 {
|
||||
// return nil if logs are not found
|
||||
store := prefix.NewStore(csdb.ctx.KVStore(csdb.storeKey), KeyPrefixLogs)
|
||||
bz := store.Get(hash.Bytes())
|
||||
if len(bz) == 0 {
|
||||
// return nil error if logs are not found
|
||||
return []*ethtypes.Log{}, nil
|
||||
}
|
||||
|
||||
return DecodeLogs(encLogs)
|
||||
return UnmarshalLogs(bz)
|
||||
}
|
||||
|
||||
// AllLogs returns all the current logs in the state.
|
||||
func (csdb *CommitStateDB) AllLogs() []*ethtypes.Log {
|
||||
// nolint: prealloc
|
||||
var logs []*ethtypes.Log
|
||||
for _, lgs := range csdb.logs {
|
||||
logs = append(logs, lgs...)
|
||||
store := csdb.ctx.KVStore(csdb.storeKey)
|
||||
iterator := sdk.KVStorePrefixIterator(store, KeyPrefixLogs)
|
||||
defer iterator.Close()
|
||||
|
||||
allLogs := []*ethtypes.Log{}
|
||||
for ; iterator.Valid(); iterator.Next() {
|
||||
var logs []*ethtypes.Log
|
||||
ModuleCdc.MustUnmarshalBinaryLengthPrefixed(iterator.Value(), &logs)
|
||||
allLogs = append(allLogs, logs...)
|
||||
}
|
||||
|
||||
return logs
|
||||
return allLogs
|
||||
}
|
||||
|
||||
// GetRefund returns the current value of the refund counter.
|
||||
@ -576,7 +592,6 @@ func (csdb *CommitStateDB) Reset(_ ethcmn.Hash) error {
|
||||
csdb.thash = ethcmn.Hash{}
|
||||
csdb.bhash = ethcmn.Hash{}
|
||||
csdb.txIndex = 0
|
||||
csdb.logs = make(map[ethcmn.Hash][]*ethtypes.Log)
|
||||
csdb.logSize = 0
|
||||
csdb.preimages = make(map[ethcmn.Hash][]byte)
|
||||
|
||||
@ -651,14 +666,12 @@ func (csdb *CommitStateDB) Copy() *CommitStateDB {
|
||||
// copy all the basic fields, initialize the memory ones
|
||||
state := &CommitStateDB{
|
||||
ctx: csdb.ctx,
|
||||
codeKey: csdb.codeKey,
|
||||
storeKey: csdb.storeKey,
|
||||
accountKeeper: csdb.accountKeeper,
|
||||
bankKeeper: csdb.bankKeeper,
|
||||
stateObjects: make(map[ethcmn.Address]*stateObject, len(csdb.journal.dirties)),
|
||||
stateObjectsDirty: make(map[ethcmn.Address]struct{}, len(csdb.journal.dirties)),
|
||||
refund: csdb.refund,
|
||||
logs: make(map[ethcmn.Hash][]*ethtypes.Log, len(csdb.logs)),
|
||||
logSize: csdb.logSize,
|
||||
preimages: make(map[ethcmn.Hash][]byte),
|
||||
journal: newJournal(),
|
||||
@ -687,16 +700,6 @@ func (csdb *CommitStateDB) Copy() *CommitStateDB {
|
||||
}
|
||||
}
|
||||
|
||||
// copy logs
|
||||
for hash, logs := range csdb.logs {
|
||||
cpy := make([]*ethtypes.Log, len(logs))
|
||||
for i, l := range logs {
|
||||
cpy[i] = new(ethtypes.Log)
|
||||
*cpy[i] = *l
|
||||
}
|
||||
state.logs[hash] = cpy
|
||||
}
|
||||
|
||||
// copy pre-images
|
||||
for hash, preimage := range csdb.preimages {
|
||||
state.preimages[hash] = preimage
|
||||
@ -714,12 +717,12 @@ func (csdb *CommitStateDB) ForEachStorage(addr ethcmn.Address, cb func(key, valu
|
||||
}
|
||||
|
||||
store := csdb.ctx.KVStore(csdb.storeKey)
|
||||
iter := sdk.KVStorePrefixIterator(store, so.Address().Bytes())
|
||||
defer iter.Close()
|
||||
iterator := sdk.KVStorePrefixIterator(store, AddressStoragePrefix(so.Address()))
|
||||
defer iterator.Close()
|
||||
|
||||
for ; iter.Valid(); iter.Next() {
|
||||
key := ethcmn.BytesToHash(iter.Key())
|
||||
value := iter.Value()
|
||||
for ; iterator.Valid(); iterator.Next() {
|
||||
key := ethcmn.BytesToHash(iterator.Key())
|
||||
value := ethcmn.BytesToHash(iterator.Value())
|
||||
|
||||
if value, dirty := so.dirtyStorage[key]; dirty {
|
||||
// check if iteration stops
|
||||
@ -731,7 +734,7 @@ func (csdb *CommitStateDB) ForEachStorage(addr ethcmn.Address, cb func(key, valu
|
||||
}
|
||||
|
||||
// check if iteration stops
|
||||
if cb(key, ethcmn.BytesToHash(value)) {
|
||||
if cb(key, value) {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
@ -88,21 +88,6 @@ func DecodeResultData(in []byte) (ResultData, error) {
|
||||
return data, nil
|
||||
}
|
||||
|
||||
// EncodeLogs encodes an array of logs using amino
|
||||
func EncodeLogs(logs []*ethtypes.Log) ([]byte, error) {
|
||||
return ModuleCdc.MarshalBinaryLengthPrefixed(logs)
|
||||
}
|
||||
|
||||
// DecodeLogs decodes an amino-encoded byte array into an array of logs
|
||||
func DecodeLogs(in []byte) ([]*ethtypes.Log, error) {
|
||||
logs := []*ethtypes.Log{}
|
||||
err := ModuleCdc.UnmarshalBinaryLengthPrefixed(in, &logs)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return logs, nil
|
||||
}
|
||||
|
||||
// ----------------------------------------------------------------------------
|
||||
// Auxiliary
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user