diff --git a/README.md b/README.md index 77b2980..6cd9452 100644 --- a/README.md +++ b/README.md @@ -37,7 +37,7 @@ A distributed system for processing Ethereum execution layer data with support f ### Core Components - **Ethereum Nodes**: Configure execution node endpoints -- **Redis**: Task queue and leader election coordination +- **Redis**: Task queue and leader election coordination - **State Manager**: Track processing progress in ClickHouse - **Processors**: Configure structlog extraction settings @@ -57,6 +57,93 @@ A distributed system for processing Ethereum execution layer data with support f └─────────────────────────────────────────┘ ``` +## Embedded Mode (Library Usage) + +The execution-processor can be embedded as a library within an execution client, providing direct data access without JSON-RPC overhead. + +### Implementing DataSource + +```go +import ( + "context" + "math/big" + + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethpandaops/execution-processor/pkg/ethereum/execution" +) + +type MyDataSource struct { + client *MyExecutionClient +} + +func (ds *MyDataSource) BlockNumber(ctx context.Context) (*uint64, error) { + num := ds.client.CurrentBlock() + return &num, nil +} + +func (ds *MyDataSource) BlockByNumber(ctx context.Context, number *big.Int) (*types.Block, error) { + return ds.client.GetBlock(number), nil +} + +func (ds *MyDataSource) BlockReceipts(ctx context.Context, number *big.Int) ([]*types.Receipt, error) { + return ds.client.GetBlockReceipts(number), nil +} + +func (ds *MyDataSource) TransactionReceipt(ctx context.Context, hash string) (*types.Receipt, error) { + return ds.client.GetReceipt(hash), nil +} + +func (ds *MyDataSource) DebugTraceTransaction( + ctx context.Context, + hash string, + blockNumber *big.Int, + opts execution.TraceOptions, +) (*execution.TraceTransaction, error) { + return ds.client.TraceTransaction(hash, opts), nil +} + +func (ds *MyDataSource) ChainID() int64 { + return ds.client.ChainID() +} + +func (ds *MyDataSource) ClientType() string { + return "my-client/1.0.0" +} + +func (ds *MyDataSource) IsSynced() bool { + return ds.client.IsSynced() +} +``` + +### Creating an Embedded Pool + +```go +import ( + "github.com/ethpandaops/execution-processor/pkg/ethereum" + "github.com/ethpandaops/execution-processor/pkg/ethereum/execution" +) + +// Create embedded node with your data source +dataSource := &MyDataSource{client: myClient} +node := execution.NewEmbeddedNode(log, "embedded", dataSource) + +// Create pool with the embedded node +pool := ethereum.NewPoolWithNodes(log, "processor", []execution.Node{node}, nil) +pool.Start(ctx) + +// Mark ready when your client is synced and ready to serve data +node.MarkReady(ctx) +``` + +### Embedded vs RPC Mode + +| Aspect | RPC Mode | Embedded Mode | +|--------|----------|---------------| +| Data Access | JSON-RPC over HTTP | Direct function calls | +| Readiness | Auto-detected via RPC health checks | Host calls MarkReady() | +| Performance | Network + serialization overhead | Zero serialization overhead | +| Use Case | External execution clients | Library integration | + ## Manual Block Queue API The execution processor provides an HTTP API for manually queuing blocks for reprocessing. This is useful for fixing data issues or reprocessing specific blocks. @@ -80,7 +167,7 @@ curl -X POST http://localhost:8080/api/v1/queue/block/transaction_structlog/1234 "status": "queued", "block_number": 12345, "processor": "transaction_structlog", - "queue": "process:forwards", + "queue": "process:forwards", "transaction_count": 150, "tasks_created": 150 } @@ -158,7 +245,7 @@ curl -X POST http://localhost:8080/api/v1/queue/blocks/transaction_structlog \ # Run tests go test ./... -# Run with race detector +# Run with race detector go test ./... --race # Build @@ -167,4 +254,4 @@ go build . ## License -See LICENSE file. \ No newline at end of file +See LICENSE file. diff --git a/pkg/config/config.go b/pkg/config/config.go new file mode 100644 index 0000000..f20d0a7 --- /dev/null +++ b/pkg/config/config.go @@ -0,0 +1,84 @@ +// Package config provides configuration types for execution-processor. +// This package is designed to be imported without pulling in go-ethereum dependencies, +// making it suitable for embedded mode integrations. +package config + +import ( + "fmt" + "time" + + "github.com/ethpandaops/execution-processor/pkg/ethereum/execution" + "github.com/ethpandaops/execution-processor/pkg/processor" + "github.com/ethpandaops/execution-processor/pkg/redis" + "github.com/ethpandaops/execution-processor/pkg/state" +) + +// EthereumConfig is the ethereum network configuration. +// This is a copy of ethereum.Config to avoid importing pkg/ethereum +// which would pull in go-ethereum dependencies. +type EthereumConfig struct { + // Execution configuration + Execution []*execution.Config `yaml:"execution"` + // Override network name for custom networks (bypasses networkMap) + OverrideNetworkName *string `yaml:"overrideNetworkName"` +} + +// Validate validates the ethereum configuration. +func (c *EthereumConfig) Validate() error { + for i, exec := range c.Execution { + if err := exec.Validate(); err != nil { + return fmt.Errorf("invalid execution configuration at index %d: %w", i, err) + } + } + + return nil +} + +// Config is the main configuration for execution-processor. +type Config struct { + // MetricsAddr is the address to listen on for metrics. + MetricsAddr string `yaml:"metricsAddr" default:":9090"` + // HealthCheckAddr is the address to listen on for healthcheck. + HealthCheckAddr *string `yaml:"healthCheckAddr"` + // PProfAddr is the address to listen on for pprof. + PProfAddr *string `yaml:"pprofAddr"` + // APIAddr is the address to listen on for the API server. + APIAddr *string `yaml:"apiAddr"` + // LoggingLevel is the logging level to use. + LoggingLevel string `yaml:"logging" default:"info"` + // Ethereum is the ethereum network configuration. + Ethereum EthereumConfig `yaml:"ethereum"` + // Redis is the redis configuration. + Redis *redis.Config `yaml:"redis"` + // StateManager is the state manager configuration. + StateManager state.Config `yaml:"stateManager"` + // Processors is the processor configuration. + Processors processor.Config `yaml:"processors"` + // ShutdownTimeout is the timeout for shutting down the server. + ShutdownTimeout time.Duration `yaml:"shutdownTimeout" default:"10s"` +} + +// Validate validates the configuration. +func (c *Config) Validate() error { + if c.Redis == nil { + return fmt.Errorf("redis configuration is required") + } + + if err := c.Redis.Validate(); err != nil { + return fmt.Errorf("invalid redis configuration: %w", err) + } + + if err := c.Ethereum.Validate(); err != nil { + return fmt.Errorf("invalid ethereum configuration: %w", err) + } + + if err := c.StateManager.Validate(); err != nil { + return fmt.Errorf("invalid state manager configuration: %w", err) + } + + if err := c.Processors.Validate(); err != nil { + return fmt.Errorf("invalid processor configuration: %w", err) + } + + return nil +} diff --git a/pkg/ethereum/execution/block.go b/pkg/ethereum/execution/block.go new file mode 100644 index 0000000..e4e839d --- /dev/null +++ b/pkg/ethereum/execution/block.go @@ -0,0 +1,136 @@ +package execution + +import "math/big" + +// Hash represents a 32-byte hash. +type Hash [32]byte + +// Hex returns the hex string representation of the hash. +func (h Hash) Hex() string { + return "0x" + encodeHex(h[:]) +} + +// String returns the hex string representation of the hash. +func (h Hash) String() string { + return h.Hex() +} + +// Address represents a 20-byte Ethereum address. +type Address [20]byte + +// Hex returns the hex string representation of the address with checksum. +func (a Address) Hex() string { + return "0x" + encodeHex(a[:]) +} + +// String returns the hex string representation of the address. +func (a Address) String() string { + return a.Hex() +} + +// encodeHex encodes bytes as hex string without 0x prefix. +func encodeHex(b []byte) string { + const hexChars = "0123456789abcdef" + + result := make([]byte, len(b)*2) + + for i, v := range b { + result[i*2] = hexChars[v>>4] + result[i*2+1] = hexChars[v&0x0f] + } + + return string(result) +} + +// Transaction type constants matching go-ethereum values. +const ( + LegacyTxType = 0 + AccessListTxType = 1 + DynamicFeeTxType = 2 + BlobTxType = 3 +) + +// Block interface defines methods for accessing block data. +// Implementations are provided by data sources (RPC, embedded clients). +type Block interface { + // Number returns the block number. + Number() *big.Int + + // Hash returns the block hash. + Hash() Hash + + // ParentHash returns the parent block hash. + ParentHash() Hash + + // BaseFee returns the base fee per gas (EIP-1559), or nil for pre-London blocks. + BaseFee() *big.Int + + // Transactions returns all transactions in the block. + Transactions() []Transaction +} + +// Transaction interface defines methods for accessing transaction data. +// The From() method returns the sender address, computed by the data source +// using its own crypto implementation (avoiding go-ethereum crypto imports). +type Transaction interface { + // Hash returns the transaction hash. + Hash() Hash + + // Type returns the transaction type (0=legacy, 1=access list, 2=dynamic fee, 3=blob). + Type() uint8 + + // To returns the recipient address, or nil for contract creation. + To() *Address + + // From returns the sender address. + // This is computed by the data source using types.Sender() or equivalent. + From() Address + + // Nonce returns the sender account nonce. + Nonce() uint64 + + // Gas returns the gas limit. + Gas() uint64 + + // GasPrice returns the gas price (for legacy transactions). + GasPrice() *big.Int + + // GasTipCap returns the max priority fee per gas (EIP-1559). + GasTipCap() *big.Int + + // GasFeeCap returns the max fee per gas (EIP-1559). + GasFeeCap() *big.Int + + // Value returns the value transferred in wei. + Value() *big.Int + + // Data returns the input data (calldata). + Data() []byte + + // Size returns the encoded transaction size in bytes. + Size() uint64 + + // ChainId returns the chain ID, or nil for legacy transactions. + ChainId() *big.Int + + // BlobGas returns the blob gas used (for blob transactions). + BlobGas() uint64 + + // BlobGasFeeCap returns the max blob fee per gas (for blob transactions). + BlobGasFeeCap() *big.Int + + // BlobHashes returns the versioned hashes (for blob transactions). + BlobHashes() []Hash +} + +// Receipt interface defines methods for accessing transaction receipt data. +type Receipt interface { + // Status returns the transaction status (1=success, 0=failure). + Status() uint64 + + // TxHash returns the transaction hash. + TxHash() Hash + + // GasUsed returns the gas used by the transaction. + GasUsed() uint64 +} diff --git a/pkg/ethereum/execution/embedded_node.go b/pkg/ethereum/execution/embedded_node.go new file mode 100644 index 0000000..ff0c3ff --- /dev/null +++ b/pkg/ethereum/execution/embedded_node.go @@ -0,0 +1,211 @@ +package execution + +import ( + "context" + "math/big" + "sync" + + "github.com/sirupsen/logrus" +) + +// DataSource is the interface host applications implement to provide +// execution data directly without JSON-RPC. This enables embedding +// execution-processor as a library within an execution client. +// +// All methods must be safe for concurrent calls from multiple goroutines. +// Context cancellation should be respected for all I/O operations. +// +// The interface uses abstract types (Block, Transaction, Receipt) instead of +// go-ethereum types to avoid CGO dependencies. Host applications should +// implement these interfaces with their own types. +// +// Example implementation: +// +// type MyDataSource struct { +// client *MyExecutionClient +// } +// +// func (ds *MyDataSource) BlockNumber(ctx context.Context) (*uint64, error) { +// num := ds.client.CurrentBlock() +// return &num, nil +// } +type DataSource interface { + // BlockNumber returns the current block number. + BlockNumber(ctx context.Context) (*uint64, error) + + // BlockByNumber returns the block at the given number. + BlockByNumber(ctx context.Context, number *big.Int) (Block, error) + + // BlockReceipts returns all receipts for the block at the given number. + BlockReceipts(ctx context.Context, number *big.Int) ([]Receipt, error) + + // TransactionReceipt returns the receipt for the transaction with the given hash. + TransactionReceipt(ctx context.Context, hash string) (Receipt, error) + + // DebugTraceTransaction returns the execution trace for the transaction. + DebugTraceTransaction(ctx context.Context, hash string, blockNumber *big.Int, opts TraceOptions) (*TraceTransaction, error) + + // ChainID returns the chain ID. + ChainID() int64 + + // ClientType returns the client type/version string. + ClientType() string + + // IsSynced returns true if the data source is fully synced. + IsSynced() bool +} + +// Compile-time check that EmbeddedNode implements Node interface. +var _ Node = (*EmbeddedNode)(nil) + +// EmbeddedNode implements Node by delegating to a DataSource. +// This allows host applications to provide execution data directly +// without going through JSON-RPC, eliminating serialization overhead. +// +// Lifecycle: +// 1. Create with NewEmbeddedNode(log, name, dataSource) +// 2. Register OnReady callbacks (optional) +// 3. Pool calls Start() (no-op for embedded) +// 4. Host calls MarkReady() when DataSource is ready to serve data +// 5. Callbacks execute in registration order, node becomes healthy in pool +// 6. Pool calls Stop() on shutdown (no-op for embedded) +// +// Thread-safety: All methods are safe for concurrent use. +type EmbeddedNode struct { + log logrus.FieldLogger + name string + source DataSource + ready bool + onReadyCallbacks []func(ctx context.Context) error + mu sync.RWMutex +} + +// NewEmbeddedNode creates a new EmbeddedNode with the given DataSource. +// +// Parameters: +// - log: Logger for node operations +// - name: Human-readable name for this node (used in logs and metrics) +// - source: DataSource implementation providing execution data +// +// The returned node is not yet ready. Call MarkReady() when the DataSource +// is ready to serve data. +func NewEmbeddedNode(log logrus.FieldLogger, name string, source DataSource) *EmbeddedNode { + return &EmbeddedNode{ + log: log.WithFields(logrus.Fields{"type": "execution", "source": name, "mode": "embedded"}), + name: name, + source: source, + onReadyCallbacks: make([]func(ctx context.Context) error, 0), + } +} + +// Start is a no-op for EmbeddedNode. The host controls readiness via MarkReady(). +func (n *EmbeddedNode) Start(_ context.Context) error { + n.log.Info("EmbeddedNode started - waiting for host to call MarkReady()") + + return nil +} + +// Stop is a no-op for EmbeddedNode. The host manages the DataSource lifecycle. +func (n *EmbeddedNode) Stop(_ context.Context) error { + n.log.Info("EmbeddedNode stopped") + + return nil +} + +// MarkReady is called by the host application when the DataSource is ready. +// This triggers all registered OnReady callbacks. +func (n *EmbeddedNode) MarkReady(ctx context.Context) error { + n.mu.Lock() + n.ready = true + callbacks := n.onReadyCallbacks + n.mu.Unlock() + + n.log.WithField("callback_count", len(callbacks)).Info("EmbeddedNode marked as ready, executing callbacks") + + for i, cb := range callbacks { + n.log.WithField("callback_index", i).Info("Executing OnReady callback") + + if err := cb(ctx); err != nil { + n.log.WithError(err).Error("Failed to execute OnReady callback") + + return err + } + } + + return nil +} + +// OnReady registers a callback to be called when the node becomes ready. +func (n *EmbeddedNode) OnReady(_ context.Context, callback func(ctx context.Context) error) { + n.mu.Lock() + defer n.mu.Unlock() + + n.onReadyCallbacks = append(n.onReadyCallbacks, callback) +} + +// IsReady returns true if the node has been marked as ready. +func (n *EmbeddedNode) IsReady() bool { + n.mu.RLock() + defer n.mu.RUnlock() + + return n.ready +} + +// BlockNumber delegates to the DataSource. +func (n *EmbeddedNode) BlockNumber(ctx context.Context) (*uint64, error) { + return n.source.BlockNumber(ctx) +} + +// BlockByNumber delegates to the DataSource. +func (n *EmbeddedNode) BlockByNumber(ctx context.Context, number *big.Int) (Block, error) { + return n.source.BlockByNumber(ctx, number) +} + +// BlockReceipts delegates to the DataSource. +func (n *EmbeddedNode) BlockReceipts(ctx context.Context, number *big.Int) ([]Receipt, error) { + return n.source.BlockReceipts(ctx, number) +} + +// TransactionReceipt delegates to the DataSource. +func (n *EmbeddedNode) TransactionReceipt(ctx context.Context, hash string) (Receipt, error) { + return n.source.TransactionReceipt(ctx, hash) +} + +// DebugTraceTransaction delegates to the DataSource. +// +// OPTIMIZATION: In embedded mode, the tracer extracts CallToAddress directly +// for CALL-family opcodes instead of capturing the full stack. We explicitly +// set DisableStack: true to signal this intent, even though the tracer ignores +// this setting (it always uses the optimized path). +func (n *EmbeddedNode) DebugTraceTransaction( + ctx context.Context, + hash string, + blockNumber *big.Int, + opts TraceOptions, +) (*TraceTransaction, error) { + // Override DisableStack for embedded mode optimization. + // The tracer extracts CallToAddress directly, so full stack capture is unnecessary. + opts.DisableStack = true + + return n.source.DebugTraceTransaction(ctx, hash, blockNumber, opts) +} + +// ChainID delegates to the DataSource. +func (n *EmbeddedNode) ChainID() int64 { + return n.source.ChainID() +} + +// ClientType delegates to the DataSource. +func (n *EmbeddedNode) ClientType() string { + return n.source.ClientType() +} + +// IsSynced delegates to the DataSource. +func (n *EmbeddedNode) IsSynced() bool { + return n.source.IsSynced() +} + +// Name returns the configured name for this node. +func (n *EmbeddedNode) Name() string { + return n.name +} diff --git a/pkg/ethereum/execution/embedded_node_test.go b/pkg/ethereum/execution/embedded_node_test.go new file mode 100644 index 0000000..ea3ea7b --- /dev/null +++ b/pkg/ethereum/execution/embedded_node_test.go @@ -0,0 +1,713 @@ +package execution_test + +import ( + "context" + "errors" + "math/big" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/ethpandaops/execution-processor/pkg/ethereum/execution" + "github.com/sirupsen/logrus" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" +) + +// MockBlock implements execution.Block for testing. +type MockBlock struct { + number *big.Int + hash execution.Hash + parentHash execution.Hash + baseFee *big.Int + txs []execution.Transaction +} + +func (b *MockBlock) Number() *big.Int { return b.number } +func (b *MockBlock) Hash() execution.Hash { return b.hash } +func (b *MockBlock) ParentHash() execution.Hash { return b.parentHash } +func (b *MockBlock) BaseFee() *big.Int { return b.baseFee } +func (b *MockBlock) Transactions() []execution.Transaction { return b.txs } + +// NewMockBlock creates a mock block with the given number. +func NewMockBlock(number *big.Int) *MockBlock { + return &MockBlock{ + number: number, + hash: execution.Hash{}, + parentHash: execution.Hash{}, + baseFee: big.NewInt(1000000000), + txs: []execution.Transaction{}, + } +} + +// MockReceipt implements execution.Receipt for testing. +type MockReceipt struct { + status uint64 + txHash execution.Hash + gasUsed uint64 +} + +func (r *MockReceipt) Status() uint64 { return r.status } +func (r *MockReceipt) TxHash() execution.Hash { return r.txHash } +func (r *MockReceipt) GasUsed() uint64 { return r.gasUsed } + +// NewMockReceipt creates a mock receipt with the given status. +func NewMockReceipt(status uint64, gasUsed uint64) *MockReceipt { + return &MockReceipt{ + status: status, + txHash: execution.Hash{}, + gasUsed: gasUsed, + } +} + +// MockDataSource implements execution.DataSource for testing. +type MockDataSource struct { + mock.Mock +} + +func (m *MockDataSource) BlockNumber(ctx context.Context) (*uint64, error) { + args := m.Called(ctx) + + if args.Get(0) == nil { + return nil, args.Error(1) + } + + val, ok := args.Get(0).(*uint64) + if !ok { + return nil, args.Error(1) + } + + return val, args.Error(1) +} + +func (m *MockDataSource) BlockByNumber(ctx context.Context, number *big.Int) (execution.Block, error) { + args := m.Called(ctx, number) + + if args.Get(0) == nil { + return nil, args.Error(1) + } + + val, ok := args.Get(0).(execution.Block) + if !ok { + return nil, args.Error(1) + } + + return val, args.Error(1) +} + +func (m *MockDataSource) BlockReceipts(ctx context.Context, number *big.Int) ([]execution.Receipt, error) { + args := m.Called(ctx, number) + + if args.Get(0) == nil { + return nil, args.Error(1) + } + + val, ok := args.Get(0).([]execution.Receipt) + if !ok { + return nil, args.Error(1) + } + + return val, args.Error(1) +} + +func (m *MockDataSource) TransactionReceipt(ctx context.Context, hash string) (execution.Receipt, error) { + args := m.Called(ctx, hash) + + if args.Get(0) == nil { + return nil, args.Error(1) + } + + val, ok := args.Get(0).(execution.Receipt) + if !ok { + return nil, args.Error(1) + } + + return val, args.Error(1) +} + +func (m *MockDataSource) DebugTraceTransaction( + ctx context.Context, + hash string, + blockNumber *big.Int, + opts execution.TraceOptions, +) (*execution.TraceTransaction, error) { + args := m.Called(ctx, hash, blockNumber, opts) + + if args.Get(0) == nil { + return nil, args.Error(1) + } + + val, ok := args.Get(0).(*execution.TraceTransaction) + if !ok { + return nil, args.Error(1) + } + + return val, args.Error(1) +} + +func (m *MockDataSource) ChainID() int64 { + args := m.Called() + + val, ok := args.Get(0).(int64) + if !ok { + return 0 + } + + return val +} + +func (m *MockDataSource) ClientType() string { + args := m.Called() + + return args.String(0) +} + +func (m *MockDataSource) IsSynced() bool { + args := m.Called() + + return args.Bool(0) +} + +func TestEmbeddedNode_Creation(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + require.NotNil(t, node) + assert.Equal(t, "test-node", node.Name()) + assert.False(t, node.IsReady()) +} + +func TestEmbeddedNode_Start_NoOp(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx := context.Background() + err := node.Start(ctx) + + assert.NoError(t, err) + // Start should not mark the node as ready + assert.False(t, node.IsReady()) +} + +func TestEmbeddedNode_Stop_NoOp(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx := context.Background() + + // Start and mark ready first + err := node.Start(ctx) + require.NoError(t, err) + + err = node.MarkReady(ctx) + require.NoError(t, err) + + // Stop should complete without error + err = node.Stop(ctx) + assert.NoError(t, err) +} + +func TestEmbeddedNode_MarkReady_ExecutesCallbacks(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx := context.Background() + + // Track callback execution order + var order []int + + var mu sync.Mutex + + node.OnReady(ctx, func(_ context.Context) error { + mu.Lock() + defer mu.Unlock() + + order = append(order, 1) + + return nil + }) + + node.OnReady(ctx, func(_ context.Context) error { + mu.Lock() + defer mu.Unlock() + + order = append(order, 2) + + return nil + }) + + node.OnReady(ctx, func(_ context.Context) error { + mu.Lock() + defer mu.Unlock() + + order = append(order, 3) + + return nil + }) + + assert.False(t, node.IsReady()) + + err := node.MarkReady(ctx) + require.NoError(t, err) + + assert.True(t, node.IsReady()) + assert.Equal(t, []int{1, 2, 3}, order) +} + +func TestEmbeddedNode_MarkReady_CallbackError(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx := context.Background() + + expectedErr := errors.New("callback failed") + + var callbacksCalled int + + node.OnReady(ctx, func(_ context.Context) error { + callbacksCalled++ + + return nil + }) + + node.OnReady(ctx, func(_ context.Context) error { + callbacksCalled++ + + return expectedErr + }) + + node.OnReady(ctx, func(_ context.Context) error { + callbacksCalled++ + + return nil + }) + + err := node.MarkReady(ctx) + assert.ErrorIs(t, err, expectedErr) + // Only first two callbacks should have been called (second one failed) + assert.Equal(t, 2, callbacksCalled) +} + +func TestEmbeddedNode_OnReady_MultipleCallbacks(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx := context.Background() + + var count atomic.Int32 + + const numCallbacks = 10 + for i := 0; i < numCallbacks; i++ { + node.OnReady(ctx, func(_ context.Context) error { + count.Add(1) + + return nil + }) + } + + err := node.MarkReady(ctx) + require.NoError(t, err) + + assert.Equal(t, int32(numCallbacks), count.Load()) +} + +func TestEmbeddedNode_IsReady_States(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx := context.Background() + + // Initially not ready + assert.False(t, node.IsReady()) + + // Still not ready after Start + err := node.Start(ctx) + require.NoError(t, err) + + assert.False(t, node.IsReady()) + + // Ready after MarkReady + err = node.MarkReady(ctx) + require.NoError(t, err) + + assert.True(t, node.IsReady()) + + // Still ready after Stop + err = node.Stop(ctx) + require.NoError(t, err) + + assert.True(t, node.IsReady()) +} + +func TestEmbeddedNode_DelegatesToDataSource_BlockNumber(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx := context.Background() + expectedBlock := uint64(12345) + + ds.On("BlockNumber", ctx).Return(&expectedBlock, nil) + + result, err := node.BlockNumber(ctx) + require.NoError(t, err) + require.NotNil(t, result) + assert.Equal(t, expectedBlock, *result) + + ds.AssertExpectations(t) +} + +func TestEmbeddedNode_DelegatesToDataSource_BlockByNumber(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx := context.Background() + blockNum := big.NewInt(12345) + expectedBlock := NewMockBlock(blockNum) + + ds.On("BlockByNumber", ctx, blockNum).Return(expectedBlock, nil) + + result, err := node.BlockByNumber(ctx, blockNum) + require.NoError(t, err) + assert.Equal(t, expectedBlock.Number(), result.Number()) + + ds.AssertExpectations(t) +} + +func TestEmbeddedNode_DelegatesToDataSource_BlockReceipts(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx := context.Background() + blockNum := big.NewInt(12345) + expectedReceipts := []execution.Receipt{ + NewMockReceipt(1, 21000), + NewMockReceipt(0, 50000), + } + + ds.On("BlockReceipts", ctx, blockNum).Return(expectedReceipts, nil) + + result, err := node.BlockReceipts(ctx, blockNum) + require.NoError(t, err) + assert.Len(t, result, 2) + + ds.AssertExpectations(t) +} + +func TestEmbeddedNode_DelegatesToDataSource_TransactionReceipt(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx := context.Background() + txHash := "0x1234567890abcdef" + expectedReceipt := NewMockReceipt(1, 21000) + + ds.On("TransactionReceipt", ctx, txHash).Return(expectedReceipt, nil) + + result, err := node.TransactionReceipt(ctx, txHash) + require.NoError(t, err) + assert.Equal(t, uint64(1), result.Status()) + assert.Equal(t, uint64(21000), result.GasUsed()) + + ds.AssertExpectations(t) +} + +func TestEmbeddedNode_DelegatesToDataSource_DebugTraceTransaction(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx := context.Background() + txHash := "0x1234567890abcdef" + blockNum := big.NewInt(12345) + opts := execution.DefaultTraceOptions() + expectedTrace := &execution.TraceTransaction{ + Gas: 21000, + Failed: false, + } + + ds.On("DebugTraceTransaction", ctx, txHash, blockNum, opts).Return(expectedTrace, nil) + + result, err := node.DebugTraceTransaction(ctx, txHash, blockNum, opts) + require.NoError(t, err) + assert.Equal(t, expectedTrace, result) + + ds.AssertExpectations(t) +} + +func TestEmbeddedNode_DelegatesToDataSource_ClientType(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ds.On("ClientType").Return("geth/1.10.0") + + result := node.ClientType() + assert.Equal(t, "geth/1.10.0", result) + + ds.AssertExpectations(t) +} + +func TestEmbeddedNode_DelegatesToDataSource_IsSynced(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ds.On("IsSynced").Return(true) + + result := node.IsSynced() + assert.True(t, result) + + ds.AssertExpectations(t) +} + +func TestEmbeddedNode_Name(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + testCases := []struct { + name string + expectedName string + }{ + {name: "simple-name", expectedName: "simple-name"}, + {name: "with-numbers-123", expectedName: "with-numbers-123"}, + {name: "embedded-erigon", expectedName: "embedded-erigon"}, + {name: "", expectedName: ""}, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, tc.name, ds) + + assert.Equal(t, tc.expectedName, node.Name()) + }) + } +} + +func TestEmbeddedNode_ConcurrentMarkReady(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx := context.Background() + + var callbackCount atomic.Int32 + + node.OnReady(ctx, func(_ context.Context) error { + callbackCount.Add(1) + + return nil + }) + + // Start multiple goroutines calling MarkReady concurrently + const numGoroutines = 10 + + var wg sync.WaitGroup + + for i := 0; i < numGoroutines; i++ { + wg.Add(1) + + go func() { + defer wg.Done() + // Ignore errors - only the first MarkReady should execute callbacks + _ = node.MarkReady(ctx) + }() + } + + wg.Wait() + + // Node should be ready + assert.True(t, node.IsReady()) + + // Callback should have been called at least once + // (implementation may allow multiple calls, but at least one should succeed) + assert.GreaterOrEqual(t, callbackCount.Load(), int32(1)) +} + +func TestEmbeddedNode_ConcurrentOnReady(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx := context.Background() + + var callbackCount atomic.Int32 + + // Register callbacks concurrently + const numGoroutines = 10 + + var wg sync.WaitGroup + + for i := 0; i < numGoroutines; i++ { + wg.Add(1) + + go func() { + defer wg.Done() + + node.OnReady(ctx, func(_ context.Context) error { + callbackCount.Add(1) + + return nil + }) + }() + } + + wg.Wait() + + // Now mark ready and verify all callbacks execute + err := node.MarkReady(ctx) + require.NoError(t, err) + + assert.Equal(t, int32(numGoroutines), callbackCount.Load()) +} + +func TestEmbeddedNode_InterfaceCompliance(t *testing.T) { + // Compile-time check that EmbeddedNode implements Node interface + var _ execution.Node = (*execution.EmbeddedNode)(nil) + + // Create an actual instance and verify it can be used as Node + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + + var node execution.Node = execution.NewEmbeddedNode(log, "test-node", ds) + + require.NotNil(t, node) + assert.Equal(t, "test-node", node.Name()) +} + +func TestEmbeddedNode_DataSourceError(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx := context.Background() + expectedErr := errors.New("data source error") + + ds.On("BlockNumber", ctx).Return(nil, expectedErr) + + result, err := node.BlockNumber(ctx) + assert.ErrorIs(t, err, expectedErr) + assert.Nil(t, result) + + ds.AssertExpectations(t) +} + +func TestEmbeddedNode_ContextCancellation(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx, cancel := context.WithCancel(context.Background()) + cancel() // Cancel immediately + + // DataSource should receive cancelled context + ds.On("BlockNumber", ctx).Return(nil, ctx.Err()) + + result, err := node.BlockNumber(ctx) + assert.Error(t, err) + assert.Nil(t, result) +} + +func TestEmbeddedNode_CallbackWithContext(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx := context.Background() + + var receivedCtx context.Context + + node.OnReady(ctx, func(cbCtx context.Context) error { + receivedCtx = cbCtx + + return nil + }) + + err := node.MarkReady(ctx) + require.NoError(t, err) + + // The callback should receive the context passed to MarkReady + assert.Equal(t, ctx, receivedCtx) +} + +func TestEmbeddedNode_CallbackWithTimeout(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + ds := new(MockDataSource) + node := execution.NewEmbeddedNode(log, "test-node", ds) + + ctx, cancel := context.WithTimeout(context.Background(), 100*time.Millisecond) + defer cancel() + + callbackExecuted := make(chan struct{}) + + node.OnReady(ctx, func(_ context.Context) error { + close(callbackExecuted) + + return nil + }) + + err := node.MarkReady(ctx) + require.NoError(t, err) + + select { + case <-callbackExecuted: + // Success + case <-time.After(1 * time.Second): + t.Fatal("callback did not execute") + } +} diff --git a/pkg/ethereum/execution/geth/adapter.go b/pkg/ethereum/execution/geth/adapter.go new file mode 100644 index 0000000..447ad76 --- /dev/null +++ b/pkg/ethereum/execution/geth/adapter.go @@ -0,0 +1,227 @@ +//go:build !embedded + +// Package geth provides go-ethereum adapters for the execution interfaces. +// This package contains all go-ethereum dependencies, allowing the core +// execution package to remain free of CGO-dependent imports. +package geth + +import ( + "math/big" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + + "github.com/ethpandaops/execution-processor/pkg/ethereum/execution" +) + +// Compile-time interface checks. +var ( + _ execution.Block = (*BlockAdapter)(nil) + _ execution.Transaction = (*TransactionAdapter)(nil) + _ execution.Receipt = (*ReceiptAdapter)(nil) +) + +// BlockAdapter wraps a go-ethereum Block to implement execution.Block. +type BlockAdapter struct { + block *types.Block + txs []execution.Transaction +} + +// NewBlockAdapter creates a new BlockAdapter from a go-ethereum Block. +// It extracts sender addresses for all transactions using the appropriate signer. +func NewBlockAdapter(block *types.Block) *BlockAdapter { + gethTxs := block.Transactions() + txs := make([]execution.Transaction, len(gethTxs)) + + for i, tx := range gethTxs { + txs[i] = NewTransactionAdapter(tx) + } + + return &BlockAdapter{ + block: block, + txs: txs, + } +} + +// Number returns the block number. +func (b *BlockAdapter) Number() *big.Int { + return b.block.Number() +} + +// Hash returns the block hash. +func (b *BlockAdapter) Hash() execution.Hash { + return execution.Hash(b.block.Hash()) +} + +// ParentHash returns the parent block hash. +func (b *BlockAdapter) ParentHash() execution.Hash { + return execution.Hash(b.block.ParentHash()) +} + +// BaseFee returns the base fee per gas (EIP-1559), or nil for pre-London blocks. +func (b *BlockAdapter) BaseFee() *big.Int { + return b.block.BaseFee() +} + +// Transactions returns all transactions in the block. +func (b *BlockAdapter) Transactions() []execution.Transaction { + return b.txs +} + +// TransactionAdapter wraps a go-ethereum Transaction to implement execution.Transaction. +type TransactionAdapter struct { + tx *types.Transaction + from common.Address +} + +// NewTransactionAdapter creates a new TransactionAdapter from a go-ethereum Transaction. +// It computes the sender address using the appropriate signer. +func NewTransactionAdapter(tx *types.Transaction) *TransactionAdapter { + // Determine the appropriate signer for extracting the sender + var signer types.Signer + + chainID := tx.ChainId() + if chainID == nil || chainID.Sign() == 0 { + // Legacy transaction without EIP-155 replay protection + signer = types.HomesteadSigner{} + } else { + signer = types.LatestSignerForChainID(chainID) + } + + // Extract sender - this uses go-ethereum's crypto package internally + from, _ := types.Sender(signer, tx) + + return &TransactionAdapter{ + tx: tx, + from: from, + } +} + +// Hash returns the transaction hash. +func (t *TransactionAdapter) Hash() execution.Hash { + return execution.Hash(t.tx.Hash()) +} + +// Type returns the transaction type. +func (t *TransactionAdapter) Type() uint8 { + return t.tx.Type() +} + +// To returns the recipient address, or nil for contract creation. +func (t *TransactionAdapter) To() *execution.Address { + if t.tx.To() == nil { + return nil + } + + addr := execution.Address(*t.tx.To()) + + return &addr +} + +// From returns the sender address. +func (t *TransactionAdapter) From() execution.Address { + return execution.Address(t.from) +} + +// Nonce returns the sender account nonce. +func (t *TransactionAdapter) Nonce() uint64 { + return t.tx.Nonce() +} + +// Gas returns the gas limit. +func (t *TransactionAdapter) Gas() uint64 { + return t.tx.Gas() +} + +// GasPrice returns the gas price (for legacy transactions). +func (t *TransactionAdapter) GasPrice() *big.Int { + return t.tx.GasPrice() +} + +// GasTipCap returns the max priority fee per gas (EIP-1559). +func (t *TransactionAdapter) GasTipCap() *big.Int { + return t.tx.GasTipCap() +} + +// GasFeeCap returns the max fee per gas (EIP-1559). +func (t *TransactionAdapter) GasFeeCap() *big.Int { + return t.tx.GasFeeCap() +} + +// Value returns the value transferred in wei. +func (t *TransactionAdapter) Value() *big.Int { + return t.tx.Value() +} + +// Data returns the input data (calldata). +func (t *TransactionAdapter) Data() []byte { + return t.tx.Data() +} + +// Size returns the encoded transaction size in bytes. +func (t *TransactionAdapter) Size() uint64 { + return t.tx.Size() +} + +// ChainId returns the chain ID, or nil for legacy transactions. +func (t *TransactionAdapter) ChainId() *big.Int { + return t.tx.ChainId() +} + +// BlobGas returns the blob gas used (for blob transactions). +func (t *TransactionAdapter) BlobGas() uint64 { + return t.tx.BlobGas() +} + +// BlobGasFeeCap returns the max blob fee per gas (for blob transactions). +func (t *TransactionAdapter) BlobGasFeeCap() *big.Int { + return t.tx.BlobGasFeeCap() +} + +// BlobHashes returns the versioned hashes (for blob transactions). +func (t *TransactionAdapter) BlobHashes() []execution.Hash { + gethHashes := t.tx.BlobHashes() + hashes := make([]execution.Hash, len(gethHashes)) + + for i, h := range gethHashes { + hashes[i] = execution.Hash(h) + } + + return hashes +} + +// ReceiptAdapter wraps a go-ethereum Receipt to implement execution.Receipt. +type ReceiptAdapter struct { + receipt *types.Receipt +} + +// NewReceiptAdapter creates a new ReceiptAdapter from a go-ethereum Receipt. +func NewReceiptAdapter(receipt *types.Receipt) *ReceiptAdapter { + return &ReceiptAdapter{receipt: receipt} +} + +// Status returns the transaction status (1=success, 0=failure). +func (r *ReceiptAdapter) Status() uint64 { + return r.receipt.Status +} + +// TxHash returns the transaction hash. +func (r *ReceiptAdapter) TxHash() execution.Hash { + return execution.Hash(r.receipt.TxHash) +} + +// GasUsed returns the gas used by the transaction. +func (r *ReceiptAdapter) GasUsed() uint64 { + return r.receipt.GasUsed +} + +// AdaptReceipts converts a slice of go-ethereum receipts to execution.Receipt interfaces. +func AdaptReceipts(receipts []*types.Receipt) []execution.Receipt { + result := make([]execution.Receipt, len(receipts)) + + for i, r := range receipts { + result[i] = NewReceiptAdapter(r) + } + + return result +} diff --git a/pkg/ethereum/execution/rpc.go b/pkg/ethereum/execution/geth/rpc.go similarity index 66% rename from pkg/ethereum/execution/rpc.go rename to pkg/ethereum/execution/geth/rpc.go index 638cb84..c34da64 100644 --- a/pkg/ethereum/execution/rpc.go +++ b/pkg/ethereum/execution/geth/rpc.go @@ -1,4 +1,6 @@ -package execution +//go:build !embedded + +package geth import ( "context" @@ -8,18 +10,18 @@ import ( "time" "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/rpc" pcommon "github.com/ethpandaops/execution-processor/pkg/common" + "github.com/ethpandaops/execution-processor/pkg/ethereum/execution" ) const ( - STATUS_ERROR = "error" - STATUS_SUCCESS = "success" + statusError = "error" + statusSuccess = "success" ) -func (n *Node) BlockNumber(ctx context.Context) (*uint64, error) { +func (n *RPCNode) blockNumber(ctx context.Context) (*uint64, error) { start := time.Now() blockNumber, err := n.client.BlockNumber(ctx) @@ -27,9 +29,9 @@ func (n *Node) BlockNumber(ctx context.Context) (*uint64, error) { duration := time.Since(start) // Record RPC metrics - status := STATUS_SUCCESS + status := statusSuccess if err != nil { - status = STATUS_ERROR + status = statusError } network := n.Metadata().ChainID() @@ -44,7 +46,7 @@ func (n *Node) BlockNumber(ctx context.Context) (*uint64, error) { return &blockNumber, nil } -func (n *Node) BlockByNumber(ctx context.Context, blockNumber *big.Int) (*types.Block, error) { +func (n *RPCNode) blockByNumber(ctx context.Context, blockNumber *big.Int) (execution.Block, error) { start := time.Now() block, err := n.client.BlockByNumber(ctx, blockNumber) @@ -52,9 +54,9 @@ func (n *Node) BlockByNumber(ctx context.Context, blockNumber *big.Int) (*types. duration := time.Since(start) // Record RPC metrics - status := STATUS_SUCCESS + status := statusSuccess if err != nil { - status = STATUS_ERROR + status = statusError } network := n.Metadata().ChainID() @@ -66,11 +68,11 @@ func (n *Node) BlockByNumber(ctx context.Context, blockNumber *big.Int) (*types. return nil, err } - return block, nil + return NewBlockAdapter(block), nil } // getTraceParams returns VM trace parameters with configurable options. -func getTraceParams(hash string, options TraceOptions) []any { +func getTraceParams(hash string, options execution.TraceOptions) []any { return []any{ hash, map[string]any{ @@ -83,8 +85,8 @@ func getTraceParams(hash string, options TraceOptions) []any { } // traceTransactionErigon handles tracing for Erigon clients. -func (n *Node) traceTransactionErigon(ctx context.Context, hash string, options TraceOptions) (*TraceTransaction, error) { - var rsp ErigonResult +func (n *RPCNode) traceTransactionErigon(ctx context.Context, hash string, options execution.TraceOptions) (*execution.TraceTransaction, error) { + var rsp erigonResult start := time.Now() @@ -93,9 +95,9 @@ func (n *Node) traceTransactionErigon(ctx context.Context, hash string, options duration := time.Since(start) // Record RPC metrics - status := STATUS_SUCCESS + status := statusSuccess if err != nil { - status = STATUS_ERROR + status = statusError } network := n.Metadata().ChainID() @@ -112,11 +114,11 @@ func (n *Node) traceTransactionErigon(ctx context.Context, hash string, options returnValue = nil } - result := &TraceTransaction{ + result := &execution.TraceTransaction{ Gas: rsp.Gas, Failed: rsp.Failed, ReturnValue: returnValue, - Structlogs: []StructLog{}, + Structlogs: []execution.StructLog{}, } // Empty array on transfer @@ -128,7 +130,7 @@ func (n *Node) traceTransactionErigon(ctx context.Context, hash string, options *returnData = hex.EncodeToString(log.ReturnData) } - result.Structlogs = append(result.Structlogs, StructLog{ + result.Structlogs = append(result.Structlogs, execution.StructLog{ PC: log.PC, Op: log.Op, Gas: log.Gas, @@ -144,8 +146,8 @@ func (n *Node) traceTransactionErigon(ctx context.Context, hash string, options return result, nil } -// BlockReceipts fetches all receipts for a block by number (much faster than per-tx). -func (n *Node) BlockReceipts(ctx context.Context, blockNumber *big.Int) ([]*types.Receipt, error) { +// blockReceipts fetches all receipts for a block by number (much faster than per-tx). +func (n *RPCNode) blockReceipts(ctx context.Context, blockNumber *big.Int) ([]execution.Receipt, error) { start := time.Now() blockNrOrHash := rpc.BlockNumberOrHashWithNumber(rpc.BlockNumber(blockNumber.Int64())) @@ -155,9 +157,9 @@ func (n *Node) BlockReceipts(ctx context.Context, blockNumber *big.Int) ([]*type duration := time.Since(start) // Record RPC metrics - status := STATUS_SUCCESS + status := statusSuccess if err != nil { - status = STATUS_ERROR + status = statusError } network := n.Metadata().ChainID() @@ -180,11 +182,11 @@ func (n *Node) BlockReceipts(ctx context.Context, blockNumber *big.Int) ([]*type return nil, err } - return receipts, nil + return AdaptReceipts(receipts), nil } -// TransactionReceipt fetches the receipt for a transaction by hash. -func (n *Node) TransactionReceipt(ctx context.Context, hash string) (*types.Receipt, error) { +// transactionReceipt fetches the receipt for a transaction by hash. +func (n *RPCNode) transactionReceipt(ctx context.Context, hash string) (execution.Receipt, error) { start := time.Now() txHash := common.HexToHash(hash) @@ -194,9 +196,9 @@ func (n *Node) TransactionReceipt(ctx context.Context, hash string) (*types.Rece duration := time.Since(start) // Record RPC metrics - status := STATUS_SUCCESS + status := statusSuccess if err != nil { - status = STATUS_ERROR + status = statusError } network := n.Metadata().ChainID() @@ -219,11 +221,16 @@ func (n *Node) TransactionReceipt(ctx context.Context, hash string) (*types.Rece return nil, err } - return receipt, nil + return NewReceiptAdapter(receipt), nil } -// DebugTraceTransaction traces a transaction execution using the client's debug API. -func (n *Node) DebugTraceTransaction(ctx context.Context, hash string, blockNumber *big.Int, options TraceOptions) (*TraceTransaction, error) { +// debugTraceTransaction traces a transaction execution using the client's debug API. +func (n *RPCNode) debugTraceTransaction( + ctx context.Context, + hash string, + _ *big.Int, + options execution.TraceOptions, +) (*execution.TraceTransaction, error) { // Add a timeout if the context doesn't already have one if _, hasDeadline := ctx.Deadline(); !hasDeadline { var cancel context.CancelFunc @@ -251,3 +258,24 @@ func (n *Node) DebugTraceTransaction(ctx context.Context, hash string, blockNumb return n.traceTransactionErigon(ctx, hash, options) } } + +// erigonResult represents the result from an Erigon debug_traceTransaction call. +type erigonResult struct { + Gas uint64 `json:"gas"` + Failed bool `json:"failed"` + ReturnValue *string `json:"returnValue"` + StructLogs []erigonStructLog `json:"structLogs"` +} + +// erigonStructLog represents a single structlog entry from Erigon. +type erigonStructLog struct { + PC uint32 `json:"pc"` + Op string `json:"op"` + Gas uint64 `json:"gas"` + GasCost uint64 `json:"gasCost"` + Depth uint64 `json:"depth"` + ReturnData []byte `json:"returnData"` + Refund *uint64 `json:"refund"` + Error *string `json:"error"` + Stack *[]string `json:"stack"` +} diff --git a/pkg/ethereum/execution/node.go b/pkg/ethereum/execution/geth/rpc_node.go similarity index 68% rename from pkg/ethereum/execution/node.go rename to pkg/ethereum/execution/geth/rpc_node.go index d070193..394fc54 100644 --- a/pkg/ethereum/execution/node.go +++ b/pkg/ethereum/execution/geth/rpc_node.go @@ -1,9 +1,12 @@ -package execution +//go:build !embedded + +package geth import ( "context" "errors" "fmt" + "math/big" "net" "net/http" "sync" @@ -11,10 +14,15 @@ import ( "github.com/ethereum/go-ethereum/ethclient" "github.com/ethereum/go-ethereum/rpc" - "github.com/ethpandaops/execution-processor/pkg/ethereum/execution/services" "github.com/sirupsen/logrus" + + "github.com/ethpandaops/execution-processor/pkg/ethereum/execution" + "github.com/ethpandaops/execution-processor/pkg/ethereum/execution/geth/services" ) +// Compile-time check that RPCNode implements execution.Node interface. +var _ execution.Node = (*RPCNode)(nil) + // headerTransport adds custom headers to requests and respects context cancellation. type headerTransport struct { headers map[string]string @@ -36,8 +44,9 @@ func (t *headerTransport) RoundTrip(req *http.Request) (*http.Response, error) { return t.base.RoundTrip(req) } -type Node struct { - config *Config +// RPCNode implements execution.Node using JSON-RPC connections. +type RPCNode struct { + config *execution.Config log logrus.FieldLogger client *ethclient.Client rpcClient *rpc.Client @@ -53,19 +62,20 @@ type Node struct { cancel context.CancelFunc } -func NewNode(log logrus.FieldLogger, conf *Config) *Node { - return &Node{ +// NewRPCNode creates a new RPC-based execution node. +func NewRPCNode(log logrus.FieldLogger, conf *execution.Config) *RPCNode { + return &RPCNode{ config: conf, log: log.WithFields(logrus.Fields{"type": "execution", "source": conf.Name}), services: []services.Service{}, } } -func (n *Node) OnReady(_ context.Context, callback func(ctx context.Context) error) { +func (n *RPCNode) OnReady(_ context.Context, callback func(ctx context.Context) error) { n.onReadyCallbacks = append(n.onReadyCallbacks, callback) } -func (n *Node) Start(ctx context.Context) error { +func (n *RPCNode) Start(ctx context.Context) error { n.log.WithFields(logrus.Fields{ "node_name": n.name, }).Info("Starting execution node") @@ -193,7 +203,7 @@ func (n *Node) Start(ctx context.Context) error { return nil } -func (n *Node) Stop(ctx context.Context) error { +func (n *RPCNode) Stop(ctx context.Context) error { n.log.Info("Stopping execution node") // Cancel the node context to signal all goroutines to stop @@ -230,7 +240,7 @@ func (n *Node) Stop(ctx context.Context) error { return nil } -func (n *Node) getServiceByName(name services.Name) (services.Service, error) { +func (n *RPCNode) getServiceByName(name services.Name) (services.Service, error) { for _, service := range n.services { if service.Name() == name { return service, nil @@ -240,7 +250,8 @@ func (n *Node) getServiceByName(name services.Name) (services.Service, error) { return nil, errors.New("service not found") } -func (n *Node) Metadata() *services.MetadataService { +// Metadata returns the metadata service for this node. +func (n *RPCNode) Metadata() *services.MetadataService { service, err := n.getServiceByName("metadata") if err != nil { // This should never happen. If it does, good luck. @@ -255,6 +266,64 @@ func (n *Node) Metadata() *services.MetadataService { return svc } -func (n *Node) Name() string { +// Name returns the configured name for this node. +func (n *RPCNode) Name() string { return n.config.Name } + +// ChainID returns the chain ID from the metadata service. +func (n *RPCNode) ChainID() int64 { + if meta := n.Metadata(); meta != nil { + return meta.ChainID() + } + + return 0 +} + +// ClientType returns the client type from the metadata service. +func (n *RPCNode) ClientType() string { + if meta := n.Metadata(); meta != nil { + return meta.ClientVersion() + } + + return "" +} + +// IsSynced returns true if the node is synced. +func (n *RPCNode) IsSynced() bool { + if meta := n.Metadata(); meta != nil { + return meta.IsSynced() + } + + return false +} + +// BlockNumber returns the current block number. +func (n *RPCNode) BlockNumber(ctx context.Context) (*uint64, error) { + return n.blockNumber(ctx) +} + +// BlockByNumber returns the block at the given number. +func (n *RPCNode) BlockByNumber(ctx context.Context, number *big.Int) (execution.Block, error) { + return n.blockByNumber(ctx, number) +} + +// BlockReceipts returns all receipts for the block at the given number. +func (n *RPCNode) BlockReceipts(ctx context.Context, number *big.Int) ([]execution.Receipt, error) { + return n.blockReceipts(ctx, number) +} + +// TransactionReceipt returns the receipt for the transaction with the given hash. +func (n *RPCNode) TransactionReceipt(ctx context.Context, hash string) (execution.Receipt, error) { + return n.transactionReceipt(ctx, hash) +} + +// DebugTraceTransaction returns the execution trace for the transaction. +func (n *RPCNode) DebugTraceTransaction( + ctx context.Context, + hash string, + blockNumber *big.Int, + opts execution.TraceOptions, +) (*execution.TraceTransaction, error) { + return n.debugTraceTransaction(ctx, hash, blockNumber, opts) +} diff --git a/pkg/ethereum/execution/services/client.go b/pkg/ethereum/execution/geth/services/client.go similarity index 97% rename from pkg/ethereum/execution/services/client.go rename to pkg/ethereum/execution/geth/services/client.go index 729c8e5..eb00df0 100644 --- a/pkg/ethereum/execution/services/client.go +++ b/pkg/ethereum/execution/geth/services/client.go @@ -1,3 +1,5 @@ +//go:build !embedded + package services import ( diff --git a/pkg/ethereum/execution/services/metadata.go b/pkg/ethereum/execution/geth/services/metadata.go similarity index 99% rename from pkg/ethereum/execution/services/metadata.go rename to pkg/ethereum/execution/geth/services/metadata.go index 353a1f9..d0314ae 100644 --- a/pkg/ethereum/execution/services/metadata.go +++ b/pkg/ethereum/execution/geth/services/metadata.go @@ -1,3 +1,5 @@ +//go:build !embedded + package services import ( diff --git a/pkg/ethereum/execution/services/service.go b/pkg/ethereum/execution/geth/services/service.go similarity index 92% rename from pkg/ethereum/execution/services/service.go rename to pkg/ethereum/execution/geth/services/service.go index 7048e47..3a103fa 100644 --- a/pkg/ethereum/execution/services/service.go +++ b/pkg/ethereum/execution/geth/services/service.go @@ -1,3 +1,5 @@ +//go:build !embedded + package services import "context" diff --git a/pkg/ethereum/execution/interface.go b/pkg/ethereum/execution/interface.go new file mode 100644 index 0000000..eb994a0 --- /dev/null +++ b/pkg/ethereum/execution/interface.go @@ -0,0 +1,64 @@ +package execution + +import ( + "context" + "math/big" +) + +// Node defines the interface for execution data providers. +// +// Implementations include: +// - geth.RPCNode: connects to execution clients via JSON-RPC over HTTP +// - EmbeddedNode: receives data directly from host application via DataSource +// +// All methods must be safe for concurrent use by multiple goroutines. +// +// Lifecycle: +// 1. Create node with appropriate constructor (geth.NewRPCNode or NewEmbeddedNode) +// 2. Register OnReady callbacks before calling Start +// 3. Call Start to begin initialization +// 4. Node signals readiness by executing OnReady callbacks +// 5. Call Stop for graceful shutdown +type Node interface { + // Start initializes the node and begins any background operations. + // For RPCNode, this establishes the RPC connection and starts health monitoring. + // For EmbeddedNode, this is a no-op as the host controls the DataSource lifecycle. + Start(ctx context.Context) error + + // Stop gracefully shuts down the node and releases resources. + // Should be called when the node is no longer needed. + Stop(ctx context.Context) error + + // OnReady registers a callback to be invoked when the node becomes ready. + // For RPCNode, callbacks execute when the RPC connection is healthy. + // For EmbeddedNode, callbacks execute when MarkReady is called by the host. + // Multiple callbacks can be registered and will execute in registration order. + OnReady(ctx context.Context, callback func(ctx context.Context) error) + + // BlockNumber returns the current block number from the execution client. + BlockNumber(ctx context.Context) (*uint64, error) + + // BlockByNumber returns the block at the given number. + BlockByNumber(ctx context.Context, number *big.Int) (Block, error) + + // BlockReceipts returns all receipts for the block at the given number. + BlockReceipts(ctx context.Context, number *big.Int) ([]Receipt, error) + + // TransactionReceipt returns the receipt for the transaction with the given hash. + TransactionReceipt(ctx context.Context, hash string) (Receipt, error) + + // DebugTraceTransaction returns the execution trace for the transaction. + DebugTraceTransaction(ctx context.Context, hash string, blockNumber *big.Int, opts TraceOptions) (*TraceTransaction, error) + + // ChainID returns the chain ID reported by the execution client. + ChainID() int64 + + // ClientType returns the client type/version string (e.g., "geth/1.10.0"). + ClientType() string + + // IsSynced returns true if the execution client is fully synced. + IsSynced() bool + + // Name returns the configured name for this node. + Name() string +} diff --git a/pkg/ethereum/execution/structlog.go b/pkg/ethereum/execution/structlog.go index 4deeb2a..5b0c856 100644 --- a/pkg/ethereum/execution/structlog.go +++ b/pkg/ethereum/execution/structlog.go @@ -8,14 +8,51 @@ type TraceTransaction struct { Structlogs []StructLog } +// StructLog represents a single EVM opcode execution trace entry. +// +// This struct supports two operation modes: +// - RPC mode: Stack is populated for CALL opcodes, CallToAddress/GasUsed computed post-hoc +// - Embedded mode: CallToAddress/GasUsed pre-computed by tracer, Stack remains nil +// +// The embedded mode optimizations eliminate ~99% of stack-related allocations +// and remove the post-processing GasUsed computation pass. type StructLog struct { - PC uint32 `json:"pc"` - Op string `json:"op"` - Gas uint64 `json:"gas"` - GasCost uint64 `json:"gasCost"` - Depth uint64 `json:"depth"` - ReturnData *string `json:"returnData"` - Refund *uint64 `json:"refund,omitempty"` - Error *string `json:"error,omitempty"` - Stack *[]string `json:"stack,omitempty"` + // PC is the program counter. Kept for RPC backward compatibility but not + // populated in embedded mode (always 0). + PC uint32 `json:"pc"` + + // Op is the opcode name (e.g., "PUSH1", "CALL", "SSTORE"). + Op string `json:"op"` + + // Gas is the remaining gas before this opcode executes. + Gas uint64 `json:"gas"` + + // GasCost is the static gas cost of the opcode (may differ from actual GasUsed). + GasCost uint64 `json:"gasCost"` + + // GasUsed is the actual gas consumed by this opcode. + // In embedded mode: pre-computed by tracer using gas difference to next opcode. + // In RPC mode: computed post-hoc by ComputeGasUsed(), this field will be 0. + GasUsed uint64 `json:"gasUsed,omitempty"` + + // Depth is the call stack depth (1 = top-level, increases with CALL/CREATE). + Depth uint64 `json:"depth"` + + // ReturnData contains the return data from the last CALL/STATICCALL/etc. + ReturnData *string `json:"returnData"` + + // Refund is the gas refund counter value. + Refund *uint64 `json:"refund,omitempty"` + + // Error contains any error message if the opcode failed. + Error *string `json:"error,omitempty"` + + // Stack contains the EVM stack state (RPC mode only). + // In embedded mode this is nil - use CallToAddress instead. + Stack *[]string `json:"stack,omitempty"` + + // CallToAddress is the target address for CALL/STATICCALL/DELEGATECALL/CALLCODE. + // In embedded mode: pre-extracted by tracer from stack[len-2]. + // In RPC mode: nil, extracted post-hoc from Stack by extractCallAddress(). + CallToAddress *string `json:"callToAddress,omitempty"` } diff --git a/pkg/ethereum/pool.go b/pkg/ethereum/pool.go index 2d122e0..d04297c 100644 --- a/pkg/ethereum/pool.go +++ b/pkg/ethereum/pool.go @@ -14,35 +14,56 @@ import ( type Pool struct { log logrus.FieldLogger - executionNodes []*execution.Node + executionNodes []execution.Node metrics *Metrics config *Config mu sync.RWMutex - healthyExecutionNodes map[*execution.Node]bool + healthyExecutionNodes map[execution.Node]bool // Goroutine management wg sync.WaitGroup cancel context.CancelFunc } -func NewPool(log logrus.FieldLogger, namespace string, config *Config) *Pool { +// NewPoolWithNodes creates a pool with pre-created Node implementations. +// Use this when embedding execution-processor as a library where the host +// provides custom Node implementations (e.g., EmbeddedNode with DataSource). +// +// Parameters: +// - log: Logger for pool operations +// - namespace: Metrics namespace prefix (will have "_ethereum" appended) +// - nodes: Pre-created Node implementations +// - config: Optional configuration (nil creates empty config with defaults) +// +// Example: +// +// // Create embedded node with custom data source +// dataSource := &MyDataSource{client: myClient} +// node := execution.NewEmbeddedNode(log, "my-node", dataSource) +// +// // Create pool with the embedded node +// pool := ethereum.NewPoolWithNodes(log, "processor", []execution.Node{node}, nil) +// pool.Start(ctx) +// +// // Mark ready when data source is ready +// node.MarkReady(ctx) +func NewPoolWithNodes(log logrus.FieldLogger, namespace string, nodes []execution.Node, config *Config) *Pool { namespace = fmt.Sprintf("%s_ethereum", namespace) - p := &Pool{ + + // If config is nil, create an empty config + if config == nil { + config = &Config{} + } + + return &Pool{ log: log, - executionNodes: make([]*execution.Node, 0), - healthyExecutionNodes: make(map[*execution.Node]bool), + executionNodes: nodes, + healthyExecutionNodes: make(map[execution.Node]bool, len(nodes)), metrics: GetMetricsInstance(namespace), config: config, } - - for _, execCfg := range config.Execution { - node := execution.NewNode(log, execCfg) - p.executionNodes = append(p.executionNodes, node) - } - - return p } func (p *Pool) HasExecutionNodes() bool { @@ -62,11 +83,11 @@ func (p *Pool) HasHealthyExecutionNodes() bool { return false } -func (p *Pool) GetHealthyExecutionNodes() []*execution.Node { +func (p *Pool) GetHealthyExecutionNodes() []execution.Node { p.mu.RLock() defer p.mu.RUnlock() - var healthyNodes []*execution.Node + healthyNodes := make([]execution.Node, 0, len(p.healthyExecutionNodes)) for node, healthy := range p.healthyExecutionNodes { if healthy { @@ -77,11 +98,11 @@ func (p *Pool) GetHealthyExecutionNodes() []*execution.Node { return healthyNodes } -func (p *Pool) GetHealthyExecutionNode() *execution.Node { +func (p *Pool) GetHealthyExecutionNode() execution.Node { p.mu.RLock() defer p.mu.RUnlock() - var healthyNodes []*execution.Node + healthyNodes := make([]execution.Node, 0, len(p.healthyExecutionNodes)) for node, healthy := range p.healthyExecutionNodes { if healthy { @@ -97,7 +118,7 @@ func (p *Pool) GetHealthyExecutionNode() *execution.Node { return healthyNodes[rand.IntN(len(healthyNodes))] } -func (p *Pool) WaitForHealthyExecutionNode(ctx context.Context) (*execution.Node, error) { +func (p *Pool) WaitForHealthyExecutionNode(ctx context.Context) (execution.Node, error) { // Check if we have any execution nodes configured if len(p.executionNodes) == 0 { return nil, fmt.Errorf("no execution nodes configured") @@ -193,15 +214,24 @@ func (p *Pool) Start(ctx context.Context) { p.UpdateNodeMetrics() for _, node := range p.executionNodes { - g.Go(func() error { - node.OnReady(ctx, func(innerCtx context.Context) error { - p.mu.Lock() - p.healthyExecutionNodes[node] = true - p.mu.Unlock() - - return nil - }) + // Register OnReady callbacks synchronously BEFORE spawning goroutines. + // This ensures callbacks are registered before Start() returns, so any + // subsequent call to MarkReady() (for EmbeddedNode) will find the callback. + // Previously, registration happened inside g.Go() which created a race + // condition where MarkReady() could execute before callbacks were registered. + p.log.WithField("node", node.Name()).Info("Registering OnReady callback for node") + node.OnReady(ctx, func(innerCtx context.Context) error { + p.log.WithField("node", node.Name()).Info("OnReady callback executed, marking node healthy") + p.mu.Lock() + p.healthyExecutionNodes[node] = true + p.mu.Unlock() + + return nil + }) + // Start node asynchronously - the actual initialization can be slow + // (e.g., RPC connection establishment), but callback registration is instant. + g.Go(func() error { return node.Start(ctx) }) } diff --git a/pkg/ethereum/pool_rpc.go b/pkg/ethereum/pool_rpc.go new file mode 100644 index 0000000..38ef9e8 --- /dev/null +++ b/pkg/ethereum/pool_rpc.go @@ -0,0 +1,32 @@ +//go:build !embedded + +package ethereum + +import ( + "fmt" + + "github.com/ethpandaops/execution-processor/pkg/ethereum/execution" + "github.com/ethpandaops/execution-processor/pkg/ethereum/execution/geth" + "github.com/sirupsen/logrus" +) + +// NewPool creates a new pool from config, using RPC nodes. +// This function imports go-ethereum types through the geth package. +// For embedded mode (no go-ethereum dependency), use NewPoolWithNodes instead. +func NewPool(log logrus.FieldLogger, namespace string, config *Config) *Pool { + namespace = fmt.Sprintf("%s_ethereum", namespace) + p := &Pool{ + log: log, + executionNodes: make([]execution.Node, 0, len(config.Execution)), + healthyExecutionNodes: make(map[execution.Node]bool, len(config.Execution)), + metrics: GetMetricsInstance(namespace), + config: config, + } + + for _, execCfg := range config.Execution { + node := geth.NewRPCNode(log, execCfg) + p.executionNodes = append(p.executionNodes, node) + } + + return p +} diff --git a/pkg/ethereum/pool_test.go b/pkg/ethereum/pool_test.go index 05c4375..9e7323a 100644 --- a/pkg/ethereum/pool_test.go +++ b/pkg/ethereum/pool_test.go @@ -2,6 +2,7 @@ package ethereum_test import ( "context" + "math/big" "sync" "testing" "time" @@ -13,6 +14,126 @@ import ( "github.com/stretchr/testify/require" ) +// MockBlock implements execution.Block for testing. +type MockBlock struct { + number *big.Int +} + +func (b *MockBlock) Number() *big.Int { return b.number } +func (b *MockBlock) Hash() execution.Hash { return execution.Hash{} } +func (b *MockBlock) ParentHash() execution.Hash { return execution.Hash{} } +func (b *MockBlock) BaseFee() *big.Int { return nil } +func (b *MockBlock) Transactions() []execution.Transaction { return nil } + +// MockReceipt implements execution.Receipt for testing. +type MockReceipt struct{} + +func (r *MockReceipt) Status() uint64 { return 1 } +func (r *MockReceipt) TxHash() execution.Hash { return execution.Hash{} } +func (r *MockReceipt) GasUsed() uint64 { return 21000 } + +// MockNode implements execution.Node for testing. +type MockNode struct { + name string + started bool + stopped bool + onReadyCallbacks []func(ctx context.Context) error + mu sync.Mutex +} + +func NewMockNode(name string) *MockNode { + return &MockNode{ + name: name, + onReadyCallbacks: make([]func(ctx context.Context) error, 0), + } +} + +func (m *MockNode) Start(_ context.Context) error { + m.mu.Lock() + defer m.mu.Unlock() + + m.started = true + + return nil +} + +func (m *MockNode) Stop(_ context.Context) error { + m.mu.Lock() + defer m.mu.Unlock() + + m.stopped = true + + return nil +} + +func (m *MockNode) OnReady(_ context.Context, callback func(ctx context.Context) error) { + m.mu.Lock() + defer m.mu.Unlock() + + m.onReadyCallbacks = append(m.onReadyCallbacks, callback) +} + +// TriggerReady simulates the node becoming ready by calling all OnReady callbacks. +func (m *MockNode) TriggerReady(ctx context.Context) error { + m.mu.Lock() + callbacks := m.onReadyCallbacks + m.mu.Unlock() + + for _, cb := range callbacks { + if err := cb(ctx); err != nil { + return err + } + } + + return nil +} + +func (m *MockNode) BlockNumber(_ context.Context) (*uint64, error) { + num := uint64(12345) + + return &num, nil +} + +func (m *MockNode) BlockByNumber(_ context.Context, number *big.Int) (execution.Block, error) { + return &MockBlock{number: number}, nil +} + +func (m *MockNode) BlockReceipts(_ context.Context, _ *big.Int) ([]execution.Receipt, error) { + return []execution.Receipt{}, nil +} + +func (m *MockNode) TransactionReceipt(_ context.Context, _ string) (execution.Receipt, error) { + return &MockReceipt{}, nil +} + +func (m *MockNode) DebugTraceTransaction( + _ context.Context, + _ string, + _ *big.Int, + _ execution.TraceOptions, +) (*execution.TraceTransaction, error) { + return &execution.TraceTransaction{}, nil +} + +func (m *MockNode) ChainID() int64 { + return 1 +} + +func (m *MockNode) ClientType() string { + return "mock" +} + +func (m *MockNode) IsSynced() bool { + return true +} + +func (m *MockNode) Name() string { + return m.name +} + +// Compile-time check that MockNode implements execution.Node. +var _ execution.Node = (*MockNode)(nil) + func TestPool_Creation(t *testing.T) { log := logrus.New() log.SetLevel(logrus.ErrorLevel) @@ -448,3 +569,240 @@ func TestPool_NodeSelection(t *testing.T) { func stringPtr(s string) *string { return &s } + +// Tests for NewPoolWithNodes + +func TestPool_NewPoolWithNodes_Basic(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + node1 := NewMockNode("mock-node-1") + node2 := NewMockNode("mock-node-2") + + nodes := []execution.Node{node1, node2} + pool := ethereum.NewPoolWithNodes(log, "test", nodes, nil) + + require.NotNil(t, pool) + assert.True(t, pool.HasExecutionNodes()) + assert.False(t, pool.HasHealthyExecutionNodes()) +} + +func TestPool_NewPoolWithNodes_NilConfig(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + node := NewMockNode("mock-node") + + nodes := []execution.Node{node} + pool := ethereum.NewPoolWithNodes(log, "test", nodes, nil) + + require.NotNil(t, pool) + assert.True(t, pool.HasExecutionNodes()) + + // Verify default config behavior - unknown chain ID should error + network, err := pool.GetNetworkByChainID(999999) + assert.Error(t, err) + assert.Nil(t, network) +} + +func TestPool_NewPoolWithNodes_EmptyNodes(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + nodes := []execution.Node{} + pool := ethereum.NewPoolWithNodes(log, "test", nodes, nil) + + require.NotNil(t, pool) + assert.False(t, pool.HasExecutionNodes()) + assert.False(t, pool.HasHealthyExecutionNodes()) + assert.Nil(t, pool.GetHealthyExecutionNode()) + assert.Empty(t, pool.GetHealthyExecutionNodes()) +} + +func TestPool_NewPoolWithNodes_MultipleNodes(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + const numNodes = 5 + + nodes := make([]execution.Node, numNodes) + for i := 0; i < numNodes; i++ { + nodes[i] = NewMockNode("mock-node-" + string(rune('a'+i))) + } + + pool := ethereum.NewPoolWithNodes(log, "test", nodes, nil) + + require.NotNil(t, pool) + assert.True(t, pool.HasExecutionNodes()) +} + +func TestPool_NewPoolWithNodes_WithConfig(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + node := NewMockNode("mock-node") + overrideName := "custom-network" + + config := ðereum.Config{ + OverrideNetworkName: &overrideName, + } + + nodes := []execution.Node{node} + pool := ethereum.NewPoolWithNodes(log, "test", nodes, config) + + require.NotNil(t, pool) + + // Verify config is used - override name should be returned + network, err := pool.GetNetworkByChainID(999999) + require.NoError(t, err) + assert.Equal(t, "custom-network", network.Name) +} + +func TestPool_NewPoolWithNodes_StartStop(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + node := NewMockNode("mock-node") + + nodes := []execution.Node{node} + pool := ethereum.NewPoolWithNodes(log, "test", nodes, nil) + + ctx := context.Background() + pool.Start(ctx) + + // Wait for async Start goroutines to execute + time.Sleep(50 * time.Millisecond) + + // Node should have been started + node.mu.Lock() + assert.True(t, node.started) + node.mu.Unlock() + + stopCtx, cancel := context.WithTimeout(context.Background(), 2*time.Second) + defer cancel() + + err := pool.Stop(stopCtx) + assert.NoError(t, err) + + // Node should have been stopped + node.mu.Lock() + assert.True(t, node.stopped) + node.mu.Unlock() +} + +func TestPool_NewPoolWithNodes_NodeBecomesHealthy(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + node := NewMockNode("mock-node") + + nodes := []execution.Node{node} + pool := ethereum.NewPoolWithNodes(log, "test", nodes, nil) + + ctx := context.Background() + pool.Start(ctx) + + defer func() { + stopCtx, cancel := context.WithTimeout(context.Background(), 2*time.Second) + defer cancel() + + err := pool.Stop(stopCtx) + assert.NoError(t, err) + }() + + // Wait for async Start goroutines to register callbacks + time.Sleep(50 * time.Millisecond) + + // Initially no healthy nodes + assert.False(t, pool.HasHealthyExecutionNodes()) + + // Trigger the node to become ready (simulates OnReady callback) + err := node.TriggerReady(ctx) + require.NoError(t, err) + + // Now the pool should have a healthy node + assert.True(t, pool.HasHealthyExecutionNodes()) + assert.NotNil(t, pool.GetHealthyExecutionNode()) +} + +func TestPool_EmbeddedNodeIntegration(t *testing.T) { + log := logrus.New() + log.SetLevel(logrus.ErrorLevel) + + // Create an EmbeddedNode with a mock data source + ds := &testDataSource{} + embeddedNode := execution.NewEmbeddedNode(log, "embedded-test", ds) + + nodes := []execution.Node{embeddedNode} + pool := ethereum.NewPoolWithNodes(log, "test", nodes, nil) + + ctx := context.Background() + pool.Start(ctx) + + defer func() { + stopCtx, cancel := context.WithTimeout(context.Background(), 2*time.Second) + defer cancel() + + err := pool.Stop(stopCtx) + assert.NoError(t, err) + }() + + // Wait for async Start goroutines to register callbacks + time.Sleep(50 * time.Millisecond) + + // Initially no healthy nodes + assert.False(t, pool.HasHealthyExecutionNodes()) + + // Mark embedded node as ready + err := embeddedNode.MarkReady(ctx) + require.NoError(t, err) + + // Now the pool should have a healthy node + assert.True(t, pool.HasHealthyExecutionNodes()) + + healthyNode := pool.GetHealthyExecutionNode() + require.NotNil(t, healthyNode) + assert.Equal(t, "embedded-test", healthyNode.Name()) +} + +// testDataSource is a minimal DataSource implementation for integration tests. +type testDataSource struct{} + +func (ds *testDataSource) BlockNumber(_ context.Context) (*uint64, error) { + num := uint64(12345) + + return &num, nil +} + +func (ds *testDataSource) BlockByNumber(_ context.Context, number *big.Int) (execution.Block, error) { + return &MockBlock{number: number}, nil +} + +func (ds *testDataSource) BlockReceipts(_ context.Context, _ *big.Int) ([]execution.Receipt, error) { + return []execution.Receipt{}, nil +} + +func (ds *testDataSource) TransactionReceipt(_ context.Context, _ string) (execution.Receipt, error) { + return &MockReceipt{}, nil +} + +func (ds *testDataSource) DebugTraceTransaction( + _ context.Context, + _ string, + _ *big.Int, + _ execution.TraceOptions, +) (*execution.TraceTransaction, error) { + return &execution.TraceTransaction{}, nil +} + +func (ds *testDataSource) ChainID() int64 { + return 1 +} + +func (ds *testDataSource) ClientType() string { + return "test" +} + +func (ds *testDataSource) IsSynced() bool { + return true +} diff --git a/pkg/processor/manager.go b/pkg/processor/manager.go index 01b771e..eb094f4 100644 --- a/pkg/processor/manager.go +++ b/pkg/processor/manager.go @@ -160,7 +160,7 @@ func (m *Manager) Start(ctx context.Context) error { return fmt.Errorf("no healthy execution node available") } - m.network, err = m.pool.GetNetworkByChainID(node.Metadata().ChainID()) + m.network, err = m.pool.GetNetworkByChainID(node.ChainID()) if err != nil { return fmt.Errorf("failed to get network by chain ID: %w", err) } @@ -1070,7 +1070,7 @@ func (m *Manager) shouldSkipBlockProcessing(ctx context.Context) (bool, string) // GetQueueName returns the current queue name based on processing mode. func (m *Manager) GetQueueName() string { // For now we only have one processor - processorName := "transaction-structlog" + processorName := "transaction_structlog" if m.config.Mode == tracker.BACKWARDS_MODE { return tracker.PrefixedProcessBackwardsQueue(processorName, m.redisPrefix) } diff --git a/pkg/processor/transaction/simple/handlers.go b/pkg/processor/transaction/simple/handlers.go index f33eef6..0868ebe 100644 --- a/pkg/processor/transaction/simple/handlers.go +++ b/pkg/processor/transaction/simple/handlers.go @@ -7,11 +7,11 @@ import ( "time" "github.com/ClickHouse/ch-go" - "github.com/ethereum/go-ethereum/core/types" "github.com/hibiken/asynq" "github.com/sirupsen/logrus" "github.com/ethpandaops/execution-processor/pkg/common" + "github.com/ethpandaops/execution-processor/pkg/ethereum/execution" "github.com/ethpandaops/execution-processor/pkg/processor/tracker" ) @@ -107,13 +107,13 @@ func (p *Processor) handleProcessTask(ctx context.Context, task *asynq.Task) err blockTxs := block.Transactions() // Build receipt map - try batch first, fall back to per-tx - receiptMap := make(map[string]*types.Receipt, len(blockTxs)) + receiptMap := make(map[string]execution.Receipt, len(blockTxs)) receipts, err := node.BlockReceipts(ctx, blockNumber) if err == nil { // Use batch receipts for _, r := range receipts { - receiptMap[r.TxHash.Hex()] = r + receiptMap[r.TxHash().Hex()] = r } } @@ -179,26 +179,13 @@ func (p *Processor) handleProcessTask(ctx context.Context, task *asynq.Task) err // buildTransactionRow builds a transaction row from block, tx, and receipt data. func (p *Processor) buildTransactionRow( - block *types.Block, - tx *types.Transaction, - receipt *types.Receipt, + block execution.Block, + tx execution.Transaction, + receipt execution.Receipt, index uint64, ) (Transaction, error) { - // Get sender (from) - handle legacy transactions without chain ID - var signer types.Signer - - chainID := tx.ChainId() - if chainID == nil || chainID.Sign() == 0 { - // Legacy transaction without EIP-155 replay protection - signer = types.HomesteadSigner{} - } else { - signer = types.LatestSignerForChainID(chainID) - } - - from, err := types.Sender(signer, tx) - if err != nil { - return Transaction{}, fmt.Errorf("failed to get sender: %w", err) - } + // Get sender (from) - computed by the data source + from := tx.From() // Build to address (nil for contract creation) var toAddress *string @@ -269,7 +256,7 @@ func (p *Processor) buildTransactionRow( Size: txSize, CallDataSize: callDataSize, BlobHashes: []string{}, // Default empty array - Success: receipt.Status == 1, + Success: receipt.Status() == 1, NInputBytes: callDataSize, NInputZeroBytes: nInputZeroBytes, NInputNonzeroBytes: nInputNonzeroBytes, @@ -277,7 +264,7 @@ func (p *Processor) buildTransactionRow( } // Handle blob transaction fields (type 3) - if tx.Type() == types.BlobTxType { + if tx.Type() == execution.BlobTxType { blobGas := tx.BlobGas() txRow.BlobGas = &blobGas @@ -301,11 +288,11 @@ func (p *Processor) buildTransactionRow( // calculateEffectiveGasPrice calculates the effective gas price for a transaction. // For legacy/access list txs: returns tx.GasPrice(). // For EIP-1559+ txs: returns min(max_fee_per_gas, base_fee + max_priority_fee_per_gas). -func calculateEffectiveGasPrice(block *types.Block, tx *types.Transaction) *big.Int { +func calculateEffectiveGasPrice(block execution.Block, tx execution.Transaction) *big.Int { txType := tx.Type() // Legacy and access list transactions use GasPrice directly - if txType == types.LegacyTxType || txType == types.AccessListTxType { + if txType == execution.LegacyTxType || txType == execution.AccessListTxType { if tx.GasPrice() != nil { return tx.GasPrice() } diff --git a/pkg/processor/transaction/structlog/block_processing.go b/pkg/processor/transaction/structlog/block_processing.go index 55141a7..8801961 100644 --- a/pkg/processor/transaction/structlog/block_processing.go +++ b/pkg/processor/transaction/structlog/block_processing.go @@ -6,11 +6,11 @@ import ( "fmt" "math/big" - "github.com/ethereum/go-ethereum/core/types" "github.com/hibiken/asynq" "github.com/sirupsen/logrus" "github.com/ethpandaops/execution-processor/pkg/common" + "github.com/ethpandaops/execution-processor/pkg/ethereum/execution" "github.com/ethpandaops/execution-processor/pkg/processor/tracker" "github.com/ethpandaops/execution-processor/pkg/state" ) @@ -181,7 +181,7 @@ func (p *Processor) ProcessNextBlock(ctx context.Context) error { // enqueueTransactionTasks enqueues tasks for all transactions in a block. // EnqueueTransactionTasks enqueues transaction processing tasks for a given block. -func (p *Processor) EnqueueTransactionTasks(ctx context.Context, block *types.Block) (int, error) { +func (p *Processor) EnqueueTransactionTasks(ctx context.Context, block execution.Block) (int, error) { var enqueuedCount int var errs []error diff --git a/pkg/processor/transaction/structlog/call_tracker.go b/pkg/processor/transaction/structlog/call_tracker.go new file mode 100644 index 0000000..805eb51 --- /dev/null +++ b/pkg/processor/transaction/structlog/call_tracker.go @@ -0,0 +1,81 @@ +package structlog + +// CallFrame represents a single call frame in the EVM execution. +type CallFrame struct { + ID uint32 // Sequential frame ID within the transaction + Depth uint64 // EVM depth level +} + +// CallTracker tracks call frames during EVM opcode traversal. +// It assigns sequential frame IDs as calls are entered and maintains +// the current path from root to the active frame. +type CallTracker struct { + stack []CallFrame // Stack of active call frames + nextID uint32 // Next frame ID to assign + path []uint32 // Current path from root to active frame +} + +// NewCallTracker creates a new CallTracker initialized with the root frame. +// The root frame has ID 0 and Depth 1, matching EVM structlog traces where +// execution starts at depth 1 (not 0). +func NewCallTracker() *CallTracker { + return &CallTracker{ + stack: []CallFrame{{ID: 0, Depth: 1}}, + nextID: 1, + path: []uint32{0}, + } +} + +// ProcessDepthChange processes a depth change and returns the current frame ID and path. +// Call this for each opcode with the opcode's depth value. +func (ct *CallTracker) ProcessDepthChange(newDepth uint64) (frameID uint32, framePath []uint32) { + currentDepth := ct.stack[len(ct.stack)-1].Depth + + if newDepth > currentDepth { + // Entering new call frame + newFrame := CallFrame{ID: ct.nextID, Depth: newDepth} + ct.stack = append(ct.stack, newFrame) + ct.path = append(ct.path, ct.nextID) + ct.nextID++ + } else if newDepth < currentDepth { + // Returning from call(s) - pop frames until depth matches + for len(ct.stack) > 1 && ct.stack[len(ct.stack)-1].Depth > newDepth { + ct.stack = ct.stack[:len(ct.stack)-1] + ct.path = ct.path[:len(ct.path)-1] + } + } + + // Return current frame info (copy path to avoid mutation issues) + pathCopy := make([]uint32, len(ct.path)) + copy(pathCopy, ct.path) + + return ct.stack[len(ct.stack)-1].ID, pathCopy +} + +// IssueFrameID allocates the next frame ID without processing a depth change. +// Used for synthetic frames (e.g., EOA calls that don't increase depth). +// Returns the new frame ID and the path for the synthetic child frame. +func (ct *CallTracker) IssueFrameID() (frameID uint32, framePath []uint32) { + newID := ct.nextID + ct.nextID++ + + // Path for synthetic frame is current path + new ID + pathCopy := make([]uint32, len(ct.path)+1) + copy(pathCopy, ct.path) + pathCopy[len(ct.path)] = newID + + return newID, pathCopy +} + +// CurrentFrameID returns the current frame ID without processing a depth change. +func (ct *CallTracker) CurrentFrameID() uint32 { + return ct.stack[len(ct.stack)-1].ID +} + +// CurrentPath returns a copy of the current path. +func (ct *CallTracker) CurrentPath() []uint32 { + pathCopy := make([]uint32, len(ct.path)) + copy(pathCopy, ct.path) + + return pathCopy +} diff --git a/pkg/processor/transaction/structlog/call_tracker_test.go b/pkg/processor/transaction/structlog/call_tracker_test.go new file mode 100644 index 0000000..5c30408 --- /dev/null +++ b/pkg/processor/transaction/structlog/call_tracker_test.go @@ -0,0 +1,662 @@ +package structlog + +import ( + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestNewCallTracker(t *testing.T) { + ct := NewCallTracker() + + assert.Equal(t, uint32(0), ct.CurrentFrameID()) + assert.Equal(t, []uint32{0}, ct.CurrentPath()) +} + +func TestCallTracker_SameDepth(t *testing.T) { + ct := NewCallTracker() + + // All opcodes at depth 1 should stay in frame 0 (root) + frameID, path := ct.ProcessDepthChange(1) + assert.Equal(t, uint32(0), frameID) + assert.Equal(t, []uint32{0}, path) + + frameID, path = ct.ProcessDepthChange(1) + assert.Equal(t, uint32(0), frameID) + assert.Equal(t, []uint32{0}, path) + + frameID, path = ct.ProcessDepthChange(1) + assert.Equal(t, uint32(0), frameID) + assert.Equal(t, []uint32{0}, path) +} + +func TestCallTracker_SingleCall(t *testing.T) { + ct := NewCallTracker() + + // depth=1: root frame (EVM traces start at depth 1) + frameID, path := ct.ProcessDepthChange(1) + assert.Equal(t, uint32(0), frameID) + assert.Equal(t, []uint32{0}, path) + + // depth=2: entering first call + frameID, path = ct.ProcessDepthChange(2) + assert.Equal(t, uint32(1), frameID) + assert.Equal(t, []uint32{0, 1}, path) + + // depth=2: still in first call + frameID, path = ct.ProcessDepthChange(2) + assert.Equal(t, uint32(1), frameID) + assert.Equal(t, []uint32{0, 1}, path) + + // depth=1: returned from call + frameID, path = ct.ProcessDepthChange(1) + assert.Equal(t, uint32(0), frameID) + assert.Equal(t, []uint32{0}, path) +} + +func TestCallTracker_NestedCalls(t *testing.T) { + ct := NewCallTracker() + + // depth=1: root (EVM traces start at depth 1) + frameID, path := ct.ProcessDepthChange(1) + assert.Equal(t, uint32(0), frameID) + assert.Equal(t, []uint32{0}, path) + + // depth=2: first call + frameID, path = ct.ProcessDepthChange(2) + assert.Equal(t, uint32(1), frameID) + assert.Equal(t, []uint32{0, 1}, path) + + // depth=3: nested call + frameID, path = ct.ProcessDepthChange(3) + assert.Equal(t, uint32(2), frameID) + assert.Equal(t, []uint32{0, 1, 2}, path) + + // depth=4: deeper nested call + frameID, path = ct.ProcessDepthChange(4) + assert.Equal(t, uint32(3), frameID) + assert.Equal(t, []uint32{0, 1, 2, 3}, path) + + // depth=3: return from depth 4 + frameID, path = ct.ProcessDepthChange(3) + assert.Equal(t, uint32(2), frameID) + assert.Equal(t, []uint32{0, 1, 2}, path) + + // depth=2: return from depth 3 + frameID, path = ct.ProcessDepthChange(2) + assert.Equal(t, uint32(1), frameID) + assert.Equal(t, []uint32{0, 1}, path) + + // depth=1: return to root + frameID, path = ct.ProcessDepthChange(1) + assert.Equal(t, uint32(0), frameID) + assert.Equal(t, []uint32{0}, path) +} + +func TestCallTracker_SiblingCalls(t *testing.T) { + // Tests the scenario from the plan: + // root -> CALL (0x123) -> CALL (0x456) -> CALL (0x789) + // root -> CALL (0xabc) -> CALL (0x456) -> CALL (0x789) + ct := NewCallTracker() + + // depth=1: root (EVM traces start at depth 1) + frameID, path := ct.ProcessDepthChange(1) + assert.Equal(t, uint32(0), frameID) + assert.Equal(t, []uint32{0}, path) + + // First branch: depth=2 (call to 0x123) + frameID, path = ct.ProcessDepthChange(2) + assert.Equal(t, uint32(1), frameID) + assert.Equal(t, []uint32{0, 1}, path) + + // depth=3 (call to 0x456) + frameID, path = ct.ProcessDepthChange(3) + assert.Equal(t, uint32(2), frameID) + assert.Equal(t, []uint32{0, 1, 2}, path) + + // depth=4 (call to 0x789) + frameID, path = ct.ProcessDepthChange(4) + assert.Equal(t, uint32(3), frameID) + assert.Equal(t, []uint32{0, 1, 2, 3}, path) + + // Return all the way to root + frameID, path = ct.ProcessDepthChange(1) + assert.Equal(t, uint32(0), frameID) + assert.Equal(t, []uint32{0}, path) + + // Second branch: depth=2 (call to 0xabc) - NEW frame_id! + frameID, path = ct.ProcessDepthChange(2) + assert.Equal(t, uint32(4), frameID, "sibling call should get new frame_id") + assert.Equal(t, []uint32{0, 4}, path) + + // depth=3 (call to 0x456 again) - NEW frame_id! + frameID, path = ct.ProcessDepthChange(3) + assert.Equal(t, uint32(5), frameID, "same contract different call should get new frame_id") + assert.Equal(t, []uint32{0, 4, 5}, path) + + // depth=4 (call to 0x789 again) - NEW frame_id! + frameID, path = ct.ProcessDepthChange(4) + assert.Equal(t, uint32(6), frameID, "same contract different call should get new frame_id") + assert.Equal(t, []uint32{0, 4, 5, 6}, path) +} + +func TestCallTracker_MultipleReturns(t *testing.T) { + // Test returning multiple levels at once (e.g., REVERT that unwinds multiple frames) + ct := NewCallTracker() + + // Build up: depth 1 -> 2 -> 3 -> 4 (EVM traces start at depth 1) + ct.ProcessDepthChange(1) + ct.ProcessDepthChange(2) + ct.ProcessDepthChange(3) + frameID, path := ct.ProcessDepthChange(4) + assert.Equal(t, uint32(3), frameID) + assert.Equal(t, []uint32{0, 1, 2, 3}, path) + + // Jump directly from depth 4 to depth 2 (skipping depth 3) + frameID, path = ct.ProcessDepthChange(2) + assert.Equal(t, uint32(1), frameID) + assert.Equal(t, []uint32{0, 1}, path) +} + +func TestCallTracker_PathIsCopy(t *testing.T) { + ct := NewCallTracker() + + ct.ProcessDepthChange(1) + _, path1 := ct.ProcessDepthChange(2) + + // Modify path1, should not affect tracker's internal state + path1[0] = 999 + + _, path2 := ct.ProcessDepthChange(2) + require.Len(t, path2, 2) + assert.Equal(t, uint32(0), path2[0], "modifying returned path should not affect tracker") +} + +func TestCallTracker_DepthStartsAtOne(t *testing.T) { + // EVM traces always start at depth 1, which is the root frame (ID 0) + ct := NewCallTracker() + + // First opcode at depth 1 - should be frame 0 (root) + frameID, path := ct.ProcessDepthChange(1) + assert.Equal(t, uint32(0), frameID) + assert.Equal(t, []uint32{0}, path) + + // Stay at depth 1 + frameID, path = ct.ProcessDepthChange(1) + assert.Equal(t, uint32(0), frameID) + assert.Equal(t, []uint32{0}, path) + + // Go deeper - creates frame 1 + frameID, path = ct.ProcessDepthChange(2) + assert.Equal(t, uint32(1), frameID) + assert.Equal(t, []uint32{0, 1}, path) +} + +func TestCallTracker_IssueFrameID(t *testing.T) { + // Tests IssueFrameID for synthetic EOA frames + ct := NewCallTracker() + + // depth=1: root (EVM traces start at depth 1) + frameID, path := ct.ProcessDepthChange(1) + assert.Equal(t, uint32(0), frameID) + assert.Equal(t, []uint32{0}, path) + + // Issue a synthetic frame for EOA call (no depth increase) + eoaFrameID, eoaPath := ct.IssueFrameID() + assert.Equal(t, uint32(1), eoaFrameID, "EOA frame should get next sequential ID") + assert.Equal(t, []uint32{0, 1}, eoaPath, "EOA path should be parent path + EOA ID") + + // Regular depth increase should get the next ID after the EOA frame + frameID, path = ct.ProcessDepthChange(2) + assert.Equal(t, uint32(2), frameID, "next real frame should get ID after EOA frame") + assert.Equal(t, []uint32{0, 2}, path) + + // Issue another EOA frame from depth 2 + eoaFrameID2, eoaPath2 := ct.IssueFrameID() + assert.Equal(t, uint32(3), eoaFrameID2) + assert.Equal(t, []uint32{0, 2, 3}, eoaPath2) + + // Return to depth 1 + frameID, path = ct.ProcessDepthChange(1) + assert.Equal(t, uint32(0), frameID) + assert.Equal(t, []uint32{0}, path) + + // Issue EOA from root - should continue sequential numbering + eoaFrameID3, eoaPath3 := ct.IssueFrameID() + assert.Equal(t, uint32(4), eoaFrameID3) + assert.Equal(t, []uint32{0, 4}, eoaPath3) +} + +func TestCallTracker_IssueFrameID_MultipleConsecutive(t *testing.T) { + // Test multiple consecutive EOA calls (e.g., contract sends to multiple EOAs) + ct := NewCallTracker() + + ct.ProcessDepthChange(1) // root + + // Three consecutive EOA calls + id1, path1 := ct.IssueFrameID() + id2, path2 := ct.IssueFrameID() + id3, path3 := ct.IssueFrameID() + + assert.Equal(t, uint32(1), id1) + assert.Equal(t, uint32(2), id2) + assert.Equal(t, uint32(3), id3) + + assert.Equal(t, []uint32{0, 1}, path1) + assert.Equal(t, []uint32{0, 2}, path2) + assert.Equal(t, []uint32{0, 3}, path3) + + // Next real call should continue from 4 + frameID, path := ct.ProcessDepthChange(2) + assert.Equal(t, uint32(4), frameID) + assert.Equal(t, []uint32{0, 4}, path) +} + +func TestIsPrecompile(t *testing.T) { + tests := []struct { + name string + addr string + expected bool + }{ + // Known precompiles (should return true) + {"ecrecover", "0x0000000000000000000000000000000000000001", true}, + {"sha256", "0x0000000000000000000000000000000000000002", true}, + {"ripemd160", "0x0000000000000000000000000000000000000003", true}, + {"identity", "0x0000000000000000000000000000000000000004", true}, + {"modexp", "0x0000000000000000000000000000000000000005", true}, + {"bn256Add", "0x0000000000000000000000000000000000000006", true}, + {"bn256ScalarMul", "0x0000000000000000000000000000000000000007", true}, + {"bn256Pairing", "0x0000000000000000000000000000000000000008", true}, + {"blake2f", "0x0000000000000000000000000000000000000009", true}, + {"kzgPointEvaluation", "0x000000000000000000000000000000000000000a", true}, + {"bls12381G1Add", "0x000000000000000000000000000000000000000b", true}, + {"bls12381G1Msm", "0x000000000000000000000000000000000000000c", true}, + {"bls12381G2Add", "0x000000000000000000000000000000000000000d", true}, + {"bls12381G2Msm", "0x000000000000000000000000000000000000000e", true}, + {"bls12381PairingCheck", "0x000000000000000000000000000000000000000f", true}, + {"bls12381MapFpToG1", "0x0000000000000000000000000000000000000010", true}, + {"bls12381MapFp2ToG2", "0x0000000000000000000000000000000000000011", true}, + {"p256Verify", "0x0000000000000000000000000000000000000100", true}, + + // Low addresses that are NOT precompiles (should return false) + // These are real EOAs/contracts deployed early in Ethereum's history + {"zero address", "0x0000000000000000000000000000000000000000", false}, + {"address 0x5c", "0x000000000000000000000000000000000000005c", false}, + {"address 0x60", "0x0000000000000000000000000000000000000060", false}, + {"address 0x44", "0x0000000000000000000000000000000000000044", false}, + {"address 0x348", "0x0000000000000000000000000000000000000348", false}, + {"address 0xffff", "0x000000000000000000000000000000000000ffff", false}, + {"address 0x12", "0x0000000000000000000000000000000000000012", false}, // Just above 0x11 + + // Real contract/EOA addresses (should return false) + {"WETH", "0xc02aaa39b223fe8d0a0e5c4f27ead9083c756cc2", false}, + {"Uniswap V2 Router", "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", false}, + {"vitalik.eth", "0xd8da6bf26964af9d7eed9e03e53415d37aa96045", false}, + {"random EOA", "0x1234567890abcdef1234567890abcdef12345678", false}, + + // Case insensitivity for precompiles + {"uppercase hex 0xA", "0x000000000000000000000000000000000000000A", true}, + {"mixed case 0xB", "0x000000000000000000000000000000000000000B", true}, + {"uppercase contract", "0xC02AAA39B223FE8D0A0E5C4F27EAD9083C756CC2", false}, + + // Without 0x prefix (edge case) + {"no prefix precompile", "0000000000000000000000000000000000000001", true}, + {"no prefix contract", "c02aaa39b223fe8d0a0e5c4f27ead9083c756cc2", false}, + + // Short addresses (should be padded) + {"short precompile 0x1", "0x1", true}, + {"short precompile 0x9", "0x9", true}, + {"short precompile 0x100", "0x100", true}, + {"short non-precompile 0x5c", "0x5c", false}, + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + result := isPrecompile(tc.addr) + assert.Equal(t, tc.expected, result, "isPrecompile(%q) = %v, want %v", tc.addr, result, tc.expected) + }) + } +} + +// TestIsPrecompile_HardcodedList verifies that our hardcoded precompile list +// matches the expected Osaka precompiles from go-ethereum. +// Source: https://github.com/ethereum/go-ethereum/blob/master/core/vm/contracts.go +func TestIsPrecompile_HardcodedList(t *testing.T) { + // All expected Osaka precompiles (superset of all forks) + expectedPrecompiles := []string{ + "0x0000000000000000000000000000000000000001", // ecrecover + "0x0000000000000000000000000000000000000002", // sha256 + "0x0000000000000000000000000000000000000003", // ripemd160 + "0x0000000000000000000000000000000000000004", // identity + "0x0000000000000000000000000000000000000005", // modexp + "0x0000000000000000000000000000000000000006", // bn256Add + "0x0000000000000000000000000000000000000007", // bn256ScalarMul + "0x0000000000000000000000000000000000000008", // bn256Pairing + "0x0000000000000000000000000000000000000009", // blake2f + "0x000000000000000000000000000000000000000a", // kzgPointEvaluation + "0x000000000000000000000000000000000000000b", // bls12381G1Add + "0x000000000000000000000000000000000000000c", // bls12381G1MultiExp + "0x000000000000000000000000000000000000000d", // bls12381G2Add + "0x000000000000000000000000000000000000000e", // bls12381G2MultiExp + "0x000000000000000000000000000000000000000f", // bls12381Pairing + "0x0000000000000000000000000000000000000010", // bls12381MapG1 + "0x0000000000000000000000000000000000000011", // bls12381MapG2 + "0x0000000000000000000000000000000000000100", // p256Verify + } + + // Verify all expected precompiles are detected + for _, addr := range expectedPrecompiles { + assert.True(t, isPrecompile(addr), + "precompile %s should be detected", addr) + } + + // Verify the expected count: 0x01-0x11 (17) + 0x100 (1) = 18 precompiles + assert.Equal(t, 18, len(expectedPrecompiles), + "expected 18 precompiles in Osaka fork") + assert.Equal(t, 18, len(precompileAddresses), + "hardcoded precompileAddresses should have 18 entries") +} + +func TestIsCallOpcode(t *testing.T) { + tests := []struct { + opcode string + expected bool + }{ + // CALL-type opcodes (should return true) + {"CALL", true}, + {"CALLCODE", true}, + {"DELEGATECALL", true}, + {"STATICCALL", true}, + + // CREATE opcodes (should return false - they always increase depth) + {"CREATE", false}, + {"CREATE2", false}, + + // Other opcodes (should return false) + {"ADD", false}, + {"SUB", false}, + {"SLOAD", false}, + {"SSTORE", false}, + {"PUSH1", false}, + {"POP", false}, + {"JUMP", false}, + {"RETURN", false}, + {"REVERT", false}, + {"STOP", false}, + {"", false}, + } + + for _, tc := range tests { + t.Run(tc.opcode, func(t *testing.T) { + result := isCallOpcode(tc.opcode) + assert.Equal(t, tc.expected, result, "isCallOpcode(%q) = %v, want %v", tc.opcode, result, tc.expected) + }) + } +} + +// TestEOADetectionLogic tests the EOA call detection scenarios. +// This validates the fix for the bug where synthetic frames were incorrectly +// created for failed calls (depth decrease) instead of only for EOA calls (depth same). +func TestEOADetectionLogic(t *testing.T) { + // Helper to simulate the EOA detection logic from transaction_processing.go + shouldCreateSyntheticFrame := func(currentDepth, nextDepth uint64, hasNextOpcode bool, callToAddr string) bool { + if !hasNextOpcode { + // Last opcode is a CALL - we can't determine if it's EOA + // because we don't have a next opcode to compare depth with. + return false + } + + // Only create synthetic frame if depth stayed the same (EOA call) + // Depth increase = entered contract code (not EOA) + // Depth decrease = call returned/failed (not EOA) + // Depth same = called EOA or precompile (immediate return) + if nextDepth == currentDepth && !isPrecompile(callToAddr) { + return true + } + + return false + } + + tests := []struct { + name string + currentDepth uint64 + nextDepth uint64 + hasNextOp bool + callToAddr string + expectSynth bool + description string + }{ + // EOA call scenarios (should create synthetic frame) + { + name: "EOA call - depth stays same", + currentDepth: 2, + nextDepth: 2, + hasNextOp: true, + callToAddr: "0xd8da6bf26964af9d7eed9e03e53415d37aa96045", // vitalik.eth + expectSynth: true, + description: "CALL to EOA returns immediately, depth stays same", + }, + { + name: "EOA call from root depth", + currentDepth: 1, + nextDepth: 1, + hasNextOp: true, + callToAddr: "0x1234567890abcdef1234567890abcdef12345678", + expectSynth: true, + description: "CALL to EOA from root frame", + }, + + // Precompile call scenarios (should NOT create synthetic frame) + { + name: "precompile call - ecrecover", + currentDepth: 2, + nextDepth: 2, + hasNextOp: true, + callToAddr: "0x0000000000000000000000000000000000000001", + expectSynth: false, + description: "CALL to ecrecover precompile", + }, + { + name: "precompile call - sha256", + currentDepth: 2, + nextDepth: 2, + hasNextOp: true, + callToAddr: "0x0000000000000000000000000000000000000002", + expectSynth: false, + description: "CALL to sha256 precompile", + }, + { + name: "precompile call - kzg point eval", + currentDepth: 3, + nextDepth: 3, + hasNextOp: true, + callToAddr: "0x000000000000000000000000000000000000000a", + expectSynth: false, + description: "STATICCALL to KZG point evaluation precompile", + }, + + // Contract call scenarios (should NOT create synthetic frame) + { + name: "contract call - depth increases", + currentDepth: 2, + nextDepth: 3, + hasNextOp: true, + callToAddr: "0xc02aaa39b223fe8d0a0e5c4f27ead9083c756cc2", // WETH + expectSynth: false, + description: "CALL to contract enters code, depth increases", + }, + { + name: "delegatecall - depth increases", + currentDepth: 1, + nextDepth: 2, + hasNextOp: true, + callToAddr: "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", // Uniswap Router + expectSynth: false, + description: "DELEGATECALL enters implementation code", + }, + { + name: "nested contract call", + currentDepth: 3, + nextDepth: 4, + hasNextOp: true, + callToAddr: "0xabcdef1234567890abcdef1234567890abcdef12", + expectSynth: false, + description: "Nested CALL enters deeper contract", + }, + + // Failed/returning call scenarios (should NOT create synthetic frame) + // This is the bug we fixed - depth DECREASE was incorrectly treated as EOA + { + name: "failed call - depth decreases by 1", + currentDepth: 3, + nextDepth: 2, + hasNextOp: true, + callToAddr: "0xde9c774cde34f85ee69c22e9a1077a0c9091f09b", + expectSynth: false, + description: "CALL failed/reverted, returned to caller depth", + }, + { + name: "failed call - depth decreases by 2", + currentDepth: 4, + nextDepth: 2, + hasNextOp: true, + callToAddr: "0xabcdef1234567890abcdef1234567890abcdef12", + expectSynth: false, + description: "CALL caused revert unwinding multiple frames", + }, + { + name: "out of gas - depth returns to root", + currentDepth: 3, + nextDepth: 1, + hasNextOp: true, + callToAddr: "0xfe02a32cbe0cb9ad9a945576a5bb53a3c123a3a3", + expectSynth: false, + description: "Out of gas unwinds all the way to root", + }, + { + name: "call returns normally", + currentDepth: 2, + nextDepth: 1, + hasNextOp: true, + callToAddr: "0xc02aaa39b223fe8d0a0e5c4f27ead9083c756cc2", + expectSynth: false, + description: "Contract call completed and returned", + }, + + // Last opcode scenarios (should NOT create synthetic frame) + { + name: "last opcode is CALL - no next opcode", + currentDepth: 2, + nextDepth: 0, // doesn't matter + hasNextOp: false, + callToAddr: "0xd8da6bf26964af9d7eed9e03e53415d37aa96045", + expectSynth: false, + description: "Transaction ends with CALL (likely failed)", + }, + { + name: "last opcode CALL to contract", + currentDepth: 1, + nextDepth: 0, + hasNextOp: false, + callToAddr: "0xc02aaa39b223fe8d0a0e5c4f27ead9083c756cc2", + expectSynth: false, + description: "Can't determine if EOA without next opcode", + }, + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + result := shouldCreateSyntheticFrame(tc.currentDepth, tc.nextDepth, tc.hasNextOp, tc.callToAddr) + assert.Equal(t, tc.expectSynth, result, + "%s: shouldCreateSyntheticFrame(depth=%d, nextDepth=%d, hasNext=%v, addr=%s) = %v, want %v", + tc.description, tc.currentDepth, tc.nextDepth, tc.hasNextOp, tc.callToAddr, result, tc.expectSynth) + }) + } +} + +// TestEOADetectionBugScenario_DepthDecrease verifies the fix for the bug where +// a CALL followed by depth decrease was incorrectly treated as an EOA call. +// Real-world example: transaction 0x4f7494... had a CALL at depth 3, next opcode +// was at depth 2 (returned/failed). The old <= check created a phantom synthetic frame. +func TestEOADetectionBugScenario_DepthDecrease(t *testing.T) { + // Simulate the buggy scenario from tx 0x4f7494c9f3b1bb7fb9f4d928aae41d971f0799a3d5c24df209074b70f04211f5 + // Index 235: GAS at depth 3 + // Index 236: CALL at depth 3 (to 0xde9c774cde34f85ee69c22e9a1077a0c9091f09b) + // Index 237: RETURNDATASIZE at depth 2 (call returned/failed) + currentDepth := uint64(3) + nextDepth := uint64(2) // Depth DECREASED (call returned/failed) + callToAddr := "0xde9c774cde34f85ee69c22e9a1077a0c9091f09b" + + // Old buggy logic: nextDepth <= currentDepth → 2 <= 3 → TRUE (wrong!) + buggyLogic := nextDepth <= currentDepth && !isPrecompile(callToAddr) + assert.True(t, buggyLogic, "Old buggy logic would have created synthetic frame") + + // Fixed logic: nextDepth == currentDepth → 2 == 3 → FALSE (correct!) + fixedLogic := nextDepth == currentDepth && !isPrecompile(callToAddr) + assert.False(t, fixedLogic, "Fixed logic should NOT create synthetic frame") +} + +// TestEOADetectionBugScenario_OutOfGas verifies the fix for the bug where +// a CALL as the last opcode (out of gas) was incorrectly treated as an EOA call. +// Real-world example: transaction 0x7178d8e3... ended with a CALL that ran out of gas. +func TestEOADetectionBugScenario_OutOfGas(t *testing.T) { + // Simulate the buggy scenario from tx 0x7178d8e3a33331ee0b2c42372c357cb6135bf3acd6e1eea5dbca7d9dbedfa418 + // Index 10: GAS at depth 1 + // Index 11: CALL at depth 1 (last opcode - out of gas before entering target) + // No index 12 (trace ended) + callToAddr := "0xfe02a32cbe0cb9ad9a945576a5bb53a3c123a3a3" + hasNextOpcode := false + + // Old buggy logic: "Last opcode is a CALL - if not precompile, must be EOA" + buggyLogic := !hasNextOpcode && !isPrecompile(callToAddr) + assert.True(t, buggyLogic, "Old buggy logic would have created synthetic frame") + + // Fixed logic: Don't assume last CALL is EOA - we can't determine without next opcode + fixedLogic := hasNextOpcode && !isPrecompile(callToAddr) // Always false when !hasNextOpcode + assert.False(t, fixedLogic, "Fixed logic should NOT create synthetic frame for last opcode") +} + +func TestCallTracker_RealWorldExample(t *testing.T) { + // Simulate a real EVM trace where depth starts at 1: + // op=PUSH1, depth=1 → frame_id=0, path=[0] (root execution) + // op=CALL(A),depth=1 → frame_id=0, path=[0] + // op=ADD, depth=2 → frame_id=1, path=[0,1] (inside A) + // op=CALL(B),d=2 → frame_id=1, path=[0,1] + // op=MUL, d=3 → frame_id=2, path=[0,1,2] (inside B) + // op=CALL(C),d=3 → frame_id=2, path=[0,1,2] + // op=SLOAD,d=4 → frame_id=3, path=[0,1,2,3] (inside C) + // op=RETURN,d=4 → frame_id=3, path=[0,1,2,3] + // op=ADD, d=3 → frame_id=2, path=[0,1,2] (back in B) + // op=RETURN,d=3 → frame_id=2, path=[0,1,2] + // op=POP, depth=2 → frame_id=1, path=[0,1] (back in A) + // op=STOP, depth=1 → frame_id=0, path=[0] (back in root) + ct := NewCallTracker() + + type expected struct { + depth uint64 + frameID uint32 + path []uint32 + } + + testCases := []expected{ + {1, 0, []uint32{0}}, // PUSH1 (root) + {1, 0, []uint32{0}}, // CALL(A) + {2, 1, []uint32{0, 1}}, // ADD (inside A) + {2, 1, []uint32{0, 1}}, // CALL(B) + {3, 2, []uint32{0, 1, 2}}, // MUL (inside B) + {3, 2, []uint32{0, 1, 2}}, // CALL(C) + {4, 3, []uint32{0, 1, 2, 3}}, // SLOAD (inside C) + {4, 3, []uint32{0, 1, 2, 3}}, // RETURN (inside C) + {3, 2, []uint32{0, 1, 2}}, // ADD (back in B) + {3, 2, []uint32{0, 1, 2}}, // RETURN (inside B) + {2, 1, []uint32{0, 1}}, // POP (back in A) + {1, 0, []uint32{0}}, // STOP (back in root) + } + + for i, tc := range testCases { + frameID, path := ct.ProcessDepthChange(tc.depth) + assert.Equal(t, tc.frameID, frameID, "case %d: frame_id mismatch", i) + assert.Equal(t, tc.path, path, "case %d: path mismatch", i) + } +} diff --git a/pkg/processor/transaction/structlog/columns.go b/pkg/processor/transaction/structlog/columns.go index a08f2af..07ab948 100644 --- a/pkg/processor/transaction/structlog/columns.go +++ b/pkg/processor/transaction/structlog/columns.go @@ -6,6 +6,19 @@ import ( "github.com/ClickHouse/ch-go/proto" ) +// ClickHouseTime wraps time.Time for ClickHouse DateTime formatting. +type ClickHouseTime time.Time + +// NewClickHouseTime creates a new ClickHouseTime from time.Time. +func NewClickHouseTime(t time.Time) ClickHouseTime { + return ClickHouseTime(t) +} + +// Time returns the underlying time.Time. +func (t ClickHouseTime) Time() time.Time { + return time.Time(t) +} + // Columns holds all columns for structlog batch insert using ch-go columnar protocol. type Columns struct { UpdatedDateTime proto.ColDateTime @@ -21,11 +34,14 @@ type Columns struct { Gas proto.ColUInt64 GasCost proto.ColUInt64 GasUsed proto.ColUInt64 + GasSelf proto.ColUInt64 Depth proto.ColUInt64 ReturnData *proto.ColNullable[string] Refund *proto.ColNullable[uint64] Error *proto.ColNullable[string] CallToAddress *proto.ColNullable[string] + CallFrameID proto.ColUInt32 + CallFramePath *proto.ColArr[uint32] MetaNetworkName proto.ColStr } @@ -37,6 +53,7 @@ func NewColumns() *Columns { Refund: new(proto.ColUInt64).Nullable(), Error: new(proto.ColStr).Nullable(), CallToAddress: new(proto.ColStr).Nullable(), + CallFramePath: new(proto.ColUInt32).Array(), } } @@ -55,11 +72,14 @@ func (c *Columns) Append( gas uint64, gasCost uint64, gasUsed uint64, + gasSelf uint64, depth uint64, returnData *string, refund *uint64, errStr *string, callTo *string, + callFrameID uint32, + callFramePath []uint32, network string, ) { c.UpdatedDateTime.Append(updatedDateTime) @@ -75,11 +95,14 @@ func (c *Columns) Append( c.Gas.Append(gas) c.GasCost.Append(gasCost) c.GasUsed.Append(gasUsed) + c.GasSelf.Append(gasSelf) c.Depth.Append(depth) c.ReturnData.Append(nullableStr(returnData)) c.Refund.Append(nullableUint64(refund)) c.Error.Append(nullableStr(errStr)) c.CallToAddress.Append(nullableStr(callTo)) + c.CallFrameID.Append(callFrameID) + c.CallFramePath.Append(callFramePath) c.MetaNetworkName.Append(network) } @@ -98,11 +121,14 @@ func (c *Columns) Reset() { c.Gas.Reset() c.GasCost.Reset() c.GasUsed.Reset() + c.GasSelf.Reset() c.Depth.Reset() c.ReturnData.Reset() c.Refund.Reset() c.Error.Reset() c.CallToAddress.Reset() + c.CallFrameID.Reset() + c.CallFramePath.Reset() c.MetaNetworkName.Reset() } @@ -122,11 +148,14 @@ func (c *Columns) Input() proto.Input { {Name: "gas", Data: &c.Gas}, {Name: "gas_cost", Data: &c.GasCost}, {Name: "gas_used", Data: &c.GasUsed}, + {Name: "gas_self", Data: &c.GasSelf}, {Name: "depth", Data: &c.Depth}, {Name: "return_data", Data: c.ReturnData}, {Name: "refund", Data: c.Refund}, {Name: "error", Data: c.Error}, {Name: "call_to_address", Data: c.CallToAddress}, + {Name: "call_frame_id", Data: &c.CallFrameID}, + {Name: "call_frame_path", Data: c.CallFramePath}, {Name: "meta_network_name", Data: &c.MetaNetworkName}, } } diff --git a/pkg/processor/transaction/structlog/create_address_test.go b/pkg/processor/transaction/structlog/create_address_test.go new file mode 100644 index 0000000..b77f466 --- /dev/null +++ b/pkg/processor/transaction/structlog/create_address_test.go @@ -0,0 +1,262 @@ +package structlog + +import ( + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/ethpandaops/execution-processor/pkg/ethereum/execution" +) + +const testCreateAddress = "0x1234567890abcdef1234567890abcdef12345678" + +func TestComputeCreateAddresses_Empty(t *testing.T) { + result := ComputeCreateAddresses([]execution.StructLog{}) + assert.Empty(t, result) +} + +func TestComputeCreateAddresses_NoCREATE(t *testing.T) { + structlogs := []execution.StructLog{ + {Op: "PUSH1", Depth: 1}, + {Op: "CALL", Depth: 1}, + {Op: "ADD", Depth: 2}, + {Op: "RETURN", Depth: 2}, + {Op: "STOP", Depth: 1}, + } + + result := ComputeCreateAddresses(structlogs) + assert.Empty(t, result) +} + +func TestComputeCreateAddresses_SingleCREATE(t *testing.T) { + // Simulate: CREATE at depth 2, constructor runs at depth 3, returns + createdAddr := "0xdeadbeefdeadbeefdeadbeefdeadbeefdeadbeef" + stack := []string{createdAddr} + + structlogs := []execution.StructLog{ + {Op: "PUSH1", Depth: 2}, + {Op: "CREATE", Depth: 2}, // index 1 + {Op: "PUSH1", Depth: 3}, // constructor starts + {Op: "RETURN", Depth: 3}, // constructor ends + {Op: "SWAP1", Depth: 2, Stack: &stack}, // back in caller, stack has address + } + + result := ComputeCreateAddresses(structlogs) + + require.Contains(t, result, 1) + // Address is already 40 chars, so stays the same + assert.Equal(t, createdAddr, *result[1]) +} + +func TestComputeCreateAddresses_CREATE2(t *testing.T) { + createdAddr := "0xabcdefabcdefabcdefabcdefabcdefabcdefabcd" + stack := []string{createdAddr} + + structlogs := []execution.StructLog{ + {Op: "PUSH1", Depth: 1}, + {Op: "CREATE2", Depth: 1}, // index 1 + {Op: "ADD", Depth: 2}, // constructor + {Op: "RETURN", Depth: 2}, // constructor ends + {Op: "POP", Depth: 1, Stack: &stack}, // back in caller + } + + result := ComputeCreateAddresses(structlogs) + + require.Contains(t, result, 1) + assert.Equal(t, createdAddr, *result[1]) +} + +func TestComputeCreateAddresses_FailedCREATE(t *testing.T) { + // When CREATE fails immediately, next opcode is at same depth with 0 on stack + zeroAddr := "0x0" + stack := []string{zeroAddr} + + structlogs := []execution.StructLog{ + {Op: "PUSH1", Depth: 2}, + {Op: "CREATE", Depth: 2}, // index 1 - fails immediately + {Op: "ISZERO", Depth: 2, Stack: &stack}, // still at depth 2, stack has 0 + } + + result := ComputeCreateAddresses(structlogs) + + require.Contains(t, result, 1) + // Zero address is zero-padded to 40 hex chars + assert.Equal(t, "0x0000000000000000000000000000000000000000", *result[1]) +} + +func TestComputeCreateAddresses_NestedCREATEs(t *testing.T) { + // Outer CREATE at depth 1, inner CREATE at depth 2 + innerAddr := "0x1111111111111111111111111111111111111111" + outerAddr := "0x2222222222222222222222222222222222222222" + innerStack := []string{innerAddr} + outerStack := []string{outerAddr} + + structlogs := []execution.StructLog{ + {Op: "PUSH1", Depth: 1}, + {Op: "CREATE", Depth: 1}, // index 1 - outer CREATE + {Op: "PUSH1", Depth: 2}, // outer constructor starts + {Op: "CREATE", Depth: 2}, // index 3 - inner CREATE + {Op: "ADD", Depth: 3}, // inner constructor + {Op: "RETURN", Depth: 3}, // inner constructor ends + {Op: "POP", Depth: 2, Stack: &innerStack}, // back in outer constructor + {Op: "RETURN", Depth: 2}, // outer constructor ends + {Op: "SWAP1", Depth: 1, Stack: &outerStack}, // back in original caller + } + + result := ComputeCreateAddresses(structlogs) + + require.Contains(t, result, 1) + require.Contains(t, result, 3) + assert.Equal(t, outerAddr, *result[1]) + assert.Equal(t, innerAddr, *result[3]) +} + +func TestComputeCreateAddresses_MultipleCREATEsSameDepth(t *testing.T) { + // Two CREATEs at the same depth (sequential, not nested) + addr1 := "0x1111111111111111111111111111111111111111" + addr2 := "0x2222222222222222222222222222222222222222" + stack1 := []string{addr1} + stack2 := []string{addr2} + + structlogs := []execution.StructLog{ + {Op: "PUSH1", Depth: 1}, + {Op: "CREATE", Depth: 1}, // index 1 - first CREATE + {Op: "ADD", Depth: 2}, // first constructor + {Op: "RETURN", Depth: 2}, // first constructor ends + {Op: "POP", Depth: 1, Stack: &stack1}, // back, has first address + {Op: "PUSH1", Depth: 1}, + {Op: "CREATE", Depth: 1}, // index 6 - second CREATE + {Op: "MUL", Depth: 2}, // second constructor + {Op: "RETURN", Depth: 2}, // second constructor ends + {Op: "SWAP1", Depth: 1, Stack: &stack2}, // back, has second address + } + + result := ComputeCreateAddresses(structlogs) + + require.Contains(t, result, 1) + require.Contains(t, result, 6) + assert.Equal(t, addr1, *result[1]) + assert.Equal(t, addr2, *result[6]) +} + +func TestExtractCallAddressWithCreate_CREATE(t *testing.T) { + p := &Processor{} + createAddresses := map[int]*string{ + 0: ptrString(testCreateAddress), + } + + result := p.extractCallAddressWithCreate(&execution.StructLog{ + Op: "CREATE", + }, 0, createAddresses) + + assert.NotNil(t, result) + assert.Equal(t, testCreateAddress, *result) +} + +func TestExtractCallAddressWithCreate_CREATE2(t *testing.T) { + p := &Processor{} + addr := "0xabcdef1234567890abcdef1234567890abcdef12" + createAddresses := map[int]*string{ + 5: ptrString(addr), + } + + result := p.extractCallAddressWithCreate(&execution.StructLog{ + Op: "CREATE2", + }, 5, createAddresses) + + assert.NotNil(t, result) + assert.Equal(t, addr, *result) +} + +func TestExtractCallAddressWithCreate_CREATEWithNilMap(t *testing.T) { + p := &Processor{} + + result := p.extractCallAddressWithCreate(&execution.StructLog{ + Op: "CREATE", + }, 0, nil) + + assert.Nil(t, result) +} + +func TestExtractCallAddressWithCreate_CREATENotInMap(t *testing.T) { + p := &Processor{} + createAddresses := map[int]*string{ + 10: ptrString(testCreateAddress), + } + + result := p.extractCallAddressWithCreate(&execution.StructLog{ + Op: "CREATE", + }, 5, createAddresses) // index 5 not in map + + assert.Nil(t, result) +} + +func TestExtractCallAddressWithCreate_CALLDelegatesToExtractCallAddress(t *testing.T) { + p := &Processor{} + createAddresses := map[int]*string{ + 0: ptrString(testCreateAddress), + } + stack := []string{"0x5208", "0xdeadbeef"} + + result := p.extractCallAddressWithCreate(&execution.StructLog{ + Op: "CALL", + Stack: &stack, + }, 0, createAddresses) + + // Should use extractCallAddress, not createAddresses + assert.NotNil(t, result) + // Second from top of stack, zero-padded to 40 hex chars + assert.Equal(t, "0x0000000000000000000000000000000000005208", *result) +} + +func TestExtractCallAddressWithCreate_DELEGATECALLDelegatesToExtractCallAddress(t *testing.T) { + p := &Processor{} + createAddresses := map[int]*string{ + 0: ptrString(testCreateAddress), + } + stack := []string{"0x5208", "0xdeadbeef"} + + result := p.extractCallAddressWithCreate(&execution.StructLog{ + Op: "DELEGATECALL", + Stack: &stack, + }, 0, createAddresses) + + assert.NotNil(t, result) + // Zero-padded to 40 hex chars + assert.Equal(t, "0x0000000000000000000000000000000000005208", *result) +} + +func TestExtractCallAddressWithCreate_NonCallOpcodeReturnsNil(t *testing.T) { + p := &Processor{} + createAddresses := map[int]*string{ + 0: ptrString(testCreateAddress), + } + stack := []string{"0x5208", "0xdeadbeef"} + + testCases := []string{ + "PUSH1", + "ADD", + "SLOAD", + "SSTORE", + "RETURN", + "REVERT", + "STOP", + } + + for _, op := range testCases { + t.Run(op, func(t *testing.T) { + result := p.extractCallAddressWithCreate(&execution.StructLog{ + Op: op, + Stack: &stack, + }, 0, createAddresses) + + assert.Nil(t, result, "opcode %s should return nil", op) + }) + } +} + +// ptrString returns a pointer to the given string. +func ptrString(s string) *string { + return &s +} diff --git a/pkg/processor/transaction/structlog/extract_call_address_test.go b/pkg/processor/transaction/structlog/extract_call_address_test.go new file mode 100644 index 0000000..d72cb7f --- /dev/null +++ b/pkg/processor/transaction/structlog/extract_call_address_test.go @@ -0,0 +1,270 @@ +package structlog + +import ( + "testing" + + "github.com/stretchr/testify/assert" + + "github.com/ethpandaops/execution-processor/pkg/ethereum/execution" +) + +func TestExtractCallAddress_NilStack(t *testing.T) { + p := &Processor{} + + result := p.extractCallAddress(&execution.StructLog{ + Op: "CALL", + Stack: nil, + }) + + assert.Nil(t, result) +} + +func TestExtractCallAddress_EmptyStack(t *testing.T) { + p := &Processor{} + emptyStack := []string{} + + result := p.extractCallAddress(&execution.StructLog{ + Op: "CALL", + Stack: &emptyStack, + }) + + assert.Nil(t, result) +} + +func TestExtractCallAddress_InsufficientStack(t *testing.T) { + p := &Processor{} + stack := []string{"0x1234"} // Only 1 element, need at least 2 + + result := p.extractCallAddress(&execution.StructLog{ + Op: "CALL", + Stack: &stack, + }) + + assert.Nil(t, result) +} + +func TestExtractCallAddress_CALL(t *testing.T) { + p := &Processor{} + // CALL stack (index 0 = bottom, len-1 = top): + // [retSize, retOffset, argsSize, argsOffset, value, addr, gas] + // Address is at index len-2 (second from top) + stack := []string{ + "0x0", // retSize (bottom, index 0) + "0x0", // retOffset + "0x0", // argsSize + "0x0", // argsOffset + "0x0", // value + "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", // addr (index len-2) + "0x5208", // gas (top, index len-1) + } + + result := p.extractCallAddress(&execution.StructLog{ + Op: "CALL", + Stack: &stack, + }) + + assert.NotNil(t, result) + assert.Equal(t, "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", *result) +} + +func TestExtractCallAddress_CALL_MinimalStack(t *testing.T) { + p := &Processor{} + // Minimal stack with just 2 elements (addr at index 0, gas at index 1) + stack := []string{ + "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", // addr (index 0 = len-2) + "0x5208", // gas (index 1 = len-1) + } + + result := p.extractCallAddress(&execution.StructLog{ + Op: "CALL", + Stack: &stack, + }) + + assert.NotNil(t, result) + assert.Equal(t, "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", *result) +} + +func TestExtractCallAddress_CALL_WithExtraStackItemsBelow(t *testing.T) { + p := &Processor{} + // Stack with extra items BELOW CALL args (at the bottom) + // The CALL args are still at the top, so len-2 still gives addr + stack := []string{ + "0xdeadbeef", // extra item (bottom) + "0xcafebabe", // another extra item + "0x0", // retSize (start of CALL args) + "0x0", // retOffset + "0x0", // argsSize + "0x0", // argsOffset + "0x0", // value + "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", // addr (len-2) + "0x5208", // gas (top, len-1) + } + + result := p.extractCallAddress(&execution.StructLog{ + Op: "CALL", + Stack: &stack, + }) + + assert.NotNil(t, result) + assert.Equal(t, "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", *result) +} + +func TestExtractCallAddress_CALLCODE(t *testing.T) { + p := &Processor{} + // CALLCODE has same stack layout as CALL + stack := []string{ + "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", // addr + "0x5208", // gas + } + + result := p.extractCallAddress(&execution.StructLog{ + Op: "CALLCODE", + Stack: &stack, + }) + + assert.NotNil(t, result) + assert.Equal(t, "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", *result) +} + +func TestExtractCallAddress_DELEGATECALL(t *testing.T) { + p := &Processor{} + // DELEGATECALL stack (no value parameter, but addr still at len-2): + // [retSize, retOffset, argsSize, argsOffset, addr, gas] + stack := []string{ + "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", // addr + "0x5208", // gas + } + + result := p.extractCallAddress(&execution.StructLog{ + Op: "DELEGATECALL", + Stack: &stack, + }) + + assert.NotNil(t, result) + assert.Equal(t, "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", *result) +} + +func TestExtractCallAddress_STATICCALL(t *testing.T) { + p := &Processor{} + // STATICCALL has same stack layout as DELEGATECALL + stack := []string{ + "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", // addr + "0x5208", // gas + } + + result := p.extractCallAddress(&execution.StructLog{ + Op: "STATICCALL", + Stack: &stack, + }) + + assert.NotNil(t, result) + assert.Equal(t, "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", *result) +} + +func TestExtractCallAddress_NonCallOpcode(t *testing.T) { + p := &Processor{} + stack := []string{"0x1234", "0x5678"} + + testCases := []string{ + "PUSH1", + "ADD", + "SLOAD", + "SSTORE", + "JUMP", + "RETURN", + "REVERT", + "CREATE", // CREATE is not handled (address comes from trace) + "CREATE2", // CREATE2 is not handled (address comes from trace) + } + + for _, op := range testCases { + t.Run(op, func(t *testing.T) { + result := p.extractCallAddress(&execution.StructLog{ + Op: op, + Stack: &stack, + }) + assert.Nil(t, result, "opcode %s should not extract call address", op) + }) + } +} + +func TestExtractCallAddress_ShortAddressPadding(t *testing.T) { + p := &Processor{} + // Test that short addresses (like precompiles) get zero-padded + stack := []string{ + "0x1", // addr - precompile ecRecover, should be padded + "0x5208", // gas + } + + result := p.extractCallAddress(&execution.StructLog{ + Op: "CALL", + Stack: &stack, + }) + + assert.NotNil(t, result) + assert.Equal(t, "0x0000000000000000000000000000000000000001", *result) + assert.Len(t, *result, 42) +} + +func TestExtractCallAddress_Permit2Padding(t *testing.T) { + p := &Processor{} + // Test Permit2 address with leading zeros + stack := []string{ + "0x22d473030f116ddee9f6b43ac78ba3", // Permit2 truncated + "0x5208", // gas + } + + result := p.extractCallAddress(&execution.StructLog{ + Op: "CALL", + Stack: &stack, + }) + + assert.NotNil(t, result) + assert.Equal(t, "0x000000000022d473030f116ddee9f6b43ac78ba3", *result) + assert.Len(t, *result, 42) +} + +func TestExtractCallAddress_AllCallVariants(t *testing.T) { + // Table-driven test for all supported CALL variants + p := &Processor{} + + targetAddr := "0x7a250d5630b4cf539739df2c5dacb4c659f2488d" + + testCases := []struct { + name string + op string + stack []string // Stack with addr at len-2 and gas at len-1 + }{ + { + name: "CALL with full stack", + op: "CALL", + stack: []string{"0xretSize", "0xretOff", "0xargsSize", "0xargsOff", "0xvalue", targetAddr, "0xgas"}, + }, + { + name: "CALLCODE with full stack", + op: "CALLCODE", + stack: []string{"0xretSize", "0xretOff", "0xargsSize", "0xargsOff", "0xvalue", targetAddr, "0xgas"}, + }, + { + name: "DELEGATECALL with full stack", + op: "DELEGATECALL", + stack: []string{"0xretSize", "0xretOff", "0xargsSize", "0xargsOff", targetAddr, "0xgas"}, + }, + { + name: "STATICCALL with full stack", + op: "STATICCALL", + stack: []string{"0xretSize", "0xretOff", "0xargsSize", "0xargsOff", targetAddr, "0xgas"}, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + result := p.extractCallAddress(&execution.StructLog{ + Op: tc.op, + Stack: &tc.stack, + }) + assert.NotNil(t, result) + assert.Equal(t, targetAddr, *result) + }) + } +} diff --git a/pkg/processor/transaction/structlog/format_address_test.go b/pkg/processor/transaction/structlog/format_address_test.go new file mode 100644 index 0000000..7b26b62 --- /dev/null +++ b/pkg/processor/transaction/structlog/format_address_test.go @@ -0,0 +1,115 @@ +package structlog + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestFormatAddress(t *testing.T) { + testCases := []struct { + name string + input string + expected string + }{ + { + name: "already 40 chars with 0x prefix", + input: "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", + expected: "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", + }, + { + name: "already 40 chars without 0x prefix", + input: "7a250d5630b4cf539739df2c5dacb4c659f2488d", + expected: "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", + }, + { + name: "precompile address 0x1", + input: "0x1", + expected: "0x0000000000000000000000000000000000000001", + }, + { + name: "precompile address 0xa", + input: "0xa", + expected: "0x000000000000000000000000000000000000000a", + }, + { + name: "Permit2 with leading zeros truncated", + input: "0x22d473030f116ddee9f6b43ac78ba3", + expected: "0x000000000022d473030f116ddee9f6b43ac78ba3", + }, + { + name: "Uniswap PoolManager with leading zeros truncated", + input: "0x4444c5dc75cb358380d2e3de08a90", + expected: "0x000000000004444c5dc75cb358380d2e3de08a90", + }, + { + name: "zero address", + input: "0x0", + expected: "0x0000000000000000000000000000000000000000", + }, + { + name: "short address without 0x prefix", + input: "5208", + expected: "0x0000000000000000000000000000000000005208", + }, + { + name: "short address with 0x prefix", + input: "0x5208", + expected: "0x0000000000000000000000000000000000005208", + }, + { + name: "empty string", + input: "", + expected: "0x0000000000000000000000000000000000000000", + }, + { + name: "just 0x prefix", + input: "0x", + expected: "0x0000000000000000000000000000000000000000", + }, + // Full 32-byte stack values (66 chars) - extract lower 20 bytes + { + name: "full 32-byte stack value from XEN Batch Minter", + input: "0x661f30bf3a790c8687131ae8fc6e649df9f27275fc286db8f1a0be7e99b24bb2", + expected: "0xfc6e649df9f27275fc286db8f1a0be7e99b24bb2", + }, + { + name: "full 32-byte stack value - all zeros except address", + input: "0x0000000000000000000000007a250d5630b4cf539739df2c5dacb4c659f2488d", + expected: "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", + }, + { + name: "full 32-byte stack value without 0x prefix", + input: "661f30bf3a790c8687131ae8fc6e649df9f27275fc286db8f1a0be7e99b24bb2", + expected: "0xfc6e649df9f27275fc286db8f1a0be7e99b24bb2", + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + result := formatAddress(tc.input) + assert.Equal(t, tc.expected, result) + }) + } +} + +func TestFormatAddress_LengthConsistency(t *testing.T) { + // All formatted addresses should be exactly 42 characters (0x + 40 hex chars) + inputs := []string{ + "0x1", + "0xa", + "0xdeadbeef", + "0x7a250d5630b4cf539739df2c5dacb4c659f2488d", + "1", + "abcdef", + "", + } + + for _, input := range inputs { + t.Run(input, func(t *testing.T) { + result := formatAddress(input) + assert.Len(t, result, 42, "formatted address should always be 42 chars") + assert.Equal(t, "0x", result[:2], "formatted address should start with 0x") + }) + } +} diff --git a/pkg/processor/transaction/structlog/gas_cost.go b/pkg/processor/transaction/structlog/gas_cost.go index aa5c0fd..cf410bf 100644 --- a/pkg/processor/transaction/structlog/gas_cost.go +++ b/pkg/processor/transaction/structlog/gas_cost.go @@ -6,6 +6,66 @@ import ( "github.com/ethpandaops/execution-processor/pkg/ethereum/execution" ) +// ============================================================================= +// GAS FIELDS +// ============================================================================= +// +// The structlog contains three gas-related fields: +// +// GasCost +// Source: Directly from geth/erigon debug_traceTransaction response. +// For non-CALL opcodes: The static cost charged for the opcode. +// For CALL/CREATE opcodes: The gas stipend passed to the child frame. +// +// GasUsed +// Source: Computed as gas[i] - gas[i+1] for consecutive opcodes at same depth. +// For non-CALL opcodes: Actual gas consumed by the opcode. +// For CALL/CREATE opcodes: Includes the call overhead PLUS all child frame gas. +// Note: Summing gas_used across all opcodes double counts because CALL's +// gas_used includes child gas, and children also report their own gas_used. +// +// GasSelf +// Source: Computed as gas_used minus the sum of all child frame gas_used. +// For non-CALL opcodes: Equal to gas_used. +// For CALL/CREATE opcodes: Only the call overhead (warm/cold access, memory +// expansion, value transfer) without child frame gas. +// Summing gas_self across all opcodes gives total execution gas without +// double counting. +// +// Example for a CALL opcode: +// gas_cost = 7,351,321 (stipend passed to child) +// gas_used = 23,858 (overhead 2,600 + child consumed 21,258) +// gas_self = 2,600 (just the CALL overhead) +// +// ============================================================================= + +// Opcode constants for call and create operations. +const ( + OpcodeCALL = "CALL" + OpcodeCALLCODE = "CALLCODE" + OpcodeDELEGATECALL = "DELEGATECALL" + OpcodeSTATICCALL = "STATICCALL" + OpcodeCREATE = "CREATE" + OpcodeCREATE2 = "CREATE2" +) + +// hasPrecomputedGasUsed detects whether GasUsed values are pre-computed by the tracer. +// +// In embedded mode, the tracer computes GasUsed inline during trace capture, +// populating this field with non-zero values. In RPC mode, GasUsed is always 0 +// and must be computed post-hoc using ComputeGasUsed(). +// +// This enables backward compatibility: execution-processor works with both +// embedded mode (optimized, pre-computed) and RPC mode (legacy, post-computed). +func hasPrecomputedGasUsed(structlogs []execution.StructLog) bool { + if len(structlogs) == 0 { + return false + } + + // Check first structlog - if GasUsed > 0, tracer pre-computed values. + return structlogs[0].GasUsed > 0 +} + // ComputeGasUsed calculates the actual gas consumed for each structlog using // the difference between consecutive gas values at the same depth level. // @@ -62,3 +122,65 @@ func ComputeGasUsed(structlogs []execution.StructLog) []uint64 { return gasUsed } + +// ComputeGasSelf calculates the gas consumed by each opcode excluding child frame gas. +// For CALL/CREATE opcodes, this represents only the call overhead (warm/cold access, +// memory expansion, value transfer), not the gas consumed by child frames. +// For all other opcodes, this equals gasUsed. +// +// This is useful for gas analysis where you want to sum gas without double counting: +// sum(gasSelf) = total transaction execution gas (no double counting). +func ComputeGasSelf(structlogs []execution.StructLog, gasUsed []uint64) []uint64 { + if len(structlogs) == 0 { + return nil + } + + gasSelf := make([]uint64, len(structlogs)) + copy(gasSelf, gasUsed) + + for i := range structlogs { + op := structlogs[i].Op + if !isCallOrCreateOpcode(op) { + continue + } + + callDepth := structlogs[i].Depth + + var childGasSum uint64 + + // Sum gas_used for DIRECT children only (depth == callDepth + 1). + // We only sum direct children because their gas_used already includes + // any nested descendants. Summing all descendants would double count. + for j := i + 1; j < len(structlogs); j++ { + if structlogs[j].Depth <= callDepth { + break + } + + if structlogs[j].Depth == callDepth+1 { + childGasSum += gasUsed[j] + } + } + + // gasSelf = total gas attributed to this CALL minus child execution + // This gives us just the CALL overhead + if gasUsed[i] >= childGasSum { + gasSelf[i] = gasUsed[i] - childGasSum + } else { + // Edge case: if child gas exceeds parent (shouldn't happen in valid traces) + // fall back to 0 to avoid underflow + gasSelf[i] = 0 + } + } + + return gasSelf +} + +// isCallOrCreateOpcode returns true if the opcode spawns a new call frame. +func isCallOrCreateOpcode(op string) bool { + switch op { + case OpcodeCALL, OpcodeCALLCODE, OpcodeDELEGATECALL, OpcodeSTATICCALL, OpcodeCREATE, OpcodeCREATE2: + return true + default: + return false + } +} diff --git a/pkg/processor/transaction/structlog/gas_cost_test.go b/pkg/processor/transaction/structlog/gas_cost_test.go index 868d690..b645bb3 100644 --- a/pkg/processor/transaction/structlog/gas_cost_test.go +++ b/pkg/processor/transaction/structlog/gas_cost_test.go @@ -9,6 +9,33 @@ import ( "github.com/ethpandaops/execution-processor/pkg/ethereum/execution" ) +// ============================================================================= +// hasPrecomputedGasUsed Tests +// ============================================================================= + +func TestHasPrecomputedGasUsed_Empty(t *testing.T) { + assert.False(t, hasPrecomputedGasUsed(nil)) + assert.False(t, hasPrecomputedGasUsed([]execution.StructLog{})) +} + +func TestHasPrecomputedGasUsed_WithGasUsed(t *testing.T) { + structlogs := []execution.StructLog{ + {Op: "PUSH1", GasUsed: 3}, + } + assert.True(t, hasPrecomputedGasUsed(structlogs)) +} + +func TestHasPrecomputedGasUsed_WithoutGasUsed(t *testing.T) { + structlogs := []execution.StructLog{ + {Op: "PUSH1", GasUsed: 0}, + } + assert.False(t, hasPrecomputedGasUsed(structlogs)) +} + +// ============================================================================= +// ComputeGasUsed Tests +// ============================================================================= + func TestComputeGasUsed_EmptyLogs(t *testing.T) { result := ComputeGasUsed(nil) assert.Nil(t, result) @@ -313,3 +340,286 @@ func TestComputeGasUsed_LargeDepth(t *testing.T) { assert.Equal(t, uint64(2), result[8]) assert.Equal(t, uint64(2), result[9]) } + +// ============================================================================= +// ComputeGasSelf Tests +// ============================================================================= + +func TestComputeGasSelf_EmptyLogs(t *testing.T) { + result := ComputeGasSelf(nil, nil) + assert.Nil(t, result) + + result = ComputeGasSelf([]execution.StructLog{}, []uint64{}) + assert.Nil(t, result) +} + +func TestComputeGasSelf_NonCallOpcodes(t *testing.T) { + // For non-CALL opcodes, gas_self should equal gas_used + structlogs := []execution.StructLog{ + {Op: "PUSH1", Gas: 100000, GasCost: 3, Depth: 1}, + {Op: "SLOAD", Gas: 99997, GasCost: 2100, Depth: 1}, + {Op: "ADD", Gas: 97897, GasCost: 3, Depth: 1}, + } + + gasUsed := []uint64{3, 2100, 3} + + result := ComputeGasSelf(structlogs, gasUsed) + + require.Len(t, result, 3) + assert.Equal(t, uint64(3), result[0], "PUSH1 gas_self should equal gas_used") + assert.Equal(t, uint64(2100), result[1], "SLOAD gas_self should equal gas_used") + assert.Equal(t, uint64(3), result[2], "ADD gas_self should equal gas_used") +} + +func TestComputeGasSelf_SimpleCall(t *testing.T) { + // CALL at depth 1 with child opcodes at depth 2 + // gas_self for CALL should be gas_used minus sum of direct children's gas_used + structlogs := []execution.StructLog{ + {Op: "PUSH1", Gas: 100000, GasCost: 3, Depth: 1}, // index 0 + {Op: "CALL", Gas: 99997, GasCost: 100, Depth: 1}, // index 1: CALL + {Op: "PUSH1", Gas: 63000, GasCost: 3, Depth: 2}, // index 2: child + {Op: "ADD", Gas: 62000, GasCost: 3, Depth: 2}, // index 3: child + {Op: "STOP", Gas: 61000, GasCost: 0, Depth: 2}, // index 4: child + {Op: "POP", Gas: 97000, GasCost: 2, Depth: 1}, // index 5: back to parent + } + + // gas_used values (computed by ComputeGasUsed logic): + // PUSH1[0]: 100000 - 99997 = 3 + // CALL[1]: 99997 - 97000 = 2997 (includes child execution) + // PUSH1[2]: 63000 - 62000 = 1000 + // ADD[3]: 62000 - 61000 = 1000 + // STOP[4]: 0 (pre-calculated, last at depth 2) + // POP[5]: 2 (pre-calculated, last opcode) + gasUsed := []uint64{3, 2997, 1000, 1000, 0, 2} + + result := ComputeGasSelf(structlogs, gasUsed) + + require.Len(t, result, 6) + + // Non-CALL opcodes: gas_self == gas_used + assert.Equal(t, uint64(3), result[0], "PUSH1 gas_self") + assert.Equal(t, uint64(1000), result[2], "child PUSH1 gas_self") + assert.Equal(t, uint64(1000), result[3], "child ADD gas_self") + assert.Equal(t, uint64(0), result[4], "child STOP gas_self") + assert.Equal(t, uint64(2), result[5], "POP gas_self") + + // CALL: gas_self = gas_used - sum(direct children) + // direct children at depth 2: indices 2, 3, 4 + // sum = 1000 + 1000 + 0 = 2000 + // gas_self = 2997 - 2000 = 997 + assert.Equal(t, uint64(997), result[1], "CALL gas_self should be overhead only") +} + +func TestComputeGasSelf_NestedCalls(t *testing.T) { + // This is the critical test: nested CALLs where we must only sum direct children. + // If we sum ALL descendants, we double count and get incorrect (often 0) values. + // + // Structure: + // CALL A (depth 1) -> child frame at depth 2 + // ├─ PUSH (depth 2) + // ├─ CALL B (depth 2) -> grandchild frame at depth 3 + // │ ├─ ADD (depth 3) + // │ └─ STOP (depth 3) + // └─ STOP (depth 2) + structlogs := []execution.StructLog{ + {Op: "CALL", Gas: 100000, GasCost: 100, Depth: 1}, // index 0: CALL A + {Op: "PUSH1", Gas: 80000, GasCost: 3, Depth: 2}, // index 1: direct child of A + {Op: "CALL", Gas: 79000, GasCost: 100, Depth: 2}, // index 2: CALL B (direct child of A) + {Op: "ADD", Gas: 50000, GasCost: 3, Depth: 3}, // index 3: direct child of B + {Op: "STOP", Gas: 49000, GasCost: 0, Depth: 3}, // index 4: direct child of B + {Op: "STOP", Gas: 75000, GasCost: 0, Depth: 2}, // index 5: direct child of A + {Op: "POP", Gas: 90000, GasCost: 2, Depth: 1}, // index 6: back to depth 1 + } + + // gas_used values: + // CALL A[0]: 100000 - 90000 = 10000 (includes all nested) + // PUSH[1]: 80000 - 79000 = 1000 + // CALL B[2]: 79000 - 75000 = 4000 (includes grandchild) + // ADD[3]: 50000 - 49000 = 1000 + // STOP[4]: 0 (pre-calculated) + // STOP[5]: 0 (pre-calculated) + // POP[6]: 2 (pre-calculated) + gasUsed := []uint64{10000, 1000, 4000, 1000, 0, 0, 2} + + result := ComputeGasSelf(structlogs, gasUsed) + + require.Len(t, result, 7) + + // CALL A: direct children at depth 2 are indices 1, 2, 5 + // sum of direct children = 1000 + 4000 + 0 = 5000 + // gas_self = 10000 - 5000 = 5000 + // Note: We do NOT include indices 3, 4 (depth 3) because they're grandchildren, + // and CALL B's gas_used (4000) already includes them. + assert.Equal(t, uint64(5000), result[0], "CALL A gas_self should exclude nested CALL's children") + + // CALL B: direct children at depth 3 are indices 3, 4 + // sum of direct children = 1000 + 0 = 1000 + // gas_self = 4000 - 1000 = 3000 + assert.Equal(t, uint64(3000), result[2], "CALL B gas_self should be its overhead") + + // Non-CALL opcodes: gas_self == gas_used + assert.Equal(t, uint64(1000), result[1], "PUSH gas_self") + assert.Equal(t, uint64(1000), result[3], "ADD gas_self") + assert.Equal(t, uint64(0), result[4], "STOP depth 3 gas_self") + assert.Equal(t, uint64(0), result[5], "STOP depth 2 gas_self") + assert.Equal(t, uint64(2), result[6], "POP gas_self") +} + +func TestComputeGasSelf_SiblingCalls(t *testing.T) { + // Two sibling CALLs at the same depth, each with their own children + structlogs := []execution.StructLog{ + {Op: "CALL", Gas: 100000, GasCost: 100, Depth: 1}, // index 0: first CALL + {Op: "ADD", Gas: 60000, GasCost: 3, Depth: 2}, // index 1: child of first CALL + {Op: "STOP", Gas: 59000, GasCost: 0, Depth: 2}, // index 2: child of first CALL + {Op: "CALL", Gas: 90000, GasCost: 100, Depth: 1}, // index 3: second CALL + {Op: "MUL", Gas: 50000, GasCost: 5, Depth: 2}, // index 4: child of second CALL + {Op: "STOP", Gas: 49000, GasCost: 0, Depth: 2}, // index 5: child of second CALL + {Op: "POP", Gas: 80000, GasCost: 2, Depth: 1}, // index 6 + } + + // gas_used: + // CALL[0]: 100000 - 90000 = 10000 + // ADD[1]: 60000 - 59000 = 1000 + // STOP[2]: 0 + // CALL[3]: 90000 - 80000 = 10000 + // MUL[4]: 50000 - 49000 = 1000 + // STOP[5]: 0 + // POP[6]: 2 + gasUsed := []uint64{10000, 1000, 0, 10000, 1000, 0, 2} + + result := ComputeGasSelf(structlogs, gasUsed) + + require.Len(t, result, 7) + + // First CALL: direct children = indices 1, 2 + // gas_self = 10000 - (1000 + 0) = 9000 + assert.Equal(t, uint64(9000), result[0], "first CALL gas_self") + + // Second CALL: direct children = indices 4, 5 + // gas_self = 10000 - (1000 + 0) = 9000 + assert.Equal(t, uint64(9000), result[3], "second CALL gas_self") +} + +func TestComputeGasSelf_CreateOpcode(t *testing.T) { + // CREATE should be handled the same as CALL + structlogs := []execution.StructLog{ + {Op: "CREATE", Gas: 100000, GasCost: 32000, Depth: 1}, // index 0 + {Op: "PUSH1", Gas: 70000, GasCost: 3, Depth: 2}, // index 1: constructor + {Op: "RETURN", Gas: 69000, GasCost: 0, Depth: 2}, // index 2: constructor + {Op: "POP", Gas: 80000, GasCost: 2, Depth: 1}, // index 3 + } + + // gas_used: + // CREATE[0]: 100000 - 80000 = 20000 + // PUSH[1]: 70000 - 69000 = 1000 + // RETURN[2]: 0 + // POP[3]: 2 + gasUsed := []uint64{20000, 1000, 0, 2} + + result := ComputeGasSelf(structlogs, gasUsed) + + require.Len(t, result, 4) + + // CREATE: direct children = indices 1, 2 + // gas_self = 20000 - (1000 + 0) = 19000 + assert.Equal(t, uint64(19000), result[0], "CREATE gas_self should be overhead only") + assert.Equal(t, uint64(1000), result[1], "PUSH gas_self") + assert.Equal(t, uint64(0), result[2], "RETURN gas_self") + assert.Equal(t, uint64(2), result[3], "POP gas_self") +} + +func TestComputeGasSelf_DelegateCallAndStaticCall(t *testing.T) { + // DELEGATECALL and STATICCALL should also be handled + structlogs := []execution.StructLog{ + {Op: "DELEGATECALL", Gas: 100000, GasCost: 100, Depth: 1}, + {Op: "ADD", Gas: 60000, GasCost: 3, Depth: 2}, + {Op: "STOP", Gas: 59000, GasCost: 0, Depth: 2}, + {Op: "STATICCALL", Gas: 90000, GasCost: 100, Depth: 1}, + {Op: "MUL", Gas: 50000, GasCost: 5, Depth: 2}, + {Op: "STOP", Gas: 49000, GasCost: 0, Depth: 2}, + {Op: "POP", Gas: 80000, GasCost: 2, Depth: 1}, + } + + gasUsed := []uint64{10000, 1000, 0, 10000, 1000, 0, 2} + + result := ComputeGasSelf(structlogs, gasUsed) + + require.Len(t, result, 7) + + // DELEGATECALL: gas_self = 10000 - 1000 = 9000 + assert.Equal(t, uint64(9000), result[0], "DELEGATECALL gas_self") + + // STATICCALL: gas_self = 10000 - 1000 = 9000 + assert.Equal(t, uint64(9000), result[3], "STATICCALL gas_self") +} + +func TestComputeGasSelf_CallWithNoChildren(t *testing.T) { + // CALL to precompile or empty contract - no child opcodes + // In this case, gas_self should equal gas_used + structlogs := []execution.StructLog{ + {Op: "CALL", Gas: 100000, GasCost: 100, Depth: 1}, + {Op: "POP", Gas: 97400, GasCost: 2, Depth: 1}, // immediately back at depth 1 + } + + // gas_used: + // CALL: 100000 - 97400 = 2600 (just the CALL overhead, no child execution) + // POP: 2 + gasUsed := []uint64{2600, 2} + + result := ComputeGasSelf(structlogs, gasUsed) + + require.Len(t, result, 2) + + // No children, so gas_self = gas_used + assert.Equal(t, uint64(2600), result[0], "CALL with no children: gas_self == gas_used") + assert.Equal(t, uint64(2), result[1], "POP gas_self") +} + +func TestComputeGasSelf_DeeplyNestedCalls(t *testing.T) { + // Test 4 levels of nesting to ensure correct handling + structlogs := []execution.StructLog{ + {Op: "CALL", Gas: 100000, GasCost: 100, Depth: 1}, // index 0: A + {Op: "CALL", Gas: 90000, GasCost: 100, Depth: 2}, // index 1: B + {Op: "CALL", Gas: 80000, GasCost: 100, Depth: 3}, // index 2: C + {Op: "CALL", Gas: 70000, GasCost: 100, Depth: 4}, // index 3: D + {Op: "ADD", Gas: 60000, GasCost: 3, Depth: 5}, // index 4: innermost + {Op: "STOP", Gas: 59000, GasCost: 0, Depth: 5}, // index 5 + {Op: "STOP", Gas: 65000, GasCost: 0, Depth: 4}, // index 6 + {Op: "STOP", Gas: 74000, GasCost: 0, Depth: 3}, // index 7 + {Op: "STOP", Gas: 83000, GasCost: 0, Depth: 2}, // index 8 + {Op: "POP", Gas: 92000, GasCost: 2, Depth: 1}, // index 9 + } + + // gas_used: + // A[0]: 100000 - 92000 = 8000 + // B[1]: 90000 - 83000 = 7000 + // C[2]: 80000 - 74000 = 6000 + // D[3]: 70000 - 65000 = 5000 + // ADD[4]: 60000 - 59000 = 1000 + // STOP[5]: 0 + // STOP[6]: 0 + // STOP[7]: 0 + // STOP[8]: 0 + // POP[9]: 2 + gasUsed := []uint64{8000, 7000, 6000, 5000, 1000, 0, 0, 0, 0, 2} + + result := ComputeGasSelf(structlogs, gasUsed) + + require.Len(t, result, 10) + + // CALL A: direct children at depth 2 = [B, STOP] = indices 1, 8 + // gas_self = 8000 - (7000 + 0) = 1000 + assert.Equal(t, uint64(1000), result[0], "CALL A gas_self") + + // CALL B: direct children at depth 3 = [C, STOP] = indices 2, 7 + // gas_self = 7000 - (6000 + 0) = 1000 + assert.Equal(t, uint64(1000), result[1], "CALL B gas_self") + + // CALL C: direct children at depth 4 = [D, STOP] = indices 3, 6 + // gas_self = 6000 - (5000 + 0) = 1000 + assert.Equal(t, uint64(1000), result[2], "CALL C gas_self") + + // CALL D: direct children at depth 5 = [ADD, STOP] = indices 4, 5 + // gas_self = 5000 - (1000 + 0) = 4000 + assert.Equal(t, uint64(4000), result[3], "CALL D gas_self") +} diff --git a/pkg/processor/transaction/structlog/processor.go b/pkg/processor/transaction/structlog/processor.go index 5a2b66c..bc82ff2 100644 --- a/pkg/processor/transaction/structlog/processor.go +++ b/pkg/processor/transaction/structlog/processor.go @@ -5,6 +5,7 @@ import ( "fmt" "math" + "github.com/ClickHouse/ch-go" "github.com/hibiken/asynq" "github.com/redis/go-redis/v9" "github.com/sirupsen/logrus" @@ -174,3 +175,53 @@ func (p *Processor) getProcessForwardsQueue() string { func (p *Processor) getProcessBackwardsQueue() string { return tracker.PrefixedProcessBackwardsQueue(ProcessorName, p.redisPrefix) } + +// insertStructlogs inserts structlogs into ClickHouse using columnar protocol. +func (p *Processor) insertStructlogs(ctx context.Context, structlogs []Structlog) error { + if len(structlogs) == 0 { + return nil + } + + // Add timeout for ClickHouse operation + insertCtx, cancel := context.WithTimeout(ctx, tracker.DefaultClickHouseTimeout) + defer cancel() + + cols := NewColumns() + for _, sl := range structlogs { + cols.Append( + sl.UpdatedDateTime.Time(), + sl.BlockNumber, + sl.TransactionHash, + sl.TransactionIndex, + sl.TransactionGas, + sl.TransactionFailed, + sl.TransactionReturnValue, + sl.Index, + sl.ProgramCounter, + sl.Operation, + sl.Gas, + sl.GasCost, + sl.GasUsed, + sl.GasSelf, + sl.Depth, + sl.ReturnData, + sl.Refund, + sl.Error, + sl.CallToAddress, + sl.CallFrameID, + sl.CallFramePath, + sl.MetaNetworkName, + ) + } + + input := cols.Input() + + if err := p.clickhouse.Do(insertCtx, ch.Query{ + Body: input.Into(p.config.Table), + Input: input, + }); err != nil { + return fmt.Errorf("failed to insert structlogs: %w", err) + } + + return nil +} diff --git a/pkg/processor/transaction/structlog/processor_test.go b/pkg/processor/transaction/structlog/processor_test.go index 571039e..a676b5f 100644 --- a/pkg/processor/transaction/structlog/processor_test.go +++ b/pkg/processor/transaction/structlog/processor_test.go @@ -155,11 +155,14 @@ func TestStructlogCountReturn(t *testing.T) { structLog.Gas, // gas structLog.GasCost, // gasCost structLog.GasCost, // gasUsed (simplified) + structLog.GasCost, // gasSelf (simplified) structLog.Depth, // depth structLog.ReturnData, // returnData structLog.Refund, // refund structLog.Error, // error nil, // callTo + uint32(0), // callFrameID + []uint32{}, // callFramePath "test", // network ) } @@ -228,20 +231,23 @@ func TestMemoryManagement(t *testing.T) { uint64(i), // blockNumber "0x1234567890abcdef1234567890abcdef12345678", // txHash uint32(i%100), // txIndex - 21000, // txGas + uint64(21000), // txGas false, // txFailed nil, // txReturnValue uint32(i), // index uint32(i*2), // pc "SSTORE", // op uint64(21000-i), // gas - 5000, // gasCost - 5000, // gasUsed - 1, // depth + uint64(5000), // gasCost + uint64(5000), // gasUsed + uint64(5000), // gasSelf + uint64(1), // depth nil, // returnData nil, // refund nil, // error nil, // callTo + uint32(0), // callFrameID + []uint32{}, // callFramePath "mainnet", // network ) } @@ -340,9 +346,9 @@ func TestChunkProcessing(t *testing.T) { // Fill columns with test data for i := 0; i < tt.inputSize; i++ { cols.Append( - now, uint64(i), "0xtest", 0, 21000, false, nil, - uint32(i), uint32(i), "PUSH1", 20000, 3, 3, 1, - nil, nil, nil, nil, "test", + now, uint64(i), "0xtest", uint32(0), uint64(21000), false, nil, + uint32(i), uint32(i), "PUSH1", uint64(20000), uint64(3), uint64(3), uint64(3), uint64(1), + nil, nil, nil, nil, uint32(0), []uint32{}, "test", ) } @@ -390,9 +396,9 @@ func TestColumnsAppendAndReset(t *testing.T) { num := uint64(42) cols.Append( - now, 100, "0xabc", 0, 21000, false, &str, - 0, 100, "PUSH1", 20000, 3, 3, 1, - nil, &num, nil, nil, "mainnet", + now, uint64(100), "0xabc", uint32(0), uint64(21000), false, &str, + uint32(0), uint32(100), "PUSH1", uint64(20000), uint64(3), uint64(3), uint64(3), uint64(1), + nil, &num, nil, nil, uint32(0), []uint32{}, "mainnet", ) assert.Equal(t, 1, cols.Rows()) @@ -400,9 +406,9 @@ func TestColumnsAppendAndReset(t *testing.T) { // Append more rows for i := 0; i < 99; i++ { cols.Append( - now, 100, "0xabc", 0, 21000, false, nil, - uint32(i+1), 100, "PUSH1", 20000, 3, 3, 1, - nil, nil, nil, nil, "mainnet", + now, uint64(100), "0xabc", uint32(0), uint64(21000), false, nil, + uint32(i+1), uint32(100), "PUSH1", uint64(20000), uint64(3), uint64(3), uint64(3), uint64(1), + nil, nil, nil, nil, uint32(0), []uint32{}, "mainnet", ) } @@ -417,10 +423,10 @@ func TestColumnsInput(t *testing.T) { cols := transaction_structlog.NewColumns() input := cols.Input() - // Verify all 19 columns are present - assert.Len(t, input, 19) + // Verify all 22 columns are present + assert.Len(t, input, 22) assert.Equal(t, "updated_date_time", input[0].Name) - assert.Equal(t, "meta_network_name", input[18].Name) + assert.Equal(t, "meta_network_name", input[21].Name) } // Tests from tasks_test.go diff --git a/pkg/processor/transaction/structlog/transaction_processing.go b/pkg/processor/transaction/structlog/transaction_processing.go index 6902d18..5c25e97 100644 --- a/pkg/processor/transaction/structlog/transaction_processing.go +++ b/pkg/processor/transaction/structlog/transaction_processing.go @@ -4,19 +4,153 @@ import ( "context" "fmt" "io" + "strings" "time" "github.com/ClickHouse/ch-go" - "github.com/ethereum/go-ethereum/core/types" "github.com/sirupsen/logrus" - "github.com/ethpandaops/execution-processor/pkg/common" + pcommon "github.com/ethpandaops/execution-processor/pkg/common" "github.com/ethpandaops/execution-processor/pkg/ethereum/execution" "github.com/ethpandaops/execution-processor/pkg/processor/tracker" ) +// Structlog represents a single EVM opcode execution within a transaction trace. +// See gas_cost.go for detailed documentation on the gas fields. +// +//nolint:tagliatelle // ClickHouse uses snake_case column names +type Structlog struct { + UpdatedDateTime ClickHouseTime `json:"updated_date_time"` + BlockNumber uint64 `json:"block_number"` + TransactionHash string `json:"transaction_hash"` + TransactionIndex uint32 `json:"transaction_index"` + TransactionGas uint64 `json:"transaction_gas"` + TransactionFailed bool `json:"transaction_failed"` + TransactionReturnValue *string `json:"transaction_return_value"` + Index uint32 `json:"index"` + ProgramCounter uint32 `json:"program_counter"` + Operation string `json:"operation"` + + // Gas is the remaining gas before this opcode executes. + Gas uint64 `json:"gas"` + + // GasCost is from the execution node trace. For CALL/CREATE opcodes, this is the + // gas stipend passed to the child frame, not the call overhead. + GasCost uint64 `json:"gas_cost"` + + // GasUsed is computed as gas[i] - gas[i+1] at the same depth level. + // For CALL/CREATE opcodes, this includes the call overhead plus all child frame gas. + // Summing across all opcodes will double count child frame gas. + GasUsed uint64 `json:"gas_used"` + + // GasSelf excludes child frame gas. For CALL/CREATE opcodes, this is just the call + // overhead (warm/cold access, memory expansion). For other opcodes, equals GasUsed. + // Summing across all opcodes gives total execution gas without double counting. + GasSelf uint64 `json:"gas_self"` + + Depth uint64 `json:"depth"` + ReturnData *string `json:"return_data"` + Refund *uint64 `json:"refund"` + Error *string `json:"error"` + CallToAddress *string `json:"call_to_address"` + CallFrameID uint32 `json:"call_frame_id"` + CallFramePath []uint32 `json:"call_frame_path"` + MetaNetworkName string `json:"meta_network_name"` +} + +// isCallOpcode returns true if the opcode initiates a call that creates a child frame. +// Note: CREATE/CREATE2 always execute code (constructor), so they always increase depth. +// CALL-type opcodes may target EOAs (no code) or precompiles (special handling). +func isCallOpcode(op string) bool { + switch op { + case OpcodeCALL, OpcodeCALLCODE, OpcodeDELEGATECALL, OpcodeSTATICCALL: + return true + default: + return false + } +} + +// precompileAddresses contains all known EVM precompile addresses. +// +// Precompile calls don't appear in trace_transaction results (unlike EOA calls which do). +// This is used to distinguish EOA calls from precompile calls when depth doesn't increase. +// +// Note: Low addresses like 0x5c, 0x60, etc. are NOT precompiles - they're real EOAs/contracts +// deployed early in Ethereum's history. Only the addresses below are actual precompiles. +// +// Addresses sourced from go-ethereum PrecompiledContractsOsaka (superset of all forks): +// https://github.com/ethereum/go-ethereum/blob/master/core/vm/contracts.go +// +// We cannot import go-ethereum directly because it depends on github.com/holiman/bloomfilter/v2, +// which conflicts with Erigon's fork (github.com/AskAlexSharov/bloomfilter/v2) when this package +// is embedded in Erigon. The two bloomfilter versions have incompatible APIs. +var precompileAddresses = map[string]bool{ + "0x0000000000000000000000000000000000000001": true, // ecrecover + "0x0000000000000000000000000000000000000002": true, // sha256 + "0x0000000000000000000000000000000000000003": true, // ripemd160 + "0x0000000000000000000000000000000000000004": true, // identity (dataCopy) + "0x0000000000000000000000000000000000000005": true, // modexp (bigModExp) + "0x0000000000000000000000000000000000000006": true, // bn256Add (ecAdd) + "0x0000000000000000000000000000000000000007": true, // bn256ScalarMul (ecMul) + "0x0000000000000000000000000000000000000008": true, // bn256Pairing (ecPairing) + "0x0000000000000000000000000000000000000009": true, // blake2f + "0x000000000000000000000000000000000000000a": true, // kzgPointEvaluation (EIP-4844, Cancun) + "0x000000000000000000000000000000000000000b": true, // bls12381G1Add (EIP-2537, Osaka) + "0x000000000000000000000000000000000000000c": true, // bls12381G1MultiExp (EIP-2537, Osaka) + "0x000000000000000000000000000000000000000d": true, // bls12381G2Add (EIP-2537, Osaka) + "0x000000000000000000000000000000000000000e": true, // bls12381G2MultiExp (EIP-2537, Osaka) + "0x000000000000000000000000000000000000000f": true, // bls12381Pairing (EIP-2537, Osaka) + "0x0000000000000000000000000000000000000010": true, // bls12381MapG1 (EIP-2537, Osaka) + "0x0000000000000000000000000000000000000011": true, // bls12381MapG2 (EIP-2537, Osaka) + "0x0000000000000000000000000000000000000100": true, // p256Verify (EIP-7212, Osaka) +} + +// isPrecompile returns true if the address is a known EVM precompile. +// Precompile calls don't appear in trace_transaction results (unlike EOA calls which do). +func isPrecompile(addr string) bool { + // Normalize to lowercase with 0x prefix and full 40 hex chars + hex := strings.TrimPrefix(strings.ToLower(addr), "0x") + + for len(hex) < 40 { + hex = "0" + hex + } + + return precompileAddresses["0x"+hex] +} + +// ProcessSingleTransaction processes a single transaction and inserts its structlogs directly to ClickHouse. +func (p *Processor) ProcessSingleTransaction(ctx context.Context, block execution.Block, index int, tx execution.Transaction) (int, error) { + // Extract structlog data + structlogs, err := p.ExtractStructlogs(ctx, block, index, tx) + if err != nil { + return 0, err + } + + // Store count before processing + structlogCount := len(structlogs) + + // Ensure we clear the slice on exit to allow GC + defer func() { + // Clear the slice to release memory + structlogs = nil + }() + + // Send for direct insertion + if err := p.insertStructlogs(ctx, structlogs); err != nil { + pcommon.TransactionsProcessed.WithLabelValues(p.network.Name, "structlog", "failed").Inc() + + return 0, fmt.Errorf("failed to insert structlogs: %w", err) + } + + // Record success metrics + pcommon.TransactionsProcessed.WithLabelValues(p.network.Name, "structlog", "success").Inc() + + return structlogCount, nil +} + // ProcessTransaction processes a transaction using ch-go columnar streaming. -func (p *Processor) ProcessTransaction(ctx context.Context, block *types.Block, index int, tx *types.Transaction) (int, error) { +func (p *Processor) ProcessTransaction(ctx context.Context, block execution.Block, index int, tx execution.Transaction) (int, error) { + // Get trace from execution node trace, err := p.getTransactionTrace(ctx, tx, block) if err != nil { return 0, fmt.Errorf("failed to get trace: %w", err) @@ -27,8 +161,30 @@ func (p *Processor) ProcessTransaction(ctx context.Context, block *types.Block, return 0, nil } - // Compute actual gas used for each structlog - gasUsed := ComputeGasUsed(trace.Structlogs) + // Check if GasUsed is pre-computed by the tracer (embedded mode). + // In embedded mode, skip the post-processing computation. + // In RPC mode, compute GasUsed from gas differences. + precomputedGasUsed := hasPrecomputedGasUsed(trace.Structlogs) + + var gasUsed []uint64 + if !precomputedGasUsed { + gasUsed = ComputeGasUsed(trace.Structlogs) + } else { + // Extract pre-computed GasUsed values from structlogs (embedded mode) + gasUsed = make([]uint64, len(trace.Structlogs)) + for i := range trace.Structlogs { + gasUsed[i] = trace.Structlogs[i].GasUsed + } + } + + // Compute self gas (excludes child frame gas for CALL/CREATE opcodes) + gasSelf := ComputeGasSelf(trace.Structlogs, gasUsed) + + // Initialize call frame tracker + callTracker := NewCallTracker() + + // Pre-compute CREATE/CREATE2 addresses from trace stack + createAddresses := ComputeCreateAddresses(trace.Structlogs) chunkSize := p.config.ChunkSize if chunkSize == 0 { @@ -64,6 +220,12 @@ func (p *Processor) ProcessTransaction(ctx context.Context, block *types.Block, for i := currentIdx; i < end; i++ { sl := &trace.Structlogs[i] + + // Track call frame based on depth changes + frameID, framePath := callTracker.ProcessDepthChange(sl.Depth) + + callToAddr := p.extractCallAddressWithCreate(sl, i, createAddresses) + cols.Append( now, blockNum, @@ -78,14 +240,61 @@ func (p *Processor) ProcessTransaction(ctx context.Context, block *types.Block, sl.Gas, sl.GasCost, gasUsed[i], + gasSelf[i], sl.Depth, sl.ReturnData, sl.Refund, sl.Error, - p.extractCallAddress(sl), + callToAddr, + frameID, + framePath, p.network.Name, ) + // Check for EOA call: CALL-type opcode where depth stays the same (immediate return) + // and target is not a precompile (precompiles don't create trace frames) + if isCallOpcode(sl.Op) && callToAddr != nil { + isEOACall := false + + if i+1 < totalCount { + // Next opcode exists - check if depth stayed the same + nextDepth := trace.Structlogs[i+1].Depth + if nextDepth == sl.Depth && !isPrecompile(*callToAddr) { + isEOACall = true + } + } + + if isEOACall { + // Emit synthetic structlog for EOA frame + eoaFrameID, eoaFramePath := callTracker.IssueFrameID() + + cols.Append( + now, + blockNum, + txHash, + txIndex, + trace.Gas, + trace.Failed, + trace.ReturnValue, + uint32(i), //nolint:gosec // Same index as parent CALL + uint32(0), // No PC for EOA + "", // Empty = synthetic EOA frame + uint64(0), // Gas + uint64(0), // GasCost + uint64(0), // GasUsed + uint64(0), // GasSelf + sl.Depth+1, // One level deeper than caller + nil, // ReturnData + nil, // Refund + sl.Error, // Inherit error if CALL failed + callToAddr, // The EOA address + eoaFrameID, + eoaFramePath, + p.network.Name, + ) + } + } + // Free original trace data immediately to help GC trace.Structlogs[i] = execution.StructLog{} } @@ -113,14 +322,14 @@ func (p *Processor) ProcessTransaction(ctx context.Context, block *types.Block, } // Record success metrics - common.TransactionsProcessed.WithLabelValues(p.network.Name, "structlog", "success").Inc() - common.ClickHouseInsertsRows.WithLabelValues(p.network.Name, ProcessorName, p.config.Table, "success", "").Add(float64(totalCount)) + pcommon.TransactionsProcessed.WithLabelValues(p.network.Name, "structlog", "success").Inc() + pcommon.ClickHouseInsertsRows.WithLabelValues(p.network.Name, ProcessorName, p.config.Table, "success", "").Add(float64(totalCount)) return totalCount, nil } // getTransactionTrace gets the trace for a transaction. -func (p *Processor) getTransactionTrace(ctx context.Context, tx *types.Transaction, block *types.Block) (*execution.TraceTransaction, error) { +func (p *Processor) getTransactionTrace(ctx context.Context, tx execution.Transaction, block execution.Block) (*execution.TraceTransaction, error) { // Get execution node node := p.pool.GetHealthyExecutionNode() if node == nil { @@ -140,13 +349,302 @@ func (p *Processor) getTransactionTrace(ctx context.Context, tx *types.Transacti return trace, nil } -// extractCallAddress extracts the call address from a structlog if it's a CALL operation. +// formatAddress normalizes an address to exactly 42 characters (0x + 40 hex). +// +// Background: The EVM is a 256-bit (32-byte) stack machine. ALL stack values are 32 bytes, +// including addresses. When execution clients like Erigon/Geth return debug traces, the +// stack array contains raw 32-byte values as hex strings (66 chars with 0x prefix). +// +// However, Ethereum addresses are only 160 bits (20 bytes, 40 hex chars). In EVM/ABI encoding, +// addresses are stored in the LOWER 160 bits of the 32-byte word (right-aligned, left-padded +// with zeros). For example, address 0x7a250d5630b4cf539739df2c5dacb4c659f2488d on the stack: +// +// 0x0000000000000000000000007a250d5630b4cf539739df2c5dacb4c659f2488d +// |-------- upper 12 bytes (zeros) --------||---- lower 20 bytes (address) ----| +// +// Some contracts may have non-zero upper bytes in the stack value. The EVM ignores these +// when interpreting the value as an address - only the lower 20 bytes are used. +// +// This function handles three cases: +// 1. Short addresses (e.g., "0x1" for precompiles): left-pad with zeros to 40 hex chars +// 2. Full 32-byte stack values (66 chars): extract rightmost 40 hex chars (lower 160 bits) +// 3. Normal 42-char addresses: return as-is +func formatAddress(addr string) string { + // Remove 0x prefix if present + hex := strings.TrimPrefix(addr, "0x") + + // If longer than 40 chars, extract the lower 20 bytes (rightmost 40 hex chars). + // This handles raw 32-byte stack values from execution client traces. + if len(hex) > 40 { + hex = hex[len(hex)-40:] + } + + // Left-pad with zeros to 40 chars if shorter (handles precompiles like 0x1), + // then add 0x prefix + return fmt.Sprintf("0x%040s", hex) +} + +// extractCallAddress extracts the call address from a structlog for CALL-family opcodes. +// +// Supports two modes for backward compatibility: +// - Embedded mode: CallToAddress is pre-populated by the tracer, use directly. +// - RPC mode: CallToAddress is nil, extract from Stack[len-2] for CALL-family opcodes. +// +// Stack layout in Erigon/Geth debug traces: +// - Array index 0 = bottom of stack (oldest value, first pushed) +// - Array index len-1 = top of stack (newest value, first to be popped) +// +// When a CALL opcode executes, its arguments are at the top of the stack: +// +// CALL/CALLCODE: [..., retSize, retOffset, argsSize, argsOffset, value, addr, gas] +// DELEGATECALL/STATICCALL: [..., retSize, retOffset, argsSize, argsOffset, addr, gas] +// ^ ^ +// len-2 len-1 +// +// The address is always at Stack[len-2] (second from top), regardless of how many +// other values exist below the CALL arguments on the stack. +// +// Note: The stack value is a raw 32-byte word. The formatAddress function extracts +// the actual 20-byte address from the lower 160 bits. func (p *Processor) extractCallAddress(structLog *execution.StructLog) *string { - if structLog.Op == "CALL" && structLog.Stack != nil && len(*structLog.Stack) > 1 { + // Embedded mode: use pre-extracted CallToAddress + if structLog.CallToAddress != nil { + return structLog.CallToAddress + } + + // RPC mode fallback: extract from Stack for CALL-family opcodes + if structLog.Stack == nil || len(*structLog.Stack) < 2 { + return nil + } + + switch structLog.Op { + case "CALL", "CALLCODE", "DELEGATECALL", "STATICCALL": + // Extract the raw 32-byte stack value at the address position (second from top). + // formatAddress will normalize it to a proper 20-byte address. stackValue := (*structLog.Stack)[len(*structLog.Stack)-2] + addr := formatAddress(stackValue) + + return &addr + default: + return nil + } +} + +// extractCallAddressWithCreate extracts the call address, using createAddresses map for CREATE/CREATE2 opcodes. +func (p *Processor) extractCallAddressWithCreate(structLog *execution.StructLog, index int, createAddresses map[int]*string) *string { + // For CREATE/CREATE2, use the pre-computed address from the trace + if structLog.Op == "CREATE" || structLog.Op == "CREATE2" { + if createAddresses != nil { + return createAddresses[index] + } + + return nil + } + + return p.extractCallAddress(structLog) +} + +// ComputeCreateAddresses pre-computes the created contract addresses for all CREATE/CREATE2 opcodes. +// It scans the trace and extracts addresses from the stack when each CREATE's constructor returns. +// The returned map contains opcode index -> created address (only for CREATE/CREATE2 opcodes). +func ComputeCreateAddresses(structlogs []execution.StructLog) map[int]*string { + result := make(map[int]*string) + + // Track pending CREATE operations: (index, depth) + type pendingCreate struct { + index int + depth uint64 + } + + var pending []pendingCreate + + for i, log := range structlogs { + // Resolve pending CREATEs that have completed. + // A CREATE at depth D completes when we see an opcode at depth <= D + // (either immediately if CREATE failed, or after constructor returns). + for len(pending) > 0 { + last := pending[len(pending)-1] + + // If current opcode is at or below CREATE's depth and it's not the CREATE itself + if log.Depth <= last.depth && i > last.index { + // Extract address from top of stack (created address or 0 if failed) + if log.Stack != nil && len(*log.Stack) > 0 { + addr := formatAddress((*log.Stack)[len(*log.Stack)-1]) + result[last.index] = &addr + } + + pending = pending[:len(pending)-1] + } else { + break + } + } + + // Track new CREATE/CREATE2 + if log.Op == "CREATE" || log.Op == "CREATE2" { + pending = append(pending, pendingCreate{index: i, depth: log.Depth}) + } + } + + return result +} + +// ExtractStructlogs extracts structlog data from a transaction without inserting to database. +func (p *Processor) ExtractStructlogs(ctx context.Context, block execution.Block, index int, tx execution.Transaction) ([]Structlog, error) { + start := time.Now() + + defer func() { + duration := time.Since(start) + pcommon.TransactionProcessingDuration.WithLabelValues(p.network.Name, "structlog").Observe(duration.Seconds()) + }() + + // Get execution node + node := p.pool.GetHealthyExecutionNode() + if node == nil { + return nil, fmt.Errorf("no healthy execution node available") + } + + // Process transaction with timeout + processCtx, cancel := context.WithTimeout(ctx, 30*time.Second) + defer cancel() + + // Get transaction trace + trace, err := node.DebugTraceTransaction(processCtx, tx.Hash().String(), block.Number(), execution.StackTraceOptions()) + if err != nil { + return nil, fmt.Errorf("failed to trace transaction: %w", err) + } + + // Convert trace to structlog rows + var structlogs []Structlog + + uIndex := uint32(index) //nolint:gosec // index is bounded by block.Transactions() length + + if trace != nil { + // Check if GasUsed is pre-computed by the tracer (embedded mode). + // In embedded mode, extract values from structlogs. + // In RPC mode, compute GasUsed from gas differences. + precomputedGasUsed := hasPrecomputedGasUsed(trace.Structlogs) + + var gasUsed []uint64 + if precomputedGasUsed { + // Extract pre-computed GasUsed values from structlogs (embedded mode) + gasUsed = make([]uint64, len(trace.Structlogs)) + for i := range trace.Structlogs { + gasUsed[i] = trace.Structlogs[i].GasUsed + } + } else { + // Compute GasUsed from gas differences (RPC mode) + gasUsed = ComputeGasUsed(trace.Structlogs) + } + + // Compute self gas (excludes child frame gas for CALL/CREATE opcodes) + gasSelf := ComputeGasSelf(trace.Structlogs, gasUsed) + + // Initialize call frame tracker + callTracker := NewCallTracker() + + // Pre-compute CREATE/CREATE2 addresses from trace stack + createAddresses := ComputeCreateAddresses(trace.Structlogs) + + // Pre-allocate slice for better memory efficiency + structlogs = make([]Structlog, 0, len(trace.Structlogs)) + + for i, structLog := range trace.Structlogs { + // Track call frame based on depth changes + frameID, framePath := callTracker.ProcessDepthChange(structLog.Depth) + + callToAddr := p.extractCallAddressWithCreate(&structLog, i, createAddresses) + + // Get GasUsed: use pre-computed value from tracer (embedded) or computed value (RPC). + var gasUsedValue uint64 + if precomputedGasUsed { + gasUsedValue = structLog.GasUsed + } else { + gasUsedValue = gasUsed[i] + } + + row := Structlog{ + UpdatedDateTime: NewClickHouseTime(time.Now()), + BlockNumber: block.Number().Uint64(), + TransactionHash: tx.Hash().String(), + TransactionIndex: uIndex, + TransactionGas: trace.Gas, + TransactionFailed: trace.Failed, + TransactionReturnValue: trace.ReturnValue, + Index: uint32(i), //nolint:gosec // index is bounded by structlogs length + ProgramCounter: structLog.PC, + Operation: structLog.Op, + Gas: structLog.Gas, + GasCost: structLog.GasCost, + GasUsed: gasUsedValue, + GasSelf: gasSelf[i], + Depth: structLog.Depth, + ReturnData: structLog.ReturnData, + Refund: structLog.Refund, + Error: structLog.Error, + CallToAddress: callToAddr, + CallFrameID: frameID, + CallFramePath: framePath, + MetaNetworkName: p.network.Name, + } + + structlogs = append(structlogs, row) + + // Check for EOA call: CALL-type opcode where depth stays the same (immediate return) + // and target is not a precompile (precompiles don't create trace frames) + if isCallOpcode(structLog.Op) && callToAddr != nil { + isEOACall := false + + if i+1 < len(trace.Structlogs) { + // Next opcode exists - check if depth stayed the same + // Depth increase = entered contract code (not EOA) + // Depth decrease = call returned/failed (not EOA) + // Depth same = called EOA or precompile (immediate return) + nextDepth := trace.Structlogs[i+1].Depth + if nextDepth == structLog.Depth && !isPrecompile(*callToAddr) { + isEOACall = true + } + } + // Note: If last opcode is a CALL, we can't determine if it's EOA + // because we don't have a next opcode to compare depth with. + // These are typically failed calls at end of execution. + + if isEOACall { + // Emit synthetic structlog for EOA frame + eoaFrameID, eoaFramePath := callTracker.IssueFrameID() + + eoaRow := Structlog{ + UpdatedDateTime: NewClickHouseTime(time.Now()), + BlockNumber: block.Number().Uint64(), + TransactionHash: tx.Hash().String(), + TransactionIndex: uIndex, + TransactionGas: trace.Gas, + TransactionFailed: trace.Failed, + TransactionReturnValue: trace.ReturnValue, + Index: uint32(i), //nolint:gosec // Same index as parent CALL + ProgramCounter: 0, // No PC for EOA + Operation: "", // Empty = synthetic EOA frame + Gas: 0, + GasCost: 0, + GasUsed: 0, + GasSelf: 0, + Depth: structLog.Depth + 1, // One level deeper than caller + ReturnData: nil, + Refund: nil, + Error: structLog.Error, // Inherit error if CALL failed + CallToAddress: callToAddr, // The EOA address + CallFrameID: eoaFrameID, + CallFramePath: eoaFramePath, + MetaNetworkName: p.network.Name, + } + + structlogs = append(structlogs, eoaRow) + } + } + } - return &stackValue + // Clear the original trace data to free memory + trace.Structlogs = nil } - return nil + return structlogs, nil }