Commit fafc3b37 authored by protolambda's avatar protolambda Committed by GitHub

op-node,op-e2e: optimize L1 receipts fetching by supporting different methods (#4494)

* op-node,op-e2e: optimize receipts fetching by supporting different kinds of rpc providers and receipts-fetching method alternatives

* op-node: rpc provider kinds / fetching methods review fixes

* op-e2e: add missing arg - fix lint
parent a1ced6ad
......@@ -169,7 +169,7 @@ func (s *L1Replica) RPCClient() client.RPC {
}
func (s *L1Replica) L1Client(t Testing, cfg *rollup.Config) *sources.L1Client {
l1F, err := sources.NewL1Client(s.RPCClient(), s.log, nil, sources.L1ClientDefaultConfig(cfg, false))
l1F, err := sources.NewL1Client(s.RPCClient(), s.log, nil, sources.L1ClientDefaultConfig(cfg, false, sources.RPCKindBasic))
require.NoError(t, err)
return l1F
}
......
......@@ -39,7 +39,7 @@ func TestL1Replica_ActL1RPCFail(gt *testing.T) {
// mock an RPC failure
replica.ActL1RPCFail(t)
// check RPC failure
l1Cl, err := sources.NewL1Client(replica.RPCClient(), log, nil, sources.L1ClientDefaultConfig(sd.RollupCfg, false))
l1Cl, err := sources.NewL1Client(replica.RPCClient(), log, nil, sources.L1ClientDefaultConfig(sd.RollupCfg, false, sources.RPCKindBasic))
require.NoError(t, err)
_, err = l1Cl.InfoByLabel(t.Ctx(), eth.Unsafe)
require.ErrorContains(t, err, "mock")
......
......@@ -21,7 +21,7 @@ func setupSequencerTest(t Testing, sd *e2eutils.SetupData, log log.Logger) (*L1M
miner := NewL1Miner(t, log, sd.L1Cfg)
l1F, err := sources.NewL1Client(miner.RPCClient(), log, nil, sources.L1ClientDefaultConfig(sd.RollupCfg, false))
l1F, err := sources.NewL1Client(miner.RPCClient(), log, nil, sources.L1ClientDefaultConfig(sd.RollupCfg, false, sources.RPCKindBasic))
require.NoError(t, err)
engine := NewL2Engine(t, log, sd.L2Cfg, sd.RollupCfg.Genesis.L1, jwtPath)
l2Cl, err := sources.NewEngineClient(engine.RPCClient(), log, nil, sources.EngineClientDefaultConfig(sd.RollupCfg))
......
......@@ -285,7 +285,7 @@ func TestRestartOpGeth(gt *testing.T) {
jwtPath := e2eutils.WriteDefaultJWT(t)
// L1
miner := NewL1Miner(t, log, sd.L1Cfg)
l1F, err := sources.NewL1Client(miner.RPCClient(), log, nil, sources.L1ClientDefaultConfig(sd.RollupCfg, false))
l1F, err := sources.NewL1Client(miner.RPCClient(), log, nil, sources.L1ClientDefaultConfig(sd.RollupCfg, false, sources.RPCKindBasic))
require.NoError(t, err)
// Sequencer
seqEng := NewL2Engine(t, log, sd.L2Cfg, sd.RollupCfg.Genesis.L1, jwtPath, dbOption)
......@@ -380,7 +380,7 @@ func TestConflictingL2Blocks(gt *testing.T) {
altSeqEng := NewL2Engine(t, log, sd.L2Cfg, sd.RollupCfg.Genesis.L1, jwtPath)
altSeqEngCl, err := sources.NewEngineClient(altSeqEng.RPCClient(), log, nil, sources.EngineClientDefaultConfig(sd.RollupCfg))
require.NoError(t, err)
l1F, err := sources.NewL1Client(miner.RPCClient(), log, nil, sources.L1ClientDefaultConfig(sd.RollupCfg, false))
l1F, err := sources.NewL1Client(miner.RPCClient(), log, nil, sources.L1ClientDefaultConfig(sd.RollupCfg, false, sources.RPCKindBasic))
require.NoError(t, err)
altSequencer := NewL2Sequencer(t, log, l1F, altSeqEngCl, sd.RollupCfg, 0)
altBatcher := NewL2Batcher(log, sd.RollupCfg, &BatcherCfg{
......
......@@ -12,6 +12,7 @@ import (
"time"
bss "github.com/ethereum-optimism/optimism/op-batcher/batcher"
"github.com/ethereum-optimism/optimism/op-node/sources"
l2os "github.com/ethereum-optimism/optimism/op-proposer/proposer"
oplog "github.com/ethereum-optimism/optimism/op-service/log"
......@@ -268,6 +269,7 @@ func TestMigration(t *testing.T) {
L1: &node.L1EndpointConfig{
L1NodeAddr: forkedL1URL,
L1TrustRPC: false,
L1RPCKind: sources.RPCKindBasic,
},
L2: &node.L2EndpointConfig{
L2EngineAddr: gethNode.HTTPAuthEndpoint(),
......
......@@ -32,6 +32,7 @@ import (
"github.com/ethereum-optimism/optimism/op-node/p2p"
"github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-node/rollup/driver"
"github.com/ethereum-optimism/optimism/op-node/sources"
"github.com/ethereum-optimism/optimism/op-node/testlog"
l2os "github.com/ethereum-optimism/optimism/op-proposer/proposer"
oplog "github.com/ethereum-optimism/optimism/op-service/log"
......@@ -368,6 +369,7 @@ func (cfg SystemConfig) Start() (*System, error) {
rollupCfg.L1 = &rollupNode.L1EndpointConfig{
L1NodeAddr: l1EndpointConfig,
L1TrustRPC: false,
L1RPCKind: sources.RPCKindBasic,
}
rollupCfg.L2 = &rollupNode.L2EndpointConfig{
L2EngineAddr: l2EndpointConfig,
......
......@@ -2,9 +2,7 @@ package eth
import (
"bytes"
"context"
"fmt"
"io"
"math/big"
"reflect"
......@@ -281,42 +279,6 @@ type ForkchoiceUpdatedResult struct {
PayloadID *PayloadID `json:"payloadId"`
}
// ReceiptsFetcher fetches receipts of a block,
// and enables the caller to parallelize fetching and backoff on fetching errors as needed.
type ReceiptsFetcher interface {
// Reset clears the previously fetched results for a fresh re-attempt.
Reset()
// Fetch retrieves receipts in batches, until it returns io.EOF to indicate completion.
Fetch(ctx context.Context) error
// Complete indicates when all data has been fetched.
Complete() bool
// Result returns the receipts, or an error if the Fetch-ing is not Complete,
// or an error if the results are invalid.
// If an error is returned, the fetcher is Reset automatically.
Result() (types.Receipts, error)
}
// FetchedReceipts is a simple util to implement the ReceiptsFetcher with readily available receipts.
type FetchedReceipts types.Receipts
func (f FetchedReceipts) Reset() {
// nothing to reset
}
func (f FetchedReceipts) Fetch(ctx context.Context) error {
return io.EOF
}
func (f FetchedReceipts) Complete() bool {
return true
}
func (f FetchedReceipts) Result() (types.Receipts, error) {
return types.Receipts(f), nil
}
var _ ReceiptsFetcher = (FetchedReceipts)(nil)
// SystemConfig represents the rollup system configuration that carries over in every L2 block,
// and may be changed through L1 system config events.
// The initial SystemConfig at rollup genesis is embedded in the rollup configuration.
......
......@@ -6,6 +6,7 @@ import (
"time"
"github.com/ethereum-optimism/optimism/op-node/chaincfg"
"github.com/ethereum-optimism/optimism/op-node/sources"
"github.com/urfave/cli"
)
......@@ -63,6 +64,16 @@ var (
Usage: "Trust the L1 RPC, sync faster at risk of malicious/buggy RPC providing bad or inconsistent L1 data",
EnvVar: prefixEnvVar("L1_TRUST_RPC"),
}
L1RPCProviderKind = cli.GenericFlag{
Name: "l1.rpckind",
Usage: "The kind of RPC provider, used to inform optimal transactions receipts fetching, and thus reduce costs. Valid options: " +
EnumString[sources.RPCProviderKind](sources.RPCProviderKinds),
EnvVar: prefixEnvVar("L1_RPC_KIND"),
Value: func() *sources.RPCProviderKind {
out := sources.RPCKindBasic
return &out
}(),
}
L2EngineJWTSecret = cli.StringFlag{
Name: "l2.jwt-secret",
Usage: "Path to JWT secret key. Keys are 32 bytes, hex encoded in a file. A new key will be generated if left empty.",
......@@ -182,6 +193,7 @@ var optionalFlags = append([]cli.Flag{
RollupConfig,
Network,
L1TrustRPC,
L1RPCProviderKind,
L2EngineJWTSecret,
VerifierL1Confs,
SequencerEnabledFlag,
......
package flags
import (
"fmt"
"strings"
)
func EnumString[T fmt.Stringer](values []T) string {
var out strings.Builder
for i, v := range values {
out.WriteString(v.String())
if i+1 < len(values) {
out.WriteString(", ")
}
}
return out.String()
}
......@@ -6,6 +6,8 @@ import (
"fmt"
"github.com/ethereum-optimism/optimism/op-node/client"
"github.com/ethereum-optimism/optimism/op-node/sources"
"github.com/ethereum/go-ethereum/log"
gn "github.com/ethereum/go-ethereum/node"
"github.com/ethereum/go-ethereum/rpc"
......@@ -19,7 +21,9 @@ type L2EndpointSetup interface {
type L1EndpointSetup interface {
// Setup a RPC client to a L1 node to pull rollup input-data from.
Setup(ctx context.Context, log log.Logger) (cl client.RPC, trust bool, err error)
// The results of the RPC client may be trusted for faster processing, or strictly validated.
// The kind of the RPC may be non-basic, to optimize RPC usage.
Setup(ctx context.Context, log log.Logger) (cl client.RPC, trust bool, kind sources.RPCProviderKind, err error)
}
type L2EndpointConfig struct {
......@@ -78,26 +82,31 @@ type L1EndpointConfig struct {
// against block hashes, or cached transaction sender addresses.
// Thus we can sync faster at the risk of the source RPC being wrong.
L1TrustRPC bool
// L1RPCKind identifies the RPC provider kind that serves the RPC,
// to inform the optimal usage of the RPC for transaction receipts fetching.
L1RPCKind sources.RPCProviderKind
}
var _ L1EndpointSetup = (*L1EndpointConfig)(nil)
func (cfg *L1EndpointConfig) Setup(ctx context.Context, log log.Logger) (cl client.RPC, trust bool, err error) {
func (cfg *L1EndpointConfig) Setup(ctx context.Context, log log.Logger) (cl client.RPC, trust bool, kind sources.RPCProviderKind, err error) {
l1Node, err := client.NewRPC(ctx, log, cfg.L1NodeAddr)
if err != nil {
return nil, false, fmt.Errorf("failed to dial L1 address (%s): %w", cfg.L1NodeAddr, err)
return nil, false, sources.RPCKindBasic, fmt.Errorf("failed to dial L1 address (%s): %w", cfg.L1NodeAddr, err)
}
return l1Node, cfg.L1TrustRPC, nil
return l1Node, cfg.L1TrustRPC, cfg.L1RPCKind, nil
}
// PreparedL1Endpoint enables testing with an in-process pre-setup RPC connection to L1
type PreparedL1Endpoint struct {
Client client.RPC
TrustRPC bool
Client client.RPC
TrustRPC bool
RPCProviderKind sources.RPCProviderKind
}
var _ L1EndpointSetup = (*PreparedL1Endpoint)(nil)
func (p *PreparedL1Endpoint) Setup(ctx context.Context, log log.Logger) (cl client.RPC, trust bool, err error) {
return p.Client, p.TrustRPC, nil
func (p *PreparedL1Endpoint) Setup(ctx context.Context, log log.Logger) (cl client.RPC, trust bool, kind sources.RPCProviderKind, err error) {
return p.Client, p.TrustRPC, p.RPCProviderKind, nil
}
......@@ -112,14 +112,14 @@ func (n *OpNode) initTracer(ctx context.Context, cfg *Config) error {
}
func (n *OpNode) initL1(ctx context.Context, cfg *Config) error {
l1Node, trustRPC, err := cfg.L1.Setup(ctx, n.log)
l1Node, trustRPC, rpcProvKind, err := cfg.L1.Setup(ctx, n.log)
if err != nil {
return fmt.Errorf("failed to get L1 RPC client: %w", err)
}
n.l1Source, err = sources.NewL1Client(
client.NewInstrumentedRPC(l1Node, n.metrics), n.log, n.metrics.L1SourceCache,
sources.L1ClientDefaultConfig(&cfg.Rollup, trustRPC))
sources.L1ClientDefaultConfig(&cfg.Rollup, trustRPC, rpcProvKind))
if err != nil {
return fmt.Errorf("failed to create L1 source: %w", err)
}
......
......@@ -9,17 +9,19 @@ import (
"strings"
"github.com/ethereum-optimism/optimism/op-node/chaincfg"
"github.com/ethereum-optimism/optimism/op-node/sources"
"github.com/urfave/cli"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum-optimism/optimism/op-node/flags"
"github.com/ethereum-optimism/optimism/op-node/node"
"github.com/ethereum-optimism/optimism/op-node/p2p"
"github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-node/rollup/driver"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/log"
)
// NewConfig creates a Config from the provided flags or environment variables.
......@@ -97,6 +99,7 @@ func NewL1EndpointConfig(ctx *cli.Context) (*node.L1EndpointConfig, error) {
return &node.L1EndpointConfig{
L1NodeAddr: ctx.GlobalString(flags.L1NodeAddr.Name),
L1TrustRPC: ctx.GlobalBool(flags.L1TrustRPC.Name),
L1RPCKind: sources.RPCProviderKind(strings.ToLower(ctx.GlobalString(flags.L1RPCProviderKind.Name))),
}, nil
}
......
......@@ -15,7 +15,7 @@ import (
// IterativeBatchCall is an util to create a job to fetch many RPC requests in batches,
// and enable the caller to parallelize easily and safely, handle and re-try errors,
// and pick a batch size all by simply calling Fetch again and again until it returns io.EOF.
type IterativeBatchCall[K any, V any, O any] struct {
type IterativeBatchCall[K any, V any] struct {
completed uint32 // tracks how far to completing all requests we are
resetLock sync.RWMutex // ensures we do not concurrently read (incl. fetch) / reset
......@@ -23,23 +23,19 @@ type IterativeBatchCall[K any, V any, O any] struct {
batchSize int
makeRequest func(K) (V, rpc.BatchElem)
makeResults func([]K, []V) (O, error)
getBatch BatchCallContextFn
requestsValues []V
scheduled chan rpc.BatchElem
results *O
}
// NewIterativeBatchCall constructs a batch call, fetching the values with the given keys,
// and transforms them into a verified final result.
func NewIterativeBatchCall[K any, V any, O any](
func NewIterativeBatchCall[K any, V any](
requestsKeys []K,
makeRequest func(K) (V, rpc.BatchElem),
makeResults func([]K, []V) (O, error),
getBatch BatchCallContextFn,
batchSize int) *IterativeBatchCall[K, V, O] {
batchSize int) *IterativeBatchCall[K, V] {
if len(requestsKeys) < batchSize {
batchSize = len(requestsKeys)
......@@ -48,20 +44,19 @@ func NewIterativeBatchCall[K any, V any, O any](
batchSize = 1
}
out := &IterativeBatchCall[K, V, O]{
out := &IterativeBatchCall[K, V]{
completed: 0,
getBatch: getBatch,
requestsKeys: requestsKeys,
batchSize: batchSize,
makeRequest: makeRequest,
makeResults: makeResults,
}
out.Reset()
return out
}
// Reset will clear the batch call, to start fetching all contents from scratch.
func (ibc *IterativeBatchCall[K, V, O]) Reset() {
func (ibc *IterativeBatchCall[K, V]) Reset() {
ibc.resetLock.Lock()
defer ibc.resetLock.Unlock()
......@@ -85,7 +80,7 @@ func (ibc *IterativeBatchCall[K, V, O]) Reset() {
// This method is safe to call concurrently: it will parallelize the fetching work.
// If no work is available, but the fetching is not done yet,
// then Fetch will block until the next thing can be fetched, or until the context expires.
func (ibc *IterativeBatchCall[K, V, O]) Fetch(ctx context.Context) error {
func (ibc *IterativeBatchCall[K, V]) Fetch(ctx context.Context) error {
ibc.resetLock.RLock()
defer ibc.resetLock.RUnlock()
......@@ -150,7 +145,7 @@ func (ibc *IterativeBatchCall[K, V, O]) Fetch(ctx context.Context) error {
}
// Complete indicates if the batch call is done.
func (ibc *IterativeBatchCall[K, V, O]) Complete() bool {
func (ibc *IterativeBatchCall[K, V]) Complete() bool {
ibc.resetLock.RLock()
defer ibc.resetLock.RUnlock()
return atomic.LoadUint32(&ibc.completed) >= uint32(len(ibc.requestsKeys))
......@@ -158,27 +153,12 @@ func (ibc *IterativeBatchCall[K, V, O]) Complete() bool {
// Result returns the fetched values, checked and transformed to the final output type, if available.
// If the check fails, the IterativeBatchCall will Reset itself, to be ready for a re-attempt in fetching new data.
func (ibc *IterativeBatchCall[K, V, O]) Result() (O, error) {
func (ibc *IterativeBatchCall[K, V]) Result() ([]V, error) {
ibc.resetLock.RLock()
if atomic.LoadUint32(&ibc.completed) < uint32(len(ibc.requestsKeys)) {
ibc.resetLock.RUnlock()
return *new(O), fmt.Errorf("results not available yet, Fetch more first")
}
if ibc.results != nil {
ibc.resetLock.RUnlock()
return *ibc.results, nil
return nil, fmt.Errorf("results not available yet, Fetch more first")
}
out, err := ibc.makeResults(ibc.requestsKeys, ibc.requestsValues)
ibc.resetLock.RUnlock()
if err != nil {
// start over
ibc.Reset()
} else {
// cache the valid results
ibc.resetLock.Lock()
ibc.results = &out
ibc.resetLock.Unlock()
}
return out, err
return ibc.requestsValues, nil
}
......@@ -49,12 +49,6 @@ func makeTestRequest(i int) (*string, rpc.BatchElem) {
}
}
func makeTestResults() func(keys []int, values []*string) ([]*string, error) {
return func(keys []int, values []*string) ([]*string, error) {
return values, nil
}
}
func (tc *batchTestCase) GetBatch(ctx context.Context, b []rpc.BatchElem) error {
if ctx.Err() != nil {
return ctx.Err()
......@@ -103,7 +97,7 @@ func (tc *batchTestCase) Run(t *testing.T) {
tc.On("get", batch).Once().Run(makeMock(bci, bc)).Return([]error{bc.rpcErr}) // wrap to preserve nil as type of error
}
}
iter := NewIterativeBatchCall[int, *string, []*string](keys, makeTestRequest, makeTestResults(), tc.GetBatch, tc.batchSize)
iter := NewIterativeBatchCall[int, *string](keys, makeTestRequest, tc.GetBatch, tc.batchSize)
for i, bc := range tc.batchCalls {
ctx := context.Background()
if bc.makeCtx != nil {
......
......@@ -3,7 +3,6 @@ package sources
import (
"context"
"fmt"
"io"
"github.com/ethereum/go-ethereum"
"github.com/ethereum/go-ethereum/common"
......@@ -44,6 +43,9 @@ type EthClientConfig struct {
// If this is not checked, disabled header fields like the nonce or difficulty
// may be used to get a different block-hash.
MustBePostMerge bool
// RPCProviderKind is a hint at what type of RPC provider we are dealing with
RPCProviderKind RPCProviderKind
}
func (c *EthClientConfig) Check() error {
......@@ -65,6 +67,9 @@ func (c *EthClientConfig) Check() error {
if c.MaxRequestsPerBatch < 1 {
return fmt.Errorf("expected at least 1 request per batch, but max is: %d", c.MaxRequestsPerBatch)
}
if !ValidRPCProviderKind(c.RPCProviderKind) {
return fmt.Errorf("unknown rpc provider kind: %s", c.RPCProviderKind)
}
return nil
}
......@@ -78,11 +83,13 @@ type EthClient struct {
mustBePostMerge bool
provKind RPCProviderKind
log log.Logger
// cache receipts in bundles per block hash
// We cache the receipts fetcher to not lose progress when we have to retry the `Fetch` call
// common.Hash -> eth.ReceiptsFetcher
// We cache the receipts fetching job to not lose progress when we have to retry the `Fetch` call
// common.Hash -> *receiptsFetchingJob
receiptsCache *caching.LRUCache
// cache transactions in bundles per block hash
......@@ -96,6 +103,27 @@ type EthClient struct {
// cache payloads by hash
// common.Hash -> *eth.ExecutionPayload
payloadsCache *caching.LRUCache
// availableReceiptMethods tracks which receipt methods can be used for fetching receipts
// This may be modified concurrently, but we don't lock since it's a single
// uint64 that's not critical (fine to miss or mix up a modification)
availableReceiptMethods ReceiptsFetchingMethod
}
func (s *EthClient) PickReceiptsMethod(txCount uint64) ReceiptsFetchingMethod {
return PickBestReceiptsFetchingMethod(s.provKind, s.availableReceiptMethods, txCount)
}
func (s *EthClient) OnReceiptsMethodErr(m ReceiptsFetchingMethod, err error) {
if unusableMethod(err) {
// clear the bit of the method that errored
s.availableReceiptMethods &^= m
s.log.Warn("failed to use selected RPC method for receipt fetching, falling back to alternatives",
"provider_kind", s.provKind, "failed_method", m, "fallback", s.availableReceiptMethods, "err", err)
} else {
s.log.Debug("failed to use selected RPC method for receipt fetching, but method does appear to be available, so we continue to use it",
"provider_kind", s.provKind, "failed_method", m, "fallback", s.availableReceiptMethods&^m, "err", err)
}
}
// NewEthClient wraps a RPC with bindings to fetch ethereum data,
......@@ -106,14 +134,17 @@ func NewEthClient(client client.RPC, log log.Logger, metrics caching.Metrics, co
}
client = LimitRPC(client, config.MaxConcurrentRequests)
return &EthClient{
client: client,
maxBatchSize: config.MaxRequestsPerBatch,
trustRPC: config.TrustRPC,
log: log,
receiptsCache: caching.NewLRUCache(metrics, "receipts", config.ReceiptsCacheSize),
transactionsCache: caching.NewLRUCache(metrics, "txs", config.TransactionsCacheSize),
headersCache: caching.NewLRUCache(metrics, "headers", config.HeadersCacheSize),
payloadsCache: caching.NewLRUCache(metrics, "payloads", config.PayloadsCacheSize),
client: client,
maxBatchSize: config.MaxRequestsPerBatch,
trustRPC: config.TrustRPC,
mustBePostMerge: config.MustBePostMerge,
provKind: config.RPCProviderKind,
log: log,
receiptsCache: caching.NewLRUCache(metrics, "receipts", config.ReceiptsCacheSize),
transactionsCache: caching.NewLRUCache(metrics, "txs", config.TransactionsCacheSize),
headersCache: caching.NewLRUCache(metrics, "headers", config.HeadersCacheSize),
payloadsCache: caching.NewLRUCache(metrics, "payloads", config.PayloadsCacheSize),
availableReceiptMethods: AvailableReceiptsFetchingMethods(config.RPCProviderKind),
}, nil
}
......@@ -238,26 +269,18 @@ func (s *EthClient) FetchReceipts(ctx context.Context, blockHash common.Hash) (e
// Try to reuse the receipts fetcher because is caches the results of intermediate calls. This means
// that if just one of many calls fail, we only retry the failed call rather than all of the calls.
// The underlying fetcher uses the receipts hash to verify receipt integrity.
var fetcher eth.ReceiptsFetcher
var job *receiptsFetchingJob
if v, ok := s.receiptsCache.Get(blockHash); ok {
fetcher = v.(eth.ReceiptsFetcher)
job = v.(*receiptsFetchingJob)
} else {
txHashes := make([]common.Hash, len(txs))
for i := 0; i < len(txs); i++ {
txHashes[i] = txs[i].Hash()
}
fetcher = NewReceiptsFetcher(eth.ToBlockID(info), info.ReceiptHash(), txHashes, s.client.BatchCallContext, s.maxBatchSize)
s.receiptsCache.Add(blockHash, fetcher)
}
// Fetch all receipts
for {
if err := fetcher.Fetch(ctx); err == io.EOF {
break
} else if err != nil {
return nil, nil, err
}
job = NewReceiptsFetchingJob(s, s.client, s.maxBatchSize, eth.ToBlockID(info), info.ReceiptHash(), txHashes)
s.receiptsCache.Add(blockHash, job)
}
receipts, err := fetcher.Result()
receipts, err := job.Fetch(ctx)
if err != nil {
return nil, nil, err
}
......
......@@ -52,6 +52,7 @@ var testEthClientConfig = &EthClientConfig{
MaxConcurrentRequests: 10,
TrustRPC: false,
MustBePostMerge: false,
RPCProviderKind: RPCKindBasic,
}
func randHash() (out common.Hash) {
......@@ -132,7 +133,7 @@ func TestEthClient_InfoByNumber(t *testing.T) {
"eth_getBlockByNumber", []any{n.String(), false}).Run(func(args mock.Arguments) {
*args[1].(**rpcHeader) = rhdr
}).Return([]error{nil})
s, err := NewL1Client(m, nil, nil, L1ClientDefaultConfig(&rollup.Config{SeqWindowSize: 10}, true))
s, err := NewL1Client(m, nil, nil, L1ClientDefaultConfig(&rollup.Config{SeqWindowSize: 10}, true, RPCKindBasic))
require.NoError(t, err)
info, err := s.InfoByNumber(ctx, uint64(n))
require.NoError(t, err)
......
......@@ -5,13 +5,14 @@ import (
"fmt"
"strings"
"github.com/ethereum/go-ethereum"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum-optimism/optimism/op-node/client"
"github.com/ethereum-optimism/optimism/op-node/eth"
"github.com/ethereum-optimism/optimism/op-node/rollup"
"github.com/ethereum-optimism/optimism/op-node/sources/caching"
"github.com/ethereum/go-ethereum"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/log"
)
type L1ClientConfig struct {
......@@ -20,7 +21,7 @@ type L1ClientConfig struct {
L1BlockRefsCacheSize int
}
func L1ClientDefaultConfig(config *rollup.Config, trustRPC bool) *L1ClientConfig {
func L1ClientDefaultConfig(config *rollup.Config, trustRPC bool, kind RPCProviderKind) *L1ClientConfig {
// Cache 3/2 worth of sequencing window of receipts and txs
span := int(config.SeqWindowSize) * 3 / 2
if span > 1000 { // sanity cap. If a large sequencing window is configured, do not make the cache too large
......@@ -37,6 +38,7 @@ func L1ClientDefaultConfig(config *rollup.Config, trustRPC bool) *L1ClientConfig
MaxConcurrentRequests: 10,
TrustRPC: trustRPC,
MustBePostMerge: false,
RPCProviderKind: kind,
},
L1BlockRefsCacheSize: span,
}
......
......@@ -48,6 +48,7 @@ func L2ClientDefaultConfig(config *rollup.Config, trustRPC bool) *L2ClientConfig
MaxConcurrentRequests: 10,
TrustRPC: trustRPC,
MustBePostMerge: true,
RPCProviderKind: RPCKindBasic,
},
L2BlockRefsCacheSize: span,
L1ConfigsCacheSize: span,
......
package sources
import (
"context"
"fmt"
"io"
"math/big"
"sync"
"github.com/ethereum-optimism/optimism/op-node/eth"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rpc"
"github.com/ethereum/go-ethereum/trie"
"github.com/ethereum-optimism/optimism/op-node/eth"
)
func makeReceiptsFn(block eth.BlockID, receiptHash common.Hash) func(txHashes []common.Hash, receipts []*types.Receipt) (types.Receipts, error) {
return func(txHashes []common.Hash, receipts []*types.Receipt) (types.Receipts, error) {
if len(receipts) != len(txHashes) {
return nil, fmt.Errorf("got %d receipts but expected %d", len(receipts), len(txHashes))
func validateReceipts(block eth.BlockID, receiptHash common.Hash, txHashes []common.Hash, receipts []*types.Receipt) error {
if len(receipts) != len(txHashes) {
return fmt.Errorf("got %d receipts but expected %d", len(receipts), len(txHashes))
}
if len(txHashes) == 0 {
if receiptHash != types.EmptyRootHash {
return fmt.Errorf("no transactions, but got non-empty receipt trie root: %s", receiptHash)
}
if len(txHashes) == 0 {
if receiptHash != types.EmptyRootHash {
return nil, fmt.Errorf("no transactions, but got non-empty receipt trie root: %s", receiptHash)
}
}
// We don't trust the RPC to provide consistent cached receipt info that we use for critical rollup derivation work.
// Let's check everything quickly.
logIndex := uint(0)
for i, r := range receipts {
if r == nil { // on reorgs or other cases the receipts may disappear before they can be retrieved.
return fmt.Errorf("receipt of tx %d returns nil on retrieval", i)
}
// We don't trust the RPC to provide consistent cached receipt info that we use for critical rollup derivation work.
// Let's check everything quickly.
logIndex := uint(0)
for i, r := range receipts {
if r == nil { // on reorgs or other cases the receipts may disappear before they can be retrieved.
return nil, fmt.Errorf("receipt of tx %d returns nil on retrieval", i)
if r.TransactionIndex != uint(i) {
return fmt.Errorf("receipt %d has unexpected tx index %d", i, r.TransactionIndex)
}
if r.BlockNumber == nil {
return fmt.Errorf("receipt %d has unexpected nil block number, expected %d", i, block.Number)
}
if r.BlockNumber.Uint64() != block.Number {
return fmt.Errorf("receipt %d has unexpected block number %d, expected %d", i, r.BlockNumber, block.Number)
}
if r.BlockHash != block.Hash {
return fmt.Errorf("receipt %d has unexpected block hash %s, expected %s", i, r.BlockHash, block.Hash)
}
for j, log := range r.Logs {
if log.Index != logIndex {
return fmt.Errorf("log %d (%d of tx %d) has unexpected log index %d", logIndex, j, i, log.Index)
}
if r.TransactionIndex != uint(i) {
return nil, fmt.Errorf("receipt %d has unexpected tx index %d", i, r.TransactionIndex)
if log.TxIndex != uint(i) {
return fmt.Errorf("log %d has unexpected tx index %d", log.Index, log.TxIndex)
}
if r.BlockNumber == nil {
return nil, fmt.Errorf("receipt %d has unexpected nil block number, expected %d", i, block.Number)
if log.BlockHash != block.Hash {
return fmt.Errorf("log %d of block %s has unexpected block hash %s", log.Index, block.Hash, log.BlockHash)
}
if r.BlockNumber.Uint64() != block.Number {
return nil, fmt.Errorf("receipt %d has unexpected block number %d, expected %d", i, r.BlockNumber, block.Number)
if log.BlockNumber != block.Number {
return fmt.Errorf("log %d of block %d has unexpected block number %d", log.Index, block.Number, log.BlockNumber)
}
if r.BlockHash != block.Hash {
return nil, fmt.Errorf("receipt %d has unexpected block hash %s, expected %s", i, r.BlockHash, block.Hash)
if log.TxHash != txHashes[i] {
return fmt.Errorf("log %d of tx %s has unexpected tx hash %s", log.Index, txHashes[i], log.TxHash)
}
for j, log := range r.Logs {
if log.Index != logIndex {
return nil, fmt.Errorf("log %d (%d of tx %d) has unexpected log index %d", logIndex, j, i, log.Index)
}
if log.TxIndex != uint(i) {
return nil, fmt.Errorf("log %d has unexpected tx index %d", log.Index, log.TxIndex)
}
if log.BlockHash != block.Hash {
return nil, fmt.Errorf("log %d of block %s has unexpected block hash %s", log.Index, block.Hash, log.BlockHash)
}
if log.BlockNumber != block.Number {
return nil, fmt.Errorf("log %d of block %d has unexpected block number %d", log.Index, block.Number, log.BlockNumber)
}
if log.TxHash != txHashes[i] {
return nil, fmt.Errorf("log %d of tx %s has unexpected tx hash %s", log.Index, txHashes[i], log.TxHash)
}
if log.Removed {
return nil, fmt.Errorf("canonical log (%d) must never be removed due to reorg", log.Index)
}
logIndex++
if log.Removed {
return fmt.Errorf("canonical log (%d) must never be removed due to reorg", log.Index)
}
logIndex++
}
// Note: 3 non-consensus L1 receipt fields are ignored:
// PostState - not part of L1 ethereum anymore since EIP 658 (part of Byzantium)
// ContractAddress - we do not care about contract deployments
// GasUsed - we do not care about L1 gas usage of txs
// And Optimism L1 fee meta-data in the receipt is ignored as well
}
// Sanity-check: external L1-RPC sources are notorious for not returning all receipts,
// or returning them out-of-order. Verify the receipts against the expected receipt-hash.
hasher := trie.NewStackTrie(nil)
computed := types.DeriveSha(types.Receipts(receipts), hasher)
if receiptHash != computed {
return nil, fmt.Errorf("failed to fetch list of receipts: expected receipt root %s but computed %s from retrieved receipts", receiptHash, computed)
}
return receipts, nil
// Sanity-check: external L1-RPC sources are notorious for not returning all receipts,
// or returning them out-of-order. Verify the receipts against the expected receipt-hash.
hasher := trie.NewStackTrie(nil)
computed := types.DeriveSha(types.Receipts(receipts), hasher)
if receiptHash != computed {
return fmt.Errorf("failed to fetch list of receipts: expected receipt root %s but computed %s from retrieved receipts", receiptHash, computed)
}
return nil
}
func makeReceiptRequest(txHash common.Hash) (*types.Receipt, rpc.BatchElem) {
......@@ -82,13 +91,394 @@ func makeReceiptRequest(txHash common.Hash) (*types.Receipt, rpc.BatchElem) {
}
}
// NewReceiptsFetcher creates a receipt fetcher that can iteratively fetch the receipts matching the given txs.
func NewReceiptsFetcher(block eth.BlockID, receiptHash common.Hash, txHashes []common.Hash, getBatch BatchCallContextFn, batchSize int) eth.ReceiptsFetcher {
return NewIterativeBatchCall[common.Hash, *types.Receipt, types.Receipts](
txHashes,
makeReceiptRequest,
makeReceiptsFn(block, receiptHash),
getBatch,
batchSize,
)
// Cost break-down sources:
// Alchemy: https://docs.alchemy.com/reference/compute-units
// QuickNode: https://www.quicknode.com/docs/ethereum/api_credits
// Infura: no pricing table available.
//
// Receipts are encoded the same everywhere:
//
// blockHash, blockNumber, transactionIndex, transactionHash, from, to, cumulativeGasUsed, gasUsed,
// contractAddress, logs, logsBloom, status, effectiveGasPrice, type.
//
// Note that Alchemy/Geth still have a "root" field for legacy reasons,
// but ethereum does not compute state-roots per tx anymore, so quicknode and others do not serve this data.
// RPCProviderKind identifies an RPC provider, used to hint at the optimal receipt fetching approach.
type RPCProviderKind string
const (
RPCKindAlchemy RPCProviderKind = "alchemy"
RPCKindQuickNode RPCProviderKind = "quicknode"
RPCKindInfura RPCProviderKind = "infura"
RPCKindParity RPCProviderKind = "parity"
RPCKindNethermind RPCProviderKind = "nethermind"
RPCKindDebugGeth RPCProviderKind = "debug_geth"
RPCKindErigon RPCProviderKind = "erigon"
RPCKindBasic RPCProviderKind = "basic" // try only the standard most basic receipt fetching
RPCKindAny RPCProviderKind = "any" // try any method available
)
var RPCProviderKinds = []RPCProviderKind{
RPCKindAlchemy,
RPCKindQuickNode,
RPCKindInfura,
RPCKindParity,
RPCKindNethermind,
RPCKindDebugGeth,
RPCKindErigon,
RPCKindBasic,
RPCKindAny,
}
func (kind RPCProviderKind) String() string {
return string(kind)
}
func (kind *RPCProviderKind) Set(value string) error {
if !ValidRPCProviderKind(RPCProviderKind(value)) {
return fmt.Errorf("unknown rpc kind: %q", value)
}
*kind = RPCProviderKind(value)
return nil
}
func ValidRPCProviderKind(value RPCProviderKind) bool {
for _, k := range RPCProviderKinds {
if k == value {
return true
}
}
return false
}
// ReceiptsFetchingMethod is a bitfield with 1 bit for each receipts fetching type.
// Depending on errors, tx counts and preferences the code may select different sets of fetching methods.
type ReceiptsFetchingMethod uint64
func (r ReceiptsFetchingMethod) String() string {
out := ""
x := r
addMaybe := func(m ReceiptsFetchingMethod, v string) {
if x&m != 0 {
out += v
x ^= x & m
}
if x != 0 { // add separator if there are entries left
out += ", "
}
}
addMaybe(EthGetTransactionReceiptBatch, "eth_getTransactionReceipt (batched)")
addMaybe(AlchemyGetTransactionReceipts, "alchemy_getTransactionReceipts")
addMaybe(DebugGetRawReceipts, "debug_getRawReceipts")
addMaybe(ParityGetBlockReceipts, "parity_getBlockReceipts")
addMaybe(EthGetBlockReceipts, "eth_getBlockReceipts")
addMaybe(^ReceiptsFetchingMethod(0), "unknown") // if anything is left, describe it as unknown
return out
}
const (
// EthGetTransactionReceiptBatch is standard per-tx receipt fetching with JSON-RPC batches.
// Available in: standard, everywhere.
// - Alchemy: 15 CU / tx
// - Quicknode: 2 credits / tx
// Method: eth_getTransactionReceipt
// See: https://ethereum.github.io/execution-apis/api-documentation/
EthGetTransactionReceiptBatch ReceiptsFetchingMethod = 1 << iota
// AlchemyGetTransactionReceipts is a special receipt fetching method provided by Alchemy.
// Available in:
// - Alchemy: 250 CU total
// Method: alchemy_getTransactionReceipts
// Params:
// - object with "blockNumber" or "blockHash" field
// Returns: "array of receipts" - docs lie, array is wrapped in a struct with single "receipts" field
// See: https://docs.alchemy.com/reference/alchemy-gettransactionreceipts#alchemy_gettransactionreceipts
AlchemyGetTransactionReceipts
// DebugGetRawReceipts is a debug method from Geth, faster by avoiding serialization and metadata overhead.
// Ideal for fast syncing from a local geth node.
// Available in:
// - Geth: free
// - QuickNode: 22 credits maybe? Unknown price, undocumented ("debug_getblockreceipts" exists in table though?)
// Method: debug_getRawReceipts
// Params:
// - string presenting a block number or hash
// Returns: list of strings, hex encoded RLP of receipts data. "consensus-encoding of all receipts in a single block"
// See: https://geth.ethereum.org/docs/rpc/ns-debug#debug_getrawreceipts
DebugGetRawReceipts
// ParityGetBlockReceipts is an old parity method, which has been adopted by Nethermind and some RPC providers.
// Available in:
// - Alchemy: 500 CU total
// - QuickNode: 59 credits - docs are wrong, not actually available anymore.
// - Any open-ethereum/parity legacy: free
// - Nethermind: free
// Method: parity_getBlockReceipts
// Params:
// Parity: "quantity or tag"
// Alchemy: string with block hash, number in hex, or block tag.
// Nethermind: very flexible: tag, number, hex or object with "requireCanonical"/"blockHash" fields.
// Returns: array of receipts
// See:
// - Parity: https://openethereum.github.io/JSONRPC-parity-module#parity_getblockreceipts
// - QuickNode: undocumented.
// - Alchemy: https://docs.alchemy.com/reference/eth-getblockreceipts
// - Nethermind: https://docs.nethermind.io/nethermind/ethereum-client/json-rpc/parity#parity_getblockreceipts
ParityGetBlockReceipts
// EthGetBlockReceipts is a non-standard receipt fetching method in the eth namespace,
// supported by some RPC platforms and Erigon.
// Available in:
// - Alchemy: 500 CU total (and deprecated)
// - Erigon: free
// - QuickNode: 59 credits total (does not seem to work with block hash arg, inaccurate docs)
// Method: eth_getBlockReceipts
// Params:
// - QuickNode: string, "quantity or tag", docs say incl. block hash, but API does not actually accept it.
// - Alchemy: string, block hash / num (hex) / block tag
// Returns: array of receipts
// See:
// - QuickNode: https://www.quicknode.com/docs/ethereum/eth_getBlockReceipts
// - Alchemy: https://docs.alchemy.com/reference/eth-getblockreceipts
EthGetBlockReceipts
// Other:
// - 250 credits, not supported, strictly worse than other options. In quicknode price-table.
// qn_getBlockWithReceipts - in price table, ? undocumented, but in quicknode "Single Flight RPC" description
// qn_getReceipts - in price table, ? undocumented, but in quicknode "Single Flight RPC" description
// debug_getBlockReceipts - ? undocumented, shows up in quicknode price table, not available.
)
// AvailableReceiptsFetchingMethods selects receipt fetching methods based on the RPC provider kind.
func AvailableReceiptsFetchingMethods(kind RPCProviderKind) ReceiptsFetchingMethod {
switch kind {
case RPCKindAlchemy:
return AlchemyGetTransactionReceipts | EthGetTransactionReceiptBatch
case RPCKindQuickNode:
return DebugGetRawReceipts | EthGetBlockReceipts | EthGetTransactionReceiptBatch
case RPCKindInfura:
// Infura is big, but sadly does not support more optimized receipts fetching methods (yet?)
return EthGetTransactionReceiptBatch
case RPCKindParity:
return ParityGetBlockReceipts | EthGetTransactionReceiptBatch
case RPCKindNethermind:
return ParityGetBlockReceipts | EthGetTransactionReceiptBatch
case RPCKindDebugGeth:
return DebugGetRawReceipts | EthGetTransactionReceiptBatch
case RPCKindErigon:
return EthGetBlockReceipts | EthGetTransactionReceiptBatch
case RPCKindBasic:
return EthGetTransactionReceiptBatch
case RPCKindAny:
// if it's any kind of RPC provider, then try all methods
return AlchemyGetTransactionReceipts | EthGetBlockReceipts |
DebugGetRawReceipts | ParityGetBlockReceipts | EthGetTransactionReceiptBatch
default:
return EthGetTransactionReceiptBatch
}
}
// PickBestReceiptsFetchingMethod selects an RPC method that is still available,
// and optimal for fetching the given number of tx receipts from the specified provider kind.
func PickBestReceiptsFetchingMethod(kind RPCProviderKind, available ReceiptsFetchingMethod, txCount uint64) ReceiptsFetchingMethod {
// If we have optimized methods available, it makes sense to use them, but only if the cost is
// lower than fetching transactions one by one with the standard receipts RPC method.
if kind == RPCKindAlchemy {
if available&AlchemyGetTransactionReceipts != 0 && txCount > 250/15 {
return AlchemyGetTransactionReceipts
}
if available&EthGetBlockReceipts != 0 && txCount > 500/15 {
return EthGetBlockReceipts
}
if available&ParityGetBlockReceipts != 0 && txCount > 500/15 {
return ParityGetBlockReceipts
}
return EthGetTransactionReceiptBatch
} else if kind == RPCKindQuickNode {
if available&DebugGetRawReceipts != 0 {
return DebugGetRawReceipts
}
if available&EthGetBlockReceipts != 0 && txCount > 59/2 {
return EthGetBlockReceipts
}
if available&ParityGetBlockReceipts != 0 && txCount > 59/2 {
return ParityGetBlockReceipts
}
return EthGetTransactionReceiptBatch
}
// otherwise just find the first available method
x := ReceiptsFetchingMethod(1)
for x != 0 {
if available&x != 0 {
return x
}
x <<= 1
}
// otherwise fall back on per-tx fetching
return EthGetTransactionReceiptBatch
}
type rpcClient interface {
CallContext(ctx context.Context, result any, method string, args ...any) error
BatchCallContext(ctx context.Context, b []rpc.BatchElem) error
}
// receiptsFetchingJob runs the receipt fetching for a specific block,
// and can re-run and adapt based on the fetching method preferences and errors communicated with the requester.
type receiptsFetchingJob struct {
m sync.Mutex
requester ReceiptsRequester
client rpcClient
maxBatchSize int
block eth.BlockID
receiptHash common.Hash
txHashes []common.Hash
fetcher *IterativeBatchCall[common.Hash, *types.Receipt]
result types.Receipts
}
func NewReceiptsFetchingJob(requester ReceiptsRequester, client rpcClient, maxBatchSize int, block eth.BlockID,
receiptHash common.Hash, txHashes []common.Hash) *receiptsFetchingJob {
return &receiptsFetchingJob{
requester: requester,
client: client,
maxBatchSize: maxBatchSize,
block: block,
receiptHash: receiptHash,
txHashes: txHashes,
}
}
// ReceiptsRequester helps determine which receipts fetching method can be used,
// and is given feedback upon receipt fetching errors to adapt the choice of method.
type ReceiptsRequester interface {
PickReceiptsMethod(txCount uint64) ReceiptsFetchingMethod
OnReceiptsMethodErr(m ReceiptsFetchingMethod, err error)
}
// runFetcher retrieves the result by continuing previous batched receipt fetching work,
// and starting this work if necessary.
func (job *receiptsFetchingJob) runFetcher(ctx context.Context) error {
if job.fetcher == nil {
// start new work
job.fetcher = NewIterativeBatchCall[common.Hash, *types.Receipt](
job.txHashes,
makeReceiptRequest,
job.client.BatchCallContext,
job.maxBatchSize,
)
}
// Fetch all receipts
for {
if err := job.fetcher.Fetch(ctx); err == io.EOF {
break
} else if err != nil {
return err
}
}
result, err := job.fetcher.Result()
if err != nil { // errors if results are not available yet, should never happen.
return err
}
if err := validateReceipts(job.block, job.receiptHash, job.txHashes, result); err != nil {
job.fetcher.Reset() // if results are fetched but invalid, try restart all the fetching to try and get valid data.
return err
}
// Remember the result, and don't keep the fetcher and tx hashes around for longer than needed
job.result = result
job.fetcher = nil
job.txHashes = nil
return nil
}
// receiptsWrapper is a decoding type util. Alchemy in particular wraps the receipts array result.
type receiptsWrapper struct {
Receipts []*types.Receipt `json:"receipts"`
}
// runAltMethod retrieves the result by fetching all receipts at once,
// using the given non-standard receipt fetching method.
func (job *receiptsFetchingJob) runAltMethod(ctx context.Context, m ReceiptsFetchingMethod) error {
var result []*types.Receipt
var err error
switch m {
case AlchemyGetTransactionReceipts:
var tmp receiptsWrapper
err = job.client.CallContext(ctx, &tmp, "alchemy_getTransactionReceipts", blockHashParameter{BlockHash: job.block.Hash})
result = tmp.Receipts
case DebugGetRawReceipts:
var rawReceipts []hexutil.Bytes
err = job.client.CallContext(ctx, &rawReceipts, "debug_getRawReceipts", job.block.Hash)
if err == nil {
if len(rawReceipts) == len(job.txHashes) {
result = make([]*types.Receipt, len(rawReceipts))
totalIndex := uint(0)
for i, r := range rawReceipts {
var x types.Receipt
_ = x.UnmarshalBinary(r) // safe to ignore, we verify receipts against the receipts hash later
x.TxHash = job.txHashes[i]
x.BlockHash = job.block.Hash
x.BlockNumber = new(big.Int).SetUint64(job.block.Number)
x.TransactionIndex = uint(i)
for _, l := range x.Logs {
l.BlockNumber = job.block.Number
l.TxHash = x.TxHash
l.TxIndex = uint(i)
l.BlockHash = job.block.Hash
l.Index = totalIndex
totalIndex += 1
}
result[i] = &x
}
} else {
err = fmt.Errorf("got %d raw receipts, but expected %d", len(rawReceipts), len(job.txHashes))
}
}
case ParityGetBlockReceipts:
err = job.client.CallContext(ctx, &result, "parity_getBlockReceipts", job.block.Hash)
case EthGetBlockReceipts:
err = job.client.CallContext(ctx, &result, "eth_getBlockReceipts", job.block.Hash)
default:
err = fmt.Errorf("unknown receipt fetching method: %d", uint64(m))
}
if err != nil {
job.requester.OnReceiptsMethodErr(m, err)
return err
} else {
if err := validateReceipts(job.block, job.receiptHash, job.txHashes, result); err != nil {
return err
}
job.result = result
return nil
}
}
// Fetch makes the job fetch the receipts, and returns the results, if any.
// An error may be returned if the fetching is not successfully completed,
// and fetching may be continued/re-attempted by calling Fetch again.
// The job caches the result, so repeated Fetches add no additional cost.
// Fetch is safe to be called concurrently, and will lock to avoid duplicate work or internal inconsistency.
func (job *receiptsFetchingJob) Fetch(ctx context.Context) (types.Receipts, error) {
job.m.Lock()
defer job.m.Unlock()
if job.result != nil {
return job.result, nil
}
m := job.requester.PickReceiptsMethod(uint64(len(job.txHashes)))
if m == EthGetTransactionReceiptBatch {
if err := job.runFetcher(ctx); err != nil {
return nil, err
}
} else {
if err := job.runAltMethod(ctx, m); err != nil {
return nil, err
}
}
return job.result, nil
}
......@@ -4,19 +4,23 @@ import (
"context"
"fmt"
"math/big"
"strings"
"github.com/holiman/uint256"
"github.com/ethereum-optimism/optimism/op-node/eth"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/rpc"
"github.com/ethereum/go-ethereum/trie"
"github.com/ethereum-optimism/optimism/op-node/eth"
)
type BatchCallContextFn func(ctx context.Context, b []rpc.BatchElem) error
type CallContextFn func(ctx context.Context, result any, method string, args ...any) error
// Note: these types are used, instead of the geth types, to enable:
// - batched calls of many block requests (standard bindings do extra uncle-header fetches, cannot be batched nicely)
// - ignore uncle data (does not even exist anymore post-Merge)
......@@ -258,3 +262,27 @@ func (block *rpcBlock) ExecutionPayload(trustCache bool) (*eth.ExecutionPayload,
Transactions: opaqueTxs,
}, nil
}
// blockHashParameter is used as "block parameter":
// Some Nethermind and Alchemy RPC endpoints require an object to identify a block, instead of a string.
type blockHashParameter struct {
BlockHash common.Hash `json:"blockHash"`
}
// unusableMethod identifies if an error indicates that the RPC method cannot be used as expected:
// if it's an unknown method, or if parameters were invalid.
func unusableMethod(err error) bool {
if rpcErr, ok := err.(rpc.Error); ok {
code := rpcErr.ErrorCode()
// method not found, or invalid params
if code == -32601 || code == -32602 {
return true
}
} else {
errText := strings.ToLower(err.Error())
if strings.Contains(errText, "unknown method") || strings.Contains(errText, "invalid param") || strings.Contains(errText, "is not available") {
return true
}
}
return false
}
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment