Bor proofgen tests (#8751)

Added initial proof generation tests for polygon reverse flow for devnet

Blocks tested, receipts need trie proof clarification
This commit is contained in:
Mark Holt 2023-11-17 13:41:45 +03:00 committed by GitHub
parent a5ff524740
commit f3ce5f8a36
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
25 changed files with 783 additions and 157 deletions

View File

@ -52,7 +52,7 @@ func HashTreeRoot(schema ...interface{}) ([32]byte, error) {
copy(leaves[pos:], root[:])
default:
// If the element does not match any supported types, panic with an error message
panic(fmt.Sprintf("get it out of my face, u put a bad component in the schema. index %d", i))
panic(fmt.Sprintf("Can't create TreeRoot: unsported type %T at index %d", i, obj))
}
// Move the position pointer to the next leaf

View File

@ -9,7 +9,7 @@ import (
)
func BaseFeeFromBlock(ctx context.Context) (uint64, error) {
res, err := devnet.SelectNode(ctx).GetBlockByNumber(rpc.LatestBlockNumber, false)
res, err := devnet.SelectNode(ctx).GetBlockByNumber(ctx, rpc.LatestBlockNumber, false)
if err != nil {
return 0, fmt.Errorf("failed to get base fee from block: %v\n", err)

View File

@ -0,0 +1,65 @@
package blocks
import (
"context"
"crypto/ecdsa"
"fmt"
"testing"
"github.com/ledgerwatch/erigon/accounts/abi/bind"
"github.com/ledgerwatch/erigon/accounts/abi/bind/backends"
"github.com/ledgerwatch/erigon/core"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/crypto"
"github.com/ledgerwatch/erigon/turbo/stages/mock"
)
type TxFn func(_ *core.BlockGen, backend bind.ContractBackend) (types.Transaction, bool)
type TxGen struct {
Fn TxFn
Key *ecdsa.PrivateKey
}
func GenerateBlocks(t *testing.T, gspec *types.Genesis, blocks int, txs map[int]TxGen, txPerBlock func(int) int) (*mock.MockSentry, *core.ChainPack, error) {
key, _ := crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291")
m := mock.MockWithGenesis(t, gspec, key, false)
contractBackend := backends.NewTestSimulatedBackendWithConfig(t, gspec.Alloc, gspec.Config, gspec.GasLimit)
chain, err := core.GenerateChain(m.ChainConfig, m.Genesis, m.Engine, m.DB, blocks, func(blockNum int, block *core.BlockGen) {
var tx types.Transaction
var isContractCall bool
signer := types.LatestSignerForChainID(nil)
txCount := txPerBlock(blockNum)
for i := 0; i < txCount; i++ {
if txToSend, ok := txs[i%len(txs)]; ok {
tx, isContractCall = txToSend.Fn(block, contractBackend)
var err error
tx, err = types.SignTx(tx, *signer, txToSend.Key)
if err != nil {
return
}
}
if tx != nil {
if !isContractCall {
err := contractBackend.SendTransaction(context.Background(), tx)
if err != nil {
return
}
}
block.AddTx(tx)
}
}
contractBackend.Commit()
})
if err != nil {
return nil, nil, fmt.Errorf("generate chain: %w", err)
}
return m, chain, err
}

View File

@ -132,7 +132,7 @@ func (c *blockWaiter) receive(ctx context.Context, node devnet.Node, headers cha
default:
}
block, err := node.GetBlockByNumber(rpc.AsBlockNumber(header.Number), true)
block, err := node.GetBlockByNumber(ctx, rpc.AsBlockNumber(header.Number), true)
if err != nil {
c.logger.Error("Block waiter failed to get block", "err", err)

View File

@ -77,7 +77,7 @@ contract RootReceiver {
// verify receipt inclusion
require(
MerklePatriciaProof.verify(receipt.toBytes(), branchMaskBytes, payload.getReceiptProof(), receiptRoot),
"FxRootTunnel: INVALID_RECEIPT_PROOF"
"RootTunnel: INVALID_RECEIPT_PROOF"
);
// verify checkpoint inclusion

View File

@ -62,7 +62,7 @@ func DeployAndCallLogSubscriber(ctx context.Context, deployer string) (*libcommo
blockNum := txToBlockMap[eventHash]
block, err := node.GetBlockByNumber(rpc.AsBlockNumber(blockNum), true)
block, err := node.GetBlockByNumber(ctx, rpc.AsBlockNumber(blockNum), true)
if err != nil {
return nil, err

View File

@ -1,6 +1,7 @@
package requests
import (
"context"
"fmt"
"math/big"
@ -36,7 +37,7 @@ type StorageResult struct {
func (reqGen *requestGenerator) GetCode(address libcommon.Address, blockRef rpc.BlockReference) (hexutility.Bytes, error) {
var result hexutility.Bytes
if err := reqGen.rpcCall(&result, Methods.ETHGetCode, address, blockRef); err != nil {
if err := reqGen.rpcCall(context.Background(), &result, Methods.ETHGetCode, address, blockRef); err != nil {
return nil, err
}
@ -46,7 +47,7 @@ func (reqGen *requestGenerator) GetCode(address libcommon.Address, blockRef rpc.
func (reqGen *requestGenerator) GetBalance(address libcommon.Address, blockRef rpc.BlockReference) (*big.Int, error) {
var result hexutil.Big
if err := reqGen.rpcCall(&result, Methods.ETHGetBalance, address, blockRef); err != nil {
if err := reqGen.rpcCall(context.Background(), &result, Methods.ETHGetBalance, address, blockRef); err != nil {
return nil, err
}
@ -56,7 +57,7 @@ func (reqGen *requestGenerator) GetBalance(address libcommon.Address, blockRef r
func (reqGen *requestGenerator) GetTransactionCount(address libcommon.Address, blockRef rpc.BlockReference) (*big.Int, error) {
var result hexutil.Big
if err := reqGen.rpcCall(&result, Methods.ETHGetTransactionCount, address, blockRef); err != nil {
if err := reqGen.rpcCall(context.Background(), &result, Methods.ETHGetTransactionCount, address, blockRef); err != nil {
return nil, err
}

View File

@ -1,13 +1,15 @@
package requests
import (
"context"
"github.com/ledgerwatch/erigon/p2p"
)
func (reqGen *requestGenerator) AdminNodeInfo() (p2p.NodeInfo, error) {
var result p2p.NodeInfo
if err := reqGen.rpcCall(&result, Methods.AdminNodeInfo); err != nil {
if err := reqGen.rpcCall(context.Background(), &result, Methods.AdminNodeInfo); err != nil {
return p2p.NodeInfo{}, err
}

View File

@ -1,6 +1,7 @@
package requests
import (
"context"
"encoding/json"
"math/big"
@ -109,21 +110,21 @@ type EthGetTransactionCount struct {
func (reqGen *requestGenerator) BlockNumber() (uint64, error) {
var result hexutil2.Uint64
if err := reqGen.rpcCall(&result, Methods.ETHBlockNumber); err != nil {
if err := reqGen.rpcCall(context.Background(), &result, Methods.ETHBlockNumber); err != nil {
return 0, err
}
return uint64(result), nil
}
func (reqGen *requestGenerator) GetBlockByNumber(blockNum rpc.BlockNumber, withTxs bool) (*Block, error) {
func (reqGen *requestGenerator) GetBlockByNumber(ctx context.Context, blockNum rpc.BlockNumber, withTxs bool) (*Block, error) {
var result Block
var err error
if withTxs {
err = reqGen.rpcCall(&result, Methods.ETHGetBlockByNumber, blockNum, withTxs)
err = reqGen.rpcCall(ctx, &result, Methods.ETHGetBlockByNumber, blockNum, withTxs)
} else {
err = reqGen.rpcCall(&result.BlockWithTxHashes, Methods.ETHGetBlockByNumber, blockNum, withTxs)
err = reqGen.rpcCall(ctx, &result.BlockWithTxHashes, Methods.ETHGetBlockByNumber, blockNum, withTxs)
}
if err != nil {
@ -133,10 +134,10 @@ func (reqGen *requestGenerator) GetBlockByNumber(blockNum rpc.BlockNumber, withT
return &result, nil
}
func (req *requestGenerator) GetRootHash(startBlock uint64, endBlock uint64) (libcommon.Hash, error) {
func (req *requestGenerator) GetRootHash(ctx context.Context, startBlock uint64, endBlock uint64) (libcommon.Hash, error) {
var result string
if err := req.rpcCall(&result, Methods.BorGetRootHash, startBlock, endBlock); err != nil {
if err := req.rpcCall(ctx, &result, Methods.BorGetRootHash, startBlock, endBlock); err != nil {
return libcommon.Hash{}, err
}

View File

@ -51,7 +51,7 @@ func NewLog(hash libcommon.Hash, blockNum uint64, address libcommon.Address, top
func (reqGen *requestGenerator) FilterLogs(ctx context.Context, query ethereum.FilterQuery) ([]types.Log, error) {
var result []types.Log
if err := reqGen.rpcCall(&result, Methods.ETHGetLogs, query); err != nil {
if err := reqGen.rpcCall(ctx, &result, Methods.ETHGetLogs, query); err != nil {
return nil, err
}

View File

@ -0,0 +1,105 @@
package requests
import (
"context"
"errors"
"math/big"
ethereum "github.com/ledgerwatch/erigon"
libcommon "github.com/ledgerwatch/erigon-lib/common"
"github.com/ledgerwatch/erigon-lib/common/hexutility"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/p2p"
"github.com/ledgerwatch/erigon/rpc"
"github.com/ledgerwatch/erigon/turbo/adapter/ethapi"
"github.com/ledgerwatch/erigon/turbo/jsonrpc"
)
var ErrNotImplemented = errors.New("not implemented")
type NopRequestGenerator struct {
}
func (n NopRequestGenerator) PingErigonRpc() PingResult {
return PingResult{}
}
func (n NopRequestGenerator) GetBalance(address libcommon.Address, blockRef rpc.BlockReference) (*big.Int, error) {
return nil, ErrNotImplemented
}
func (n NopRequestGenerator) AdminNodeInfo() (p2p.NodeInfo, error) {
return p2p.NodeInfo{}, ErrNotImplemented
}
func (n NopRequestGenerator) GetBlockByNumber(ctx context.Context, blockNum rpc.BlockNumber, withTxs bool) (*Block, error) {
return nil, ErrNotImplemented
}
func (n NopRequestGenerator) GetTransactionByHash(hash libcommon.Hash) (*jsonrpc.RPCTransaction, error) {
return nil, ErrNotImplemented
}
func (n NopRequestGenerator) GetTransactionReceipt(ctx context.Context, hash libcommon.Hash) (*types.Receipt, error) {
return nil, ErrNotImplemented
}
func (n NopRequestGenerator) TraceTransaction(hash libcommon.Hash) ([]TransactionTrace, error) {
return nil, ErrNotImplemented
}
func (n NopRequestGenerator) GetTransactionCount(address libcommon.Address, blockRef rpc.BlockReference) (*big.Int, error) {
return nil, ErrNotImplemented
}
func (n NopRequestGenerator) BlockNumber() (uint64, error) {
return 0, ErrNotImplemented
}
func (n NopRequestGenerator) SendTransaction(signedTx types.Transaction) (libcommon.Hash, error) {
return libcommon.Hash{}, ErrNotImplemented
}
func (n NopRequestGenerator) FilterLogs(ctx context.Context, query ethereum.FilterQuery) ([]types.Log, error) {
return nil, ErrNotImplemented
}
func (n NopRequestGenerator) SubscribeFilterLogs(ctx context.Context, query ethereum.FilterQuery, ch chan<- types.Log) (ethereum.Subscription, error) {
return nil, ErrNotImplemented
}
func (n NopRequestGenerator) Subscribe(ctx context.Context, method SubMethod, subChan interface{}, args ...interface{}) (ethereum.Subscription, error) {
return nil, ErrNotImplemented
}
func (n NopRequestGenerator) TxpoolContent() (int, int, int, error) {
return 0, 0, 0, ErrNotImplemented
}
func (n NopRequestGenerator) Call(args ethapi.CallArgs, blockRef rpc.BlockReference, overrides *ethapi.StateOverrides) ([]byte, error) {
return nil, ErrNotImplemented
}
func (n NopRequestGenerator) TraceCall(blockRef rpc.BlockReference, args ethapi.CallArgs, traceOpts ...TraceOpt) (*TraceCallResult, error) {
return nil, ErrNotImplemented
}
func (n NopRequestGenerator) DebugAccountAt(blockHash libcommon.Hash, txIndex uint64, account libcommon.Address) (*AccountResult, error) {
return nil, ErrNotImplemented
}
func (n NopRequestGenerator) GetCode(address libcommon.Address, blockRef rpc.BlockReference) (hexutility.Bytes, error) {
return nil, ErrNotImplemented
}
func (n NopRequestGenerator) EstimateGas(args ethereum.CallMsg, blockNum BlockNumber) (uint64, error) {
return 0, ErrNotImplemented
}
func (n NopRequestGenerator) GasPrice() (*big.Int, error) {
return nil, ErrNotImplemented
}
func (n NopRequestGenerator) GetRootHash(ctx context.Context, startBlock uint64, endBlock uint64) (libcommon.Hash, error) {
return libcommon.Hash{}, ErrNotImplemented
}

View File

@ -56,9 +56,9 @@ type RequestGenerator interface {
PingErigonRpc() PingResult
GetBalance(address libcommon.Address, blockRef rpc.BlockReference) (*big.Int, error)
AdminNodeInfo() (p2p.NodeInfo, error)
GetBlockByNumber(blockNum rpc.BlockNumber, withTxs bool) (*Block, error)
GetBlockByNumber(ctx context.Context, blockNum rpc.BlockNumber, withTxs bool) (*Block, error)
GetTransactionByHash(hash libcommon.Hash) (*jsonrpc.RPCTransaction, error)
GetTransactionReceipt(hash libcommon.Hash) (*types.Receipt, error)
GetTransactionReceipt(ctx context.Context, hash libcommon.Hash) (*types.Receipt, error)
TraceTransaction(hash libcommon.Hash) ([]TransactionTrace, error)
GetTransactionCount(address libcommon.Address, blockRef rpc.BlockReference) (*big.Int, error)
BlockNumber() (uint64, error)
@ -74,7 +74,7 @@ type RequestGenerator interface {
EstimateGas(args ethereum.CallMsg, blockNum BlockNumber) (uint64, error)
GasPrice() (*big.Int, error)
GetRootHash(startBlock uint64, endBlock uint64) (libcommon.Hash, error)
GetRootHash(ctx context.Context, startBlock uint64, endBlock uint64) (libcommon.Hash, error)
}
type requestGenerator struct {
@ -173,8 +173,7 @@ func (req *requestGenerator) rpcCallJSON(method RPCMethod, body string, response
}
}
func (req *requestGenerator) rpcCall(result interface{}, method RPCMethod, args ...interface{}) error {
ctx := context.Background()
func (req *requestGenerator) rpcCall(ctx context.Context, result interface{}, method RPCMethod, args ...interface{}) error {
client, err := req.rpcClient(ctx)
if err != nil {
return err

View File

@ -1,6 +1,7 @@
package requests
import (
"context"
"encoding/json"
"fmt"
@ -135,7 +136,7 @@ func (req *requestGenerator) traceCall(blockRef rpc.BlockReference, callArgs str
func (reqGen *requestGenerator) TraceTransaction(hash libcommon.Hash) ([]TransactionTrace, error) {
var result []TransactionTrace
if err := reqGen.rpcCall(&result, Methods.TraceTransaction, hash); err != nil {
if err := reqGen.rpcCall(context.Background(), &result, Methods.TraceTransaction, hash); err != nil {
return nil, err
}

View File

@ -2,6 +2,7 @@ package requests
import (
"bytes"
"context"
"encoding/json"
"fmt"
"math/big"
@ -101,7 +102,7 @@ func (req *requestGenerator) estimateGas(callArgs string, blockRef BlockNumber)
func (reqGen *requestGenerator) GasPrice() (*big.Int, error) {
var result hexutil.Big
if err := reqGen.rpcCall(&result, Methods.ETHGasPrice); err != nil {
if err := reqGen.rpcCall(context.Background(), &result, Methods.ETHGasPrice); err != nil {
return nil, err
}
@ -111,7 +112,7 @@ func (reqGen *requestGenerator) GasPrice() (*big.Int, error) {
func (reqGen *requestGenerator) Call(args ethapi.CallArgs, blockRef rpc.BlockReference, overrides *ethapi.StateOverrides) ([]byte, error) {
var result hexutility.Bytes
if err := reqGen.rpcCall(&result, Methods.ETHCall, args, blockRef, overrides); err != nil {
if err := reqGen.rpcCall(context.Background(), &result, Methods.ETHCall, args, blockRef, overrides); err != nil {
return nil, err
}
@ -126,7 +127,7 @@ func (reqGen *requestGenerator) SendTransaction(signedTx types.Transaction) (lib
return libcommon.Hash{}, fmt.Errorf("failed to marshal binary: %v", err)
}
if err := reqGen.rpcCall(&result, Methods.ETHSendRawTransaction, hexutility.Bytes(buf.Bytes())); err != nil {
if err := reqGen.rpcCall(context.Background(), &result, Methods.ETHSendRawTransaction, hexutility.Bytes(buf.Bytes())); err != nil {
return libcommon.Hash{}, err
}
@ -140,7 +141,7 @@ func (reqGen *requestGenerator) SendTransaction(signedTx types.Transaction) (lib
}
if zeroHash {
return libcommon.Hash{}, fmt.Errorf("Hash: %s, nonce %d: returned a zero transaction hash", signedTx.Hash().Hex(), signedTx.GetNonce())
return libcommon.Hash{}, fmt.Errorf("hash: %s, nonce %d: returned a zero transaction hash", signedTx.Hash().Hex(), signedTx.GetNonce())
}
return result, nil
@ -149,17 +150,17 @@ func (reqGen *requestGenerator) SendTransaction(signedTx types.Transaction) (lib
func (req *requestGenerator) GetTransactionByHash(hash libcommon.Hash) (*jsonrpc.RPCTransaction, error) {
var result jsonrpc.RPCTransaction
if err := req.rpcCall(&result, Methods.ETHGetTransactionByHash, hash); err != nil {
if err := req.rpcCall(context.Background(), &result, Methods.ETHGetTransactionByHash, hash); err != nil {
return nil, err
}
return &result, nil
}
func (req *requestGenerator) GetTransactionReceipt(hash libcommon.Hash) (*types.Receipt, error) {
func (req *requestGenerator) GetTransactionReceipt(ctx context.Context, hash libcommon.Hash) (*types.Receipt, error) {
var result types.Receipt
if err := req.rpcCall(&result, Methods.ETHGetTransactionReceipt, hash); err != nil {
if err := req.rpcCall(ctx, &result, Methods.ETHGetTransactionReceipt, hash); err != nil {
return nil, err
}

View File

@ -25,7 +25,7 @@ func (reqGen *requestGenerator) TxpoolContent() (int, int, int, error) {
resp, ok := b.Result.(map[string]interface{})
if !ok {
return 0, 0, 0, fmt.Errorf("Unexpected result type: %T", b.Result)
return 0, 0, 0, fmt.Errorf("unexpected result type: %T", b.Result)
}
pendingLen := 0

View File

@ -375,7 +375,7 @@ func (h *Heimdall) getRootHash(ctx context.Context, start uint64, end uint64) (l
return libcommon.Hash{}, errors.New("number of headers requested exceeds")
}
return devnet.SelectBlockProducer(devnet.WithCurrentNetwork(ctx, networkname.BorDevnetChainName)).GetRootHash(start, end)
return devnet.SelectBlockProducer(devnet.WithCurrentNetwork(ctx, networkname.BorDevnetChainName)).GetRootHash(ctx, start, end)
}
func (h *Heimdall) shouldSendCheckpoint(start uint64, end uint64) (bool, error) {

View File

@ -11,6 +11,7 @@ import (
"sync"
"github.com/ledgerwatch/erigon/cl/merkle_tree"
"golang.org/x/sync/errgroup"
"github.com/ledgerwatch/erigon-lib/chain/networkname"
libcommon "github.com/ledgerwatch/erigon-lib/common"
@ -73,11 +74,11 @@ func (pg *ProofGenerator) GenerateExitPayload(ctx context.Context, burnTxHash li
isCheckpointed, err := pg.isCheckPointed(ctx, burnTxHash)
if err != nil {
return nil, fmt.Errorf("Error getting burn transaction: %w", err)
return nil, fmt.Errorf("error getting burn transaction: %w", err)
}
if !isCheckpointed {
return nil, fmt.Errorf("Burn transaction has not been checkpointed yet")
return nil, fmt.Errorf("eurn transaction has not been checkpointed yet")
}
// build payload for exit
@ -85,14 +86,14 @@ func (pg *ProofGenerator) GenerateExitPayload(ctx context.Context, burnTxHash li
if err != nil {
if errors.Is(err, ErrTokenIndexOutOfRange) {
return nil, fmt.Errorf("Block not included: %w", err)
return nil, fmt.Errorf("block not included: %w", err)
}
return nil, fmt.Errorf("Null receipt received")
return nil, fmt.Errorf("null receipt received")
}
if len(result) == 0 {
return nil, fmt.Errorf("Null result received")
return nil, fmt.Errorf("null result received")
}
return result, nil
@ -107,6 +108,7 @@ func (pg *ProofGenerator) getChainBlockInfo(ctx context.Context, burnTxHash libc
var burnTransaction *jsonrpc.RPCTransaction
var err [2]error
// err group
wg.Add(1)
go func() {
defer wg.Done()
@ -146,11 +148,11 @@ func (pg *ProofGenerator) buildPayloadForExit(ctx context.Context, burnTxHash li
node := devnet.SelectBlockProducer(ctx)
if node == nil {
return nil, fmt.Errorf("No node available")
return nil, fmt.Errorf("no node available")
}
if index < 0 {
return nil, fmt.Errorf("Index must not negative")
return nil, fmt.Errorf("index must not negative")
}
var receipt *types.Receipt
@ -164,32 +166,29 @@ func (pg *ProofGenerator) buildPayloadForExit(ctx context.Context, burnTxHash li
}
if lastChildBlockNum < txBlockNum {
return nil, fmt.Errorf("Burn transaction has not been checkpointed as yet")
return nil, fmt.Errorf("burn transaction has not been checkpointed as yet")
}
// step 2- get transaction receipt from txhash and
// block information from block number
var wg sync.WaitGroup
var errs [2]error
g, gctx := errgroup.WithContext(ctx)
g.SetLimit(2)
wg.Add(1)
go func() {
defer wg.Done()
receipt, errs[0] = node.GetTransactionReceipt(burnTxHash)
}()
g.Go(func() error {
var err error
receipt, err = node.GetTransactionReceipt(gctx, burnTxHash)
return err
})
go func() {
defer wg.Done()
block, errs[1] = node.GetBlockByNumber(rpc.AsBlockNumber(txBlockNum), true)
}()
g.Go(func() error {
var err error
block, err = node.GetBlockByNumber(gctx, rpc.AsBlockNumber(txBlockNum), true)
return err
})
wg.Wait()
for _, err := range errs {
if err != nil {
return nil, err
}
if err := g.Wait(); err != nil {
return nil, err
}
// step 3 - get information about block saved in parent chain
@ -203,20 +202,22 @@ func (pg *ProofGenerator) buildPayloadForExit(ctx context.Context, burnTxHash li
return nil, err
}
blockProof, err := getBlockProof(node, txBlockNum, start, end)
blockProofs, err := getBlockProofs(ctx, node, txBlockNum, start, end)
if err != nil {
return nil, err
}
// step 5- create receipt proof
receiptProof, err := getReceiptProof(receipt, block, node, nil)
receiptProof, err := getReceiptProof(ctx, node, receipt, block, nil)
if err != nil {
return nil, err
}
// step 6 - encode payload, convert into hex
var logIndex int
if index > 0 {
logIndices := getAllLogIndices(logEventSig, receipt)
@ -224,56 +225,34 @@ func (pg *ProofGenerator) buildPayloadForExit(ctx context.Context, burnTxHash li
return nil, ErrTokenIndexOutOfRange
}
return encodePayload(
rootBlockNumber,
blockProof,
txBlockNum,
block.Time,
block.TxHash,
block.ReceiptHash,
getReceiptBytes(receipt), // rlp encoded
receiptProof.parentNodes,
receiptProof.path,
logIndices[index]), nil
logIndex = logIndices[index]
} else {
logIndex = getLogIndex(logEventSig, receipt)
}
logIndex := getLogIndex(logEventSig, receipt)
if logIndex < 0 {
return nil, fmt.Errorf("Log not found in receipt")
return nil, fmt.Errorf("log not found in receipt")
}
return encodePayload(
rootBlockNumber,
blockProof,
txBlockNum,
block.Time,
block.TxHash,
block.ReceiptHash,
getReceiptBytes(receipt), // rlp encoded
receiptProof.parentNodes,
receiptProof.path,
logIndex), nil
}
parentNodesBytes, err := rlp.EncodeToBytes(receiptProof.parentNodes)
func encodePayload(headerNumber uint64, buildBlockProof string, blockNumber uint64, timestamp uint64, transactionsRoot libcommon.Hash, receiptsRoot libcommon.Hash, receipt []byte, receiptParentNodes [][]byte, path []byte, logIndex int) []byte {
parentNodesBytes, _ := rlp.EncodeToBytes(receiptParentNodes)
if err != nil {
return nil, err
}
bytes, _ := rlp.EncodeToBytes(
return rlp.EncodeToBytes(
[]interface{}{
headerNumber,
buildBlockProof,
blockNumber,
timestamp,
hexutility.Encode(transactionsRoot[:]),
hexutility.Encode(receiptsRoot[:]),
hexutility.Encode(receipt),
rootBlockNumber,
hexutility.Encode(bytes.Join(blockProofs, []byte{})),
txBlockNum,
block.Time,
hexutility.Encode(block.TxHash[:]),
hexutility.Encode(block.ReceiptHash[:]),
hexutility.Encode(getReceiptBytes(receipt)), //rpl encoded
hexutility.Encode(parentNodesBytes),
hexutility.Encode(append([]byte{0}, path...)),
hexutility.Encode(append([]byte{0}, receiptProof.path...)),
logIndex,
})
return bytes
}
type receiptProof struct {
@ -284,40 +263,42 @@ type receiptProof struct {
value interface{}
}
func getReceiptProof(receipt *types.Receipt, block *requests.Block, node devnet.Node, receipts []*types.Receipt) (*receiptProof, error) {
func getReceiptProof(ctx context.Context, node requests.RequestGenerator, receipt *types.Receipt, block *requests.Block, receipts []*types.Receipt) (*receiptProof, error) {
stateSyncTxHash := types.ComputeBorTxHash(block.Number.Uint64(), block.Hash)
receiptsTrie := trie.New(trie.EmptyRoot)
if len(receipts) == 0 {
var wg sync.WaitGroup
g, gctx := errgroup.WithContext(ctx)
g.SetLimit(len(block.Transactions))
var lock sync.Mutex
errs := make([]error, len(block.Transactions))
for i, transaction := range block.Transactions {
for _, transaction := range block.Transactions {
if transaction.Hash == stateSyncTxHash {
// ignore if tx hash is bor state-sync tx
continue
}
hash := transaction.Hash
wg.Add(1)
go func(i int) {
defer wg.Done()
receipt, errs[i] = node.GetTransactionReceipt(hash)
g.Go(func() error {
receipt, err := node.GetTransactionReceipt(gctx, hash)
if err != nil {
return err
}
path, _ := rlp.EncodeToBytes(receipt.TransactionIndex)
rawReceipt := getReceiptBytes(receipt)
lock.Lock()
defer lock.Unlock()
receiptsTrie.Update(path, rawReceipt)
}(i)
return nil
})
}
wg.Wait()
for _, err := range errs {
if err != nil {
return nil, err
}
if err := g.Wait(); err != nil {
return nil, err
}
} else {
for _, receipt := range receipts {
@ -331,7 +312,7 @@ func getReceiptProof(receipt *types.Receipt, block *requests.Block, node devnet.
result, ok := receiptsTrie.Get(path)
if !ok {
return nil, fmt.Errorf("Node does not contain the key")
return nil, fmt.Errorf("node does not contain the key")
}
var nodeValue any
@ -351,17 +332,7 @@ func getReceiptProof(receipt *types.Receipt, block *requests.Block, node devnet.
}, nil
}
func getBlockProof(node devnet.Node, txBlockNum, startBlock, endBlock uint64) (string, error) {
proofs, err := getFastMerkleProof(node, txBlockNum, startBlock, endBlock)
if err != nil {
return "", err
}
return hexutility.Encode(bytes.Join(proofs, []byte{})), nil
}
func getFastMerkleProof(node devnet.Node, blockNumber, startBlock, endBlock uint64) ([][]byte, error) {
func getBlockProofs(ctx context.Context, node requests.RequestGenerator, blockNumber, startBlock, endBlock uint64) ([][]byte, error) {
merkleTreeDepth := int(math.Ceil(math.Log2(float64(endBlock - startBlock + 1))))
// We generate the proof root down, whereas we need from leaf up
@ -373,7 +344,7 @@ func getFastMerkleProof(node devnet.Node, blockNumber, startBlock, endBlock uint
rightBound := endBlock - offset
// console.log("Searching for", targetIndex);
for depth := 0; depth < merkleTreeDepth; depth += 1 {
for depth := 0; depth < merkleTreeDepth; depth++ {
nLeaves := uint64(2) << (merkleTreeDepth - depth)
// The pivot leaf is the last leaf which is included in the left subtree
@ -382,8 +353,7 @@ func getFastMerkleProof(node devnet.Node, blockNumber, startBlock, endBlock uint
if targetIndex > pivotLeaf {
// Get the root hash to the merkle subtree to the left
newLeftBound := pivotLeaf + 1
// eslint-disable-next-line no-await-in-loop
subTreeMerkleRoot, err := node.GetRootHash(offset+leftBound, offset+pivotLeaf)
subTreeMerkleRoot, err := node.GetRootHash(ctx, offset+leftBound, offset+pivotLeaf)
if err != nil {
return nil, err
@ -421,7 +391,7 @@ func getFastMerkleProof(node devnet.Node, blockNumber, startBlock, endBlock uint
// We need to build a tree which has heightDifference layers
// The first leaf will hold the root hash as returned by the RPC
remainingNodesHash, err := node.GetRootHash(offset+pivotLeaf+1, offset+rightBound)
remainingNodesHash, err := node.GetRootHash(ctx, offset+pivotLeaf+1, offset+rightBound)
if err != nil {
return nil, err
@ -431,14 +401,23 @@ func getFastMerkleProof(node devnet.Node, blockNumber, startBlock, endBlock uint
leafRoots := recursiveZeroHash(subTreeHeight)
// Build a merkle tree of correct size for the subtree using these merkle roots
leaves := make([][]byte, 2<<heightDifference)
var leafCount int
if heightDifference > 0 {
leafCount = 2 << heightDifference
} else {
leafCount = 1
}
leaves := make([]interface{}, leafCount)
leaves[0] = remainingNodesHash[:]
for i := 1; i < len(leaves); i++ {
leaves[i] = leafRoots[:]
}
subTreeMerkleRoot, err := merkle_tree.HashTreeRoot(leaves)
subTreeMerkleRoot, err := merkle_tree.HashTreeRoot(leaves...)
if err != nil {
return nil, err

View File

@ -0,0 +1,467 @@
package polygon
import (
"bytes"
"context"
"crypto/ecdsa"
"fmt"
"math"
"math/big"
"sync"
"testing"
"github.com/holiman/uint256"
"github.com/ledgerwatch/erigon-lib/chain"
"github.com/ledgerwatch/erigon-lib/common"
libcommon "github.com/ledgerwatch/erigon-lib/common"
"github.com/ledgerwatch/erigon-lib/kv"
"github.com/ledgerwatch/erigon-lib/kv/memdb"
"github.com/ledgerwatch/erigon/accounts/abi/bind"
"github.com/ledgerwatch/erigon/cmd/devnet/blocks"
"github.com/ledgerwatch/erigon/cmd/devnet/requests"
"github.com/ledgerwatch/erigon/consensus"
"github.com/ledgerwatch/erigon/consensus/bor"
"github.com/ledgerwatch/erigon/consensus/bor/contract"
"github.com/ledgerwatch/erigon/consensus/bor/heimdall/span"
"github.com/ledgerwatch/erigon/consensus/bor/valset"
"github.com/ledgerwatch/erigon/core"
"github.com/ledgerwatch/erigon/core/rawdb"
"github.com/ledgerwatch/erigon/core/state"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/core/vm"
"github.com/ledgerwatch/erigon/crypto"
"github.com/ledgerwatch/erigon/params"
"github.com/ledgerwatch/erigon/rlp"
"github.com/ledgerwatch/erigon/rpc"
"github.com/ledgerwatch/erigon/turbo/jsonrpc"
"github.com/ledgerwatch/erigon/turbo/services"
"github.com/ledgerwatch/erigon/turbo/stages/mock"
"github.com/ledgerwatch/erigon/turbo/transactions"
"github.com/ledgerwatch/log/v3"
"github.com/pion/randutil"
)
type requestGenerator struct {
sync.Mutex
requests.NopRequestGenerator
sentry *mock.MockSentry
bor *bor.Bor
chain *core.ChainPack
txBlockMap map[libcommon.Hash]*types.Block
}
func newRequestGenerator(sentry *mock.MockSentry, chain *core.ChainPack) (*requestGenerator, error) {
db := memdb.New("")
tx, err := db.BeginRw(context.Background())
if err != nil {
return nil, err
}
if err = rawdb.WriteHeader(tx, chain.TopBlock.Header()); err != nil {
return nil, err
}
if err = rawdb.WriteHeadHeaderHash(tx, chain.TopBlock.Header().Hash()); err != nil {
return nil, err
}
tx.Commit()
reader := blockReader{
chain: chain,
}
return &requestGenerator{
chain: chain,
sentry: sentry,
bor: bor.NewRo(params.BorDevnetChainConfig, db, reader,
&spanner{
span.NewChainSpanner(contract.ValidatorSet(), params.BorDevnetChainConfig, false, log.Root()),
libcommon.Address{},
span.Span{}},
genesisContract{}, log.Root()),
txBlockMap: map[libcommon.Hash]*types.Block{},
}, nil
}
func (rg *requestGenerator) GetRootHash(ctx context.Context, startBlock uint64, endBlock uint64) (libcommon.Hash, error) {
tx, err := rg.bor.DB.BeginRo(context.Background())
if err != nil {
return libcommon.Hash{}, err
}
defer tx.Rollback()
result, err := rg.bor.GetRootHash(ctx, tx, startBlock, endBlock)
if err != nil {
return libcommon.Hash{}, err
}
return libcommon.HexToHash(result), nil
}
func (rg *requestGenerator) GetBlockByNumber(ctx context.Context, blockNum rpc.BlockNumber, withTxs bool) (*requests.Block, error) {
if bn := int(blockNum.Uint64()); bn < len(rg.chain.Blocks) {
block := rg.chain.Blocks[bn]
transactions := make([]*jsonrpc.RPCTransaction, len(block.Transactions()))
for i, tx := range block.Transactions() {
rg.txBlockMap[tx.Hash()] = block
transactions[i] = jsonrpc.NewRPCTransaction(tx, block.Hash(), blockNum.Uint64(), uint64(i), block.BaseFee())
}
return &requests.Block{
BlockWithTxHashes: requests.BlockWithTxHashes{
Header: block.Header(),
Hash: block.Hash(),
},
Transactions: transactions,
}, nil
}
return nil, fmt.Errorf("block %d not found", blockNum.Uint64())
}
func (rg *requestGenerator) GetTransactionReceipt(ctx context.Context, hash libcommon.Hash) (*types.Receipt, error) {
rg.Lock()
defer rg.Unlock()
block, ok := rg.txBlockMap[hash]
if !ok {
return nil, fmt.Errorf("can't find block to tx: %s", hash)
}
engine := rg.bor
chainConfig := params.BorDevnetChainConfig
reader := blockReader{
chain: rg.chain,
}
tx, err := rg.sentry.DB.BeginRo(context.Background())
if err != nil {
return nil, err
}
defer tx.Rollback()
_, _, _, ibs, _, err := transactions.ComputeTxEnv(ctx, engine, block, chainConfig, reader, tx, 0, false)
if err != nil {
return nil, err
}
var usedGas uint64
var usedBlobGas uint64
gp := new(core.GasPool).AddGas(block.GasLimit()).AddBlobGas(chainConfig.GetMaxBlobGasPerBlock())
noopWriter := state.NewNoopWriter()
getHeader := func(hash common.Hash, number uint64) *types.Header {
h, e := reader.Header(ctx, tx, hash, number)
if e != nil {
log.Error("getHeader error", "number", number, "hash", hash, "err", e)
}
return h
}
header := block.Header()
for i, txn := range block.Transactions() {
ibs.SetTxContext(txn.Hash(), block.Hash(), i)
receipt, _, err := core.ApplyTransaction(chainConfig, core.GetHashFn(header, getHeader), engine, nil, gp, ibs, noopWriter, header, txn, &usedGas, &usedBlobGas, vm.Config{})
if err != nil {
return nil, err
}
if txn.Hash() == hash {
receipt.BlockHash = block.Hash()
return receipt, nil
}
}
return nil, fmt.Errorf("tx not found in block")
}
type blockReader struct {
services.FullBlockReader
chain *core.ChainPack
}
func (reader blockReader) BlockByNumber(ctx context.Context, db kv.Tx, number uint64) (*types.Block, error) {
if int(number) < len(reader.chain.Blocks) {
return reader.chain.Blocks[number], nil
}
return nil, fmt.Errorf("block not found")
}
func (reader blockReader) HeaderByNumber(ctx context.Context, tx kv.Getter, blockNum uint64) (*types.Header, error) {
if int(blockNum) < len(reader.chain.Headers) {
return reader.chain.Headers[blockNum], nil
}
return nil, fmt.Errorf("header not found")
}
func TestMerkle(t *testing.T) {
startBlock := 1600
endBlock := 3200
if depth := int(math.Ceil(math.Log2(float64(endBlock - startBlock + 1)))); depth != 11 {
t.Fatal("Unexpected depth:", depth)
}
startBlock = 0
endBlock = 100000
if depth := int(math.Ceil(math.Log2(float64(endBlock - startBlock + 1)))); depth != 17 {
t.Fatal("Unexpected depth:", depth)
}
startBlock = 0
endBlock = 500000
if depth := int(math.Ceil(math.Log2(float64(endBlock - startBlock + 1)))); depth != 19 {
t.Fatal("Unexpected depth:", depth)
}
}
func TestBlockGeneration(t *testing.T) {
_, chain, err := generateBlocks(t, 1600)
if err != nil {
t.Fatal(err)
}
reader := blockReader{
chain: chain,
}
for number := uint64(0); number < 1600; number++ {
_, err = reader.BlockByNumber(context.Background(), nil, number)
if err != nil {
t.Fatal(err)
}
header, err := reader.HeaderByNumber(context.Background(), nil, number)
if err != nil {
t.Fatal(err)
}
if header == nil {
t.Fatalf("block header not found: %d", number)
}
}
}
type genesisContract struct {
}
func (g genesisContract) CommitState(event rlp.RawValue, syscall consensus.SystemCall) error {
return nil
}
func (g genesisContract) LastStateId(syscall consensus.SystemCall) (*big.Int, error) {
return big.NewInt(0), nil
}
type spanner struct {
*span.ChainSpanner
validatorAddress libcommon.Address
currentSpan span.Span
}
func (c spanner) GetCurrentSpan(_ consensus.SystemCall) (*span.Span, error) {
return &c.currentSpan, nil
}
func (c *spanner) CommitSpan(heimdallSpan span.HeimdallSpan, syscall consensus.SystemCall) error {
c.currentSpan = heimdallSpan.Span
return nil
}
func (c *spanner) GetCurrentValidators(spanId uint64, signer libcommon.Address, chain consensus.ChainHeaderReader) ([]*valset.Validator, error) {
return []*valset.Validator{
{
ID: 1,
Address: c.validatorAddress,
VotingPower: 1000,
ProposerPriority: 1,
}}, nil
}
func TestBlockProof(t *testing.T) {
sentry, chain, err := generateBlocks(t, 1600)
if err != nil {
t.Fatal(err)
}
rg, err := newRequestGenerator(sentry, chain)
if err != nil {
t.Fatal(err)
}
_, err = rg.GetRootHash(context.Background(), 0, 1599)
if err != nil {
t.Fatal(err)
}
blockProofs, err := getBlockProofs(context.Background(), rg, 10, 0, 1599)
if err != nil {
t.Fatal(err)
}
if len := len(blockProofs); len != 11 {
t.Fatal("Unexpected block depth:", len)
}
if len := len(bytes.Join(blockProofs, []byte{})); len != 352 {
t.Fatal("Unexpected proof len:", len)
}
}
func TestReceiptProof(t *testing.T) {
sentry, chain, err := generateBlocks(t, 1)
if err != nil {
t.Fatal(err)
}
rg, err := newRequestGenerator(sentry, chain)
if err != nil {
t.Fatal(err)
}
var block *requests.Block
var blockNo uint64
for block == nil {
block, err = rg.GetBlockByNumber(context.Background(), rpc.AsBlockNumber(blockNo), true)
if err != nil {
t.Fatal(err)
}
if len(block.Transactions) == 0 {
block = nil
blockNo++
}
}
receipt, err := rg.GetTransactionReceipt(context.Background(), block.Transactions[len(block.Transactions)-1].Hash)
if err != nil {
t.Fatal(err)
}
//TODO the receiptProof needs parents formatted from trie - need to confirm format for that
_ /*receiptProof*/, err = getReceiptProof(context.Background(), rg, receipt, block, nil)
if err != nil {
t.Fatal(err)
}
//fmt.Println(receiptProof)
}
func generateBlocks(t *testing.T, number int) (*mock.MockSentry, *core.ChainPack, error) {
data := getGenesis(3)
rand := randutil.NewMathRandomGenerator()
return blocks.GenerateBlocks(t, data.genesisSpec, number, map[int]blocks.TxGen{
0: {
Fn: getBlockTx(data.addresses[0], data.addresses[1], uint256.NewInt(uint64(rand.Intn(5000))+1)),
Key: data.keys[0],
},
1: {
Fn: getBlockTx(data.addresses[1], data.addresses[2], uint256.NewInt(uint64(rand.Intn(5000))+1)),
Key: data.keys[1],
},
2: {
Fn: getBlockTx(data.addresses[2], data.addresses[0], uint256.NewInt(uint64(rand.Intn(5000))+1)),
Key: data.keys[2],
},
}, func(_ int) int {
return rand.Intn(10)
})
}
func getBlockTx(from libcommon.Address, to libcommon.Address, amount *uint256.Int) blocks.TxFn {
return func(block *core.BlockGen, _ bind.ContractBackend) (types.Transaction, bool) {
return types.NewTransaction(block.TxNonce(from), to, amount, 21000, new(uint256.Int), nil), false
}
}
type initialData struct {
keys []*ecdsa.PrivateKey
addresses []libcommon.Address
transactOpts []*bind.TransactOpts
genesisSpec *types.Genesis
}
func getGenesis(accounts int, funds ...*big.Int) initialData {
accountFunds := big.NewInt(1000000000)
if len(funds) > 0 {
accountFunds = funds[0]
}
keys := make([]*ecdsa.PrivateKey, accounts)
for i := 0; i < accounts; i++ {
keys[i], _ = crypto.GenerateKey()
}
addresses := make([]libcommon.Address, 0, len(keys))
transactOpts := make([]*bind.TransactOpts, 0, len(keys))
allocs := types.GenesisAlloc{}
for _, key := range keys {
addr := crypto.PubkeyToAddress(key.PublicKey)
addresses = append(addresses, addr)
to, err := bind.NewKeyedTransactorWithChainID(key, big.NewInt(1))
if err != nil {
panic(err)
}
transactOpts = append(transactOpts, to)
allocs[addr] = types.GenesisAccount{Balance: accountFunds}
}
return initialData{
keys: keys,
addresses: addresses,
transactOpts: transactOpts,
genesisSpec: &types.Genesis{
Config: &chain.Config{
ChainID: big.NewInt(1),
HomesteadBlock: new(big.Int),
TangerineWhistleBlock: new(big.Int),
SpuriousDragonBlock: big.NewInt(1),
ByzantiumBlock: big.NewInt(1),
ConstantinopleBlock: big.NewInt(1),
},
Alloc: allocs,
},
}
}

View File

@ -443,7 +443,7 @@ func (w rwWrapper) BeginRwNosync(ctx context.Context) (kv.RwTx, error) {
return nil, fmt.Errorf("BeginRwNosync not implemented")
}
// This is used by the rpcdaemon which needs read only access to the provided data services
// This is used by the rpcdaemon and tests which need read only access to the provided data services
func NewRo(chainConfig *chain.Config, db kv.RoDB, blockReader services.FullBlockReader, spanner Spanner,
genesisContracts GenesisContract, logger log.Logger) *Bor {
// get bor config

View File

@ -199,6 +199,8 @@ func AsBlockNumber(no interface{}) BlockNumber {
return no
case *BlockNumber:
return *no
case int:
return BlockNumber(no)
case int64:
return BlockNumber(no)
case uint64:

View File

@ -30,7 +30,7 @@ func TestInsertIncorrectStateRootDifferentAccounts(t *testing.T) {
fromKey := data.keys[0]
to := libcommon.Address{1}
m, chain, err := genBlocks(t, data.genesisSpec, map[int]txn{
m, chain, err := GenerateBlocks(t, data.genesisSpec, map[int]txn{
0: {
getBlockTx(from, to, uint256.NewInt(1000)),
fromKey,
@ -59,7 +59,7 @@ func TestInsertIncorrectStateRootDifferentAccounts(t *testing.T) {
}
// insert a correct block
m, chain, err = genBlocks(t, data.genesisSpec, map[int]txn{
m, chain, err = GenerateBlocks(t, data.genesisSpec, map[int]txn{
0: {
getBlockTx(data.addresses[1], to, uint256.NewInt(5000)),
data.keys[1],
@ -98,7 +98,7 @@ func TestInsertIncorrectStateRootSameAccount(t *testing.T) {
fromKey := data.keys[0]
to := libcommon.Address{1}
m, chain, err := genBlocks(t, data.genesisSpec, map[int]txn{
m, chain, err := GenerateBlocks(t, data.genesisSpec, map[int]txn{
0: {
getBlockTx(from, to, uint256.NewInt(1000)),
fromKey,
@ -126,7 +126,7 @@ func TestInsertIncorrectStateRootSameAccount(t *testing.T) {
}
// insert a correct block
m, chain, err = genBlocks(t, data.genesisSpec, map[int]txn{
m, chain, err = GenerateBlocks(t, data.genesisSpec, map[int]txn{
0: {
getBlockTx(from, to, uint256.NewInt(5000)),
fromKey,
@ -163,7 +163,7 @@ func TestInsertIncorrectStateRootSameAccountSameAmount(t *testing.T) {
fromKey := data.keys[0]
to := libcommon.Address{1}
m, chain, err := genBlocks(t, data.genesisSpec, map[int]txn{
m, chain, err := GenerateBlocks(t, data.genesisSpec, map[int]txn{
0: {
getBlockTx(from, to, uint256.NewInt(1000)),
fromKey,
@ -188,7 +188,7 @@ func TestInsertIncorrectStateRootSameAccountSameAmount(t *testing.T) {
}
// insert a correct block
m, chain, err = genBlocks(t, data.genesisSpec, map[int]txn{
m, chain, err = GenerateBlocks(t, data.genesisSpec, map[int]txn{
0: {
getBlockTx(from, to, uint256.NewInt(1000)),
fromKey,
@ -225,7 +225,7 @@ func TestInsertIncorrectStateRootAllFundsRoot(t *testing.T) {
fromKey := data.keys[0]
to := libcommon.Address{1}
m, chain, err := genBlocks(t, data.genesisSpec, map[int]txn{
m, chain, err := GenerateBlocks(t, data.genesisSpec, map[int]txn{
0: {
getBlockTx(from, to, uint256.NewInt(1000)),
fromKey,
@ -250,7 +250,7 @@ func TestInsertIncorrectStateRootAllFundsRoot(t *testing.T) {
}
// insert a correct block
m, chain, err = genBlocks(t, data.genesisSpec, map[int]txn{
m, chain, err = GenerateBlocks(t, data.genesisSpec, map[int]txn{
0: {
getBlockTx(from, to, uint256.NewInt(1000)),
fromKey,
@ -287,7 +287,7 @@ func TestInsertIncorrectStateRootAllFunds(t *testing.T) {
fromKey := data.keys[0]
to := libcommon.Address{1}
m, chain, err := genBlocks(t, data.genesisSpec, map[int]txn{
m, chain, err := GenerateBlocks(t, data.genesisSpec, map[int]txn{
0: {
getBlockTx(from, to, uint256.NewInt(3000)),
fromKey,
@ -312,7 +312,7 @@ func TestInsertIncorrectStateRootAllFunds(t *testing.T) {
}
// insert a correct block
m, chain, err = genBlocks(t, data.genesisSpec, map[int]txn{
m, chain, err = GenerateBlocks(t, data.genesisSpec, map[int]txn{
0: {
getBlockTx(from, to, uint256.NewInt(1000)),
fromKey,
@ -352,7 +352,7 @@ func TestAccountDeployIncorrectRoot(t *testing.T) {
var contractAddress libcommon.Address
eipContract := new(contracts.Testcontract)
m, chain, err := genBlocks(t, data.genesisSpec, map[int]txn{
m, chain, err := GenerateBlocks(t, data.genesisSpec, map[int]txn{
0: {
getBlockTx(from, to, uint256.NewInt(10)),
fromKey,
@ -434,7 +434,7 @@ func TestAccountCreateIncorrectRoot(t *testing.T) {
var contractAddress libcommon.Address
eipContract := new(contracts.Testcontract)
m, chain, err := genBlocks(t, data.genesisSpec, map[int]txn{
m, chain, err := GenerateBlocks(t, data.genesisSpec, map[int]txn{
0: {
getBlockTx(from, to, uint256.NewInt(10)),
fromKey,
@ -514,7 +514,7 @@ func TestAccountUpdateIncorrectRoot(t *testing.T) {
var contractAddress libcommon.Address
eipContract := new(contracts.Testcontract)
m, chain, err := genBlocks(t, data.genesisSpec, map[int]txn{
m, chain, err := GenerateBlocks(t, data.genesisSpec, map[int]txn{
0: {
getBlockTx(from, to, uint256.NewInt(10)),
fromKey,
@ -603,7 +603,7 @@ func TestAccountDeleteIncorrectRoot(t *testing.T) {
var contractAddress libcommon.Address
eipContract := new(contracts.Testcontract)
m, chain, err := genBlocks(t, data.genesisSpec, map[int]txn{
m, chain, err := GenerateBlocks(t, data.genesisSpec, map[int]txn{
0: {
getBlockTx(from, to, uint256.NewInt(10)),
fromKey,
@ -737,7 +737,7 @@ type txn struct {
key *ecdsa.PrivateKey
}
func genBlocks(t *testing.T, gspec *types.Genesis, txs map[int]txn) (*mock.MockSentry, *core.ChainPack, error) {
func GenerateBlocks(t *testing.T, gspec *types.Genesis, txs map[int]txn) (*mock.MockSentry, *core.ChainPack, error) {
key, _ := crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291")
m := mock.MockWithGenesis(t, gspec, key, false)

View File

@ -4,12 +4,13 @@ import (
"bytes"
"context"
"fmt"
"github.com/ledgerwatch/erigon-lib/common/hexutil"
"math/big"
"sync"
"sync/atomic"
"time"
"github.com/ledgerwatch/erigon-lib/common/hexutil"
lru "github.com/hashicorp/golang-lru/v2"
"github.com/holiman/uint256"
"github.com/ledgerwatch/log/v3"
@ -376,9 +377,9 @@ type RPCTransaction struct {
S *hexutil.Big `json:"s"`
}
// newRPCTransaction returns a transaction that will serialize to the RPC
// NewRPCTransaction returns a transaction that will serialize to the RPC
// representation, with the given location metadata set (if available).
func newRPCTransaction(tx types.Transaction, blockHash common.Hash, blockNumber uint64, index uint64, baseFee *big.Int) *RPCTransaction {
func NewRPCTransaction(tx types.Transaction, blockHash common.Hash, blockNumber uint64, index uint64, baseFee *big.Int) *RPCTransaction {
// Determine the signer. For replay-protected transactions, use the most permissive
// signer, because we assume that signers are backwards-compatible with old
// transactions. For non-protected transactions, the homestead signer signer is used
@ -489,7 +490,7 @@ func newRPCPendingTransaction(tx types.Transaction, current *types.Header, confi
if current != nil {
baseFee = misc.CalcBaseFee(config, current)
}
return newRPCTransaction(tx, common.Hash{}, 0, 0, baseFee)
return NewRPCTransaction(tx, common.Hash{}, 0, 0, baseFee)
}
// newRPCRawTransactionFromBlockIndex returns the bytes of a transaction given a block and a transaction index.

View File

@ -3,9 +3,10 @@ package jsonrpc
import (
"bytes"
"context"
"github.com/ledgerwatch/erigon-lib/common/hexutil"
"math/big"
"github.com/ledgerwatch/erigon-lib/common/hexutil"
"github.com/ledgerwatch/erigon-lib/common"
"github.com/ledgerwatch/erigon-lib/common/hexutility"
"github.com/ledgerwatch/erigon-lib/gointerfaces"
@ -76,7 +77,7 @@ func (api *APIImpl) GetTransactionByHash(ctx context.Context, txnHash common.Has
return newRPCBorTransaction(borTx, txnHash, blockHash, blockNum, uint64(len(block.Transactions())), baseFee, chainConfig.ChainID), nil
}
return newRPCTransaction(txn, blockHash, blockNum, txnIndex, baseFee), nil
return NewRPCTransaction(txn, blockHash, blockNum, txnIndex, baseFee), nil
}
curHeader := rawdb.ReadCurrentHeader(tx)
@ -191,7 +192,7 @@ func (api *APIImpl) GetTransactionByBlockHashAndIndex(ctx context.Context, block
return newRPCBorTransaction(borTx, derivedBorTxHash, block.Hash(), block.NumberU64(), uint64(txIndex), block.BaseFee(), chainConfig.ChainID), nil
}
return newRPCTransaction(txs[txIndex], block.Hash(), block.NumberU64(), uint64(txIndex), block.BaseFee()), nil
return NewRPCTransaction(txs[txIndex], block.Hash(), block.NumberU64(), uint64(txIndex), block.BaseFee()), nil
}
// GetRawTransactionByBlockHashAndIndex returns the bytes of the transaction for the given block hash and index.
@ -255,7 +256,7 @@ func (api *APIImpl) GetTransactionByBlockNumberAndIndex(ctx context.Context, blo
return newRPCBorTransaction(borTx, derivedBorTxHash, hash, blockNum, uint64(txIndex), block.BaseFee(), chainConfig.ChainID), nil
}
return newRPCTransaction(txs[txIndex], hash, blockNum, uint64(txIndex), block.BaseFee()), nil
return NewRPCTransaction(txs[txIndex], hash, blockNum, uint64(txIndex), block.BaseFee()), nil
}
// GetRawTransactionByBlockNumberAndIndex returns the bytes of the transaction for the given block number and index.

View File

@ -4,10 +4,11 @@ import (
"context"
"errors"
"fmt"
hexutil2 "github.com/ledgerwatch/erigon-lib/common/hexutil"
"math/big"
"sync"
hexutil2 "github.com/ledgerwatch/erigon-lib/common/hexutil"
"github.com/holiman/uint256"
"github.com/ledgerwatch/log/v3"
@ -324,7 +325,7 @@ func (api *OtterscanAPIImpl) searchTransactionsBeforeV3(tx kv.TemporalTx, ctx co
if err != nil {
return nil, err
}
rpcTx := newRPCTransaction(txn, blockHash, blockNum, uint64(txIndex), header.BaseFee)
rpcTx := NewRPCTransaction(txn, blockHash, blockNum, uint64(txIndex), header.BaseFee)
txs = append(txs, rpcTx)
receipt := &types.Receipt{
Type: txn.Type(), CumulativeGasUsed: res.UsedGas,

View File

@ -95,7 +95,7 @@ func (api *OtterscanAPIImpl) traceBlock(dbtx kv.Tx, ctx context.Context, blockNu
_ = ibs.FinalizeTx(rules, cachedWriter)
if tracer.Found {
rpcTx := newRPCTransaction(tx, block.Hash(), blockNum, uint64(idx), block.BaseFee())
rpcTx := NewRPCTransaction(tx, block.Hash(), blockNum, uint64(idx), block.BaseFee())
mReceipt := marshalReceipt(blockReceipts[idx], tx, chainConfig, block.HeaderNoCopy(), tx.Hash(), true)
mReceipt["timestamp"] = block.Time()
rpcTxs = append(rpcTxs, rpcTx)