Skip to content

Commit

Permalink
implement get proof rpc endpoint (#13291)
Browse files Browse the repository at this point in the history
#13201

---------

Co-authored-by: shota.silagadze <[email protected]>
Co-authored-by: antonis19 <[email protected]>
  • Loading branch information
3 people authored Jan 24, 2025
1 parent 907c9ae commit 654e24f
Show file tree
Hide file tree
Showing 4 changed files with 186 additions and 99 deletions.
14 changes: 14 additions & 0 deletions erigon-lib/state/domain_shared.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"unsafe"

"github.com/erigontech/erigon-lib/seg"
"github.com/erigontech/erigon-lib/trie"
"github.com/pkg/errors"
"golang.org/x/crypto/sha3"

Expand Down Expand Up @@ -162,6 +163,10 @@ func (sd *SharedDomains) SavePastChangesetAccumulator(blockHash common.Hash, blo
sd.pastChangesAccumulator[toStringZeroCopy(key)] = acc
}

func (sd *SharedDomains) GetCommitmentContext() *SharedDomainsCommitmentContext {
return sd.sdCtx
}

func (sd *SharedDomains) GetDiffset(tx kv.RwTx, blockHash common.Hash, blockNumber uint64) ([kv.DomainLen][]DomainEntryDiff, bool, error) {
var key [40]byte
binary.BigEndian.PutUint64(key[:8], blockNumber)
Expand Down Expand Up @@ -1310,6 +1315,15 @@ func (sdc *SharedDomainsCommitmentContext) TouchKey(d kv.Domain, key string, val
}
}

func (sdc *SharedDomainsCommitmentContext) Witness(ctx context.Context, expectedRoot []byte, logPrefix string) (proofTrie *trie.Trie, rootHash []byte, err error) {
hexPatriciaHashed, ok := sdc.Trie().(*commitment.HexPatriciaHashed)
if ok {
return hexPatriciaHashed.GenerateWitness(ctx, sdc.updates, nil, expectedRoot, logPrefix)
}

return nil, nil, errors.New("shared domains commitment context doesn't have HexPatriciaHashed")
}

// Evaluates commitment for processed state.
func (sdc *SharedDomainsCommitmentContext) ComputeCommitment(ctx context.Context, saveState bool, blockNum uint64, logPrefix string) (rootHash []byte, err error) {
sdc.ResetBranchCache()
Expand Down
26 changes: 16 additions & 10 deletions erigon-lib/trie/proof.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ func (t *Trie) Prove(key []byte, fromLevel int, storage bool) ([][]byte, error)
key = key[len(nKey):]
}
if fromLevel > 0 {
fromLevel -= len(nKey)
fromLevel--
}
case *DuoNode:
if fromLevel == 0 {
Expand Down Expand Up @@ -350,15 +350,21 @@ func VerifyStorageProofByHash(storageRoot libcommon.Hash, keyHash libcommon.Hash
if proof.Value.ToInt().Sign() != 0 {
return errors.New("empty storage root cannot have non-zero values")
}
// The spec here is a bit unclear. The yellow paper makes it clear that the
// EmptyRoot hash is a special case where the trie is empty. Since the trie
// is empty there are no proof elements to collect. But, EIP-1186 also
// clearly states that the proof must be "starting with the
// storageHash-Node", which could imply an RLP encoded `[]byte(nil)` (the
// pre-image of the EmptyRoot) should be included. This implementation
// chooses to require the proof be empty.
if len(proof.Proof) > 0 {
return errors.New("empty storage root should not have proof nodes")
// if storage root is zero (0000000) then we should have an empty proof
// if it corresponds to empty storage tree, having value EmptyRoot above
// then proof should be RLP encoding of empty proof (0x80)
if storageRoot == EmptyRoot {
for i, _ := range proof.Proof {
if len(proof.Proof[i]) != 1 || proof.Proof[i][0] != 0x80 {
return errors.New("empty storage root should have RLP encoding of empty proof")
}
}
} else {
for i, _ := range proof.Proof {
if len(proof.Proof[i]) != 0 {
return errors.New("zero storage root should have empty proof")
}
}
}
return nil
}
Expand Down
216 changes: 143 additions & 73 deletions turbo/jsonrpc/eth_call.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,13 @@ import (
"errors"
"fmt"
"math/big"
"unsafe"

"github.com/erigontech/erigon-lib/kv/dbutils"
"github.com/erigontech/erigon-lib/trie"

"github.com/erigontech/erigon-lib/common"
libstate "github.com/erigontech/erigon-lib/state"
"github.com/holiman/uint256"
"google.golang.org/grpc"

Expand All @@ -34,12 +40,9 @@ import (
"github.com/erigontech/erigon-lib/gointerfaces"
txpool_proto "github.com/erigontech/erigon-lib/gointerfaces/txpoolproto"
"github.com/erigontech/erigon-lib/kv"
"github.com/erigontech/erigon-lib/kv/dbutils"
"github.com/erigontech/erigon-lib/kv/membatchwithdb"
"github.com/erigontech/erigon-lib/kv/rawdbv3"
"github.com/erigontech/erigon-lib/log/v3"
libstate "github.com/erigontech/erigon-lib/state"
"github.com/erigontech/erigon-lib/trie"
"github.com/erigontech/erigon-lib/types/accounts"
"github.com/erigontech/erigon/consensus"
"github.com/erigontech/erigon/core"
Expand Down Expand Up @@ -332,100 +335,167 @@ func (api *APIImpl) EstimateGas(ctx context.Context, argsOrNil *ethapi2.CallArgs
return hexutil.Uint64(hi), nil
}

// maxGetProofRewindBlockCount limits the number of blocks into the past that
// GetProof will allow computing proofs. Because we must rewind the hash state
// and re-compute the state trie, the further back in time the request, the more
// computationally intensive the operation becomes. The staged sync code
// assumes that if more than 100_000 blocks are skipped, that the entire trie
// should be re-computed. Re-computing the entire trie will currently take ~15
// minutes on mainnet. The current limit has been chosen arbitrarily as
// 'useful' without likely being overly computationally intense.

// GetProof is partially implemented; no Storage proofs, and proofs must be for
// blocks within maxGetProofRewindBlockCount blocks of the head.
// GetProof is partially implemented; Proofs are available only with the `latest` block tag.
func (api *APIImpl) GetProof(ctx context.Context, address libcommon.Address, storageKeys []libcommon.Hash, blockNrOrHash rpc.BlockNumberOrHash) (*accounts.AccProofResult, error) {
return nil, errors.New("not supported by Erigon3")
/*
tx, err := api.db.BeginTemporalRo(ctx)
if err != nil {
return nil, err
}
defer tx.Rollback()
roTx, err := api.db.BeginRo(ctx)
if err != nil {
return nil, err
}
defer roTx.Rollback()

blockNr, _, _, err := rpchelper.GetBlockNumber(blockNrOrHash, tx, api.filters)
if err != nil {
return nil, err
}
requestedBlockNr, _, _, err := rpchelper.GetCanonicalBlockNumber(ctx, blockNrOrHash, roTx, api._blockReader, api.filters)
if err != nil {
return nil, err
} else if requestedBlockNr == 0 {
return nil, errors.New("block not found")
}

header, err := api._blockReader.HeaderByNumber(ctx, tx, blockNr)
if err != nil {
return nil, err
}
latestBlock, err := rpchelper.GetLatestBlockNumber(roTx)
if err != nil {
return nil, err
}

latestBlock, err := rpchelper.GetLatestBlockNumber(tx)
if err != nil {
return nil, err
}
if requestedBlockNr != latestBlock {
return nil, errors.New("proofs are available only for the 'latest' block")
}

if latestBlock < blockNr {
// shouldn't happen, but check anyway
return nil, fmt.Errorf("block number is in the future latest=%d requested=%d", latestBlock, blockNr)
}
return api.getProof(ctx, &roTx, address, storageKeys, rpc.BlockNumberOrHashWithNumber(rpc.BlockNumber(latestBlock)), api.db, api.logger)
}

rl := trie.NewRetainList(0)
var loader *trie.FlatDBTrieLoader
if blockNr < latestBlock {
if latestBlock-blockNr > uint64(api.MaxGetProofRewindBlockCount) {
return nil, fmt.Errorf("requested block is too old, block must be within %d blocks of the head block number (currently %d)", uint64(api.MaxGetProofRewindBlockCount), latestBlock)
}
batch := membatchwithdb.NewMemoryBatch(tx, api.dirs.Tmp, api.logger)
defer batch.Rollback()
func (api *APIImpl) getProof(ctx context.Context, roTx *kv.Tx, address libcommon.Address, storageKeys []libcommon.Hash, blockNrOrHash rpc.BlockNumberOrHash, db kv.RoDB, logger log.Logger) (*accounts.AccProofResult, error) {
// get the root hash from header to validate proofs along the way
header, err := api._blockReader.HeaderByNumber(ctx, *roTx, blockNrOrHash.BlockNumber.Uint64())
if err != nil {
return nil, err
}

unwindState := &stagedsync.UnwindState{UnwindPoint: blockNr}
stageState := &stagedsync.StageState{BlockNumber: latestBlock}
domains, err := libstate.NewSharedDomains(*roTx, log.New())
if err != nil {
return nil, err
}
sdCtx := domains.GetCommitmentContext()

hashStageCfg := stagedsync.StageHashStateCfg(nil, api.dirs, api.historyV3(batch))
if err := stagedsync.UnwindHashStateStage(unwindState, stageState, batch, hashStageCfg, ctx, api.logger); err != nil {
return nil, err
}
// touch account
sdCtx.TouchKey(kv.AccountsDomain, string(address.Bytes()), nil)

interHashStageCfg := stagedsync.StageTrieCfg(nil, false, false, false, api.dirs.Tmp, api._blockReader, nil, api.historyV3(batch), api._agg)
loader, err = stagedsync.UnwindIntermediateHashesForTrieLoader("eth_getProof", rl, unwindState, stageState, batch, interHashStageCfg, nil, nil, ctx.Done(), api.logger)
if err != nil {
return nil, err
// generate the trie for proofs, this works by loading the merkle paths to the touched keys
proofTrie, _, err := sdCtx.Witness(ctx, header.Root[:], "eth_getProof")
if err != nil {
return nil, err
}

// set initial response fields
proof := &accounts.AccProofResult{
Address: address,
Balance: new(hexutil.Big),
Nonce: hexutil.Uint64(0),
CodeHash: libcommon.Hash{},
StorageHash: libcommon.Hash{},
StorageProof: make([]accounts.StorProofResult, len(storageKeys)),
}

// get account proof
accountProof, err := proofTrie.Prove(crypto.Keccak256(address.Bytes()), 0, false)
if err != nil {
return nil, err
}
proof.AccountProof = *(*[]hexutility.Bytes)(unsafe.Pointer(&accountProof))

// get account data from the trie
acc, _ := proofTrie.GetAccount(crypto.Keccak256(address.Bytes()))
if acc == nil {
for i, k := range storageKeys {
proof.StorageProof[i] = accounts.StorProofResult{
Key: k,
Value: new(hexutil.Big),
Proof: nil,
}
tx = batch
} else {
loader = trie.NewFlatDBTrieLoader("eth_getProof", rl, nil, nil, false)
}
return proof, nil
}

reader, err := rpchelper.CreateStateReader(ctx, tx, blockNrOrHash, 0, api.filters, api.stateCache, "")
if err != nil {
return nil, err
proof.Balance = (*hexutil.Big)(acc.Balance.ToBig())
proof.Nonce = hexutil.Uint64(acc.Nonce)
proof.CodeHash = acc.CodeHash
proof.StorageHash = acc.Root

// if storage is not empty touch keys and build trie
if proof.StorageHash.Cmp(libcommon.BytesToHash(commitment.EmptyRootHash)) != 0 && len(storageKeys) != 0 {
// touch storage keys
for _, storageKey := range storageKeys {
sdCtx.TouchKey(kv.StorageDomain, string(common.FromHex(address.Hex()[2:]+storageKey.String()[2:])), nil)
}
a, err := reader.ReadAccountData(address)

// generate the trie for proofs, this works by loading the merkle paths to the touched keys
proofTrie, _, err = sdCtx.Witness(ctx, header.Root[:], "eth_getProof")
if err != nil {
return nil, err
}
if a == nil {
a = &accounts.Account{}
}

tx, err := api.db.BeginTemporalRo(ctx)
if err != nil {
return nil, err
}
defer tx.Rollback()
reader, err := rpchelper.CreateStateReader(ctx, tx, api._blockReader, blockNrOrHash, 0, api.filters, api.stateCache, "")
if err != nil {
return nil, err
}

// get storage key proofs
for i, keyHash := range storageKeys {
proof.StorageProof[i].Key = keyHash

// if we have simple non contract account just set values directly without requesting any key proof
if proof.StorageHash.Cmp(libcommon.BytesToHash(commitment.EmptyRootHash)) == 0 {
proof.StorageProof[i].Proof = nil
proof.StorageProof[i].Value = new(hexutil.Big)
continue
}
pr, err := trie.NewProofRetainer(address, a, storageKeys, rl)

// prepare key path (keccak(address) | keccak(key))
var fullKey []byte
fullKey = append(fullKey, crypto.Keccak256(address.Bytes())...)
fullKey = append(fullKey, crypto.Keccak256(keyHash.Bytes())...)

// get proof for the given key
storageProof, err := proofTrie.Prove(fullKey, len(proof.AccountProof), true)
if err != nil {
return nil, err
return nil, errors.New("cannot verify store proof")
}

loader.SetProofRetainer(pr)
root, err := loader.CalcTrieRoot(tx, nil)
res, err := reader.ReadAccountStorage(address, acc.Incarnation, &keyHash)
if err != nil {
return nil, err
res = []byte{}
logger.Warn(fmt.Sprintf("couldn't read account storage for the address %s\n", address.String()))
}
n := new(big.Int)
n.SetBytes(res)
proof.StorageProof[i].Value = (*hexutil.Big)(n)

// 0x80 represents RLP encoding of an empty proof slice
proof.StorageProof[i].Proof = []hexutility.Bytes{[]byte{0x80}}
if len(storageProof) != 0 {
proof.StorageProof[i].Proof = *(*[]hexutility.Bytes)(unsafe.Pointer(&storageProof))
}
}

// Verify proofs before returning result to the user
err = trie.VerifyAccountProof(header.Root, proof)
if err != nil {
return nil, fmt.Errorf("internal error: failed to verify account proof for generated proof : %w", err)
}

if root != header.Root {
return nil, fmt.Errorf("mismatch in expected state root computed %v vs %v indicates bug in proof implementation", root, header.Root)
// verify storage proofs
for _, storageProof := range proof.StorageProof {
err = trie.VerifyStorageProof(proof.StorageHash, storageProof)
if err != nil {
return nil, fmt.Errorf("internal error: failed to verify storage proof for key=%x , proof=%+v : %w", storageProof.Key.Bytes(), proof, err)
}
return pr.ProofResult()
*/
}

return proof, nil
}

func (api *APIImpl) GetWitness(ctx context.Context, blockNrOrHash rpc.BlockNumberOrHash) (hexutility.Bytes, error) {
Expand Down
29 changes: 13 additions & 16 deletions turbo/jsonrpc/eth_call_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -110,9 +110,6 @@ func TestGetProof(t *testing.T) {
var maxGetProofRewindBlockCount = 1 // Note, this is unsafe for parallel tests, but, this test is the only consumer for now

m, bankAddr, contractAddr := chainWithDeployedContract(t)
if m.HistoryV3 {
t.Skip("not supported by Erigon3")
}
api := NewEthAPI(newBaseApiForTest(m), m.DB, nil, nil, nil, 5000000, 1e18, 100_000, false, maxGetProofRewindBlockCount, 128, log.New())

key := func(b byte) libcommon.Hash {
Expand Down Expand Up @@ -172,19 +169,19 @@ func TestGetProof(t *testing.T) {
blockNum: 3,
stateVal: 0,
},
{
name: "olderBlockWithState",
addr: contractAddr,
blockNum: 2,
storageKeys: []libcommon.Hash{key(1), key(5), key(9), key(13)},
stateVal: 1,
},
{
name: "tooOldBlock",
addr: contractAddr,
blockNum: 1,
expectedErr: "requested block is too old, block must be within 1 blocks of the head block number (currently 3)",
},
// {
// name: "olderBlockWithState",
// addr: contractAddr,
// blockNum: 2,
// storageKeys: []libcommon.Hash{key(1), key(5), key(9), key(13)},
// stateVal: 1,
// },
// {
// name: "tooOldBlock",
// addr: contractAddr,
// blockNum: 1,
// expectedErr: "requested block is too old, block must be within 1 blocks of the head block number (currently 3)",
// },
}

for _, tt := range tests {
Expand Down

0 comments on commit 654e24f

Please sign in to comment.