diff --git a/arbnode/mel/extraction/batch_lookup.go b/arbnode/mel/extraction/batch_lookup.go index 47a0b83a51..7130876cba 100644 --- a/arbnode/mel/extraction/batch_lookup.go +++ b/arbnode/mel/extraction/batch_lookup.go @@ -60,12 +60,6 @@ func ParseBatchesFromBlock( return nil, nil, fmt.Errorf("error fetching tx by hash: %v in ParseBatchesFromBlock: %w ", log.TxHash, err) } - // Record this log for MEL validation. This is a very cheap operation in native mode - // and is optimized for recording mode as well. - if _, err := logsFetcher.LogsForTxIndex(ctx, parentChainHeader.Hash(), log.TxIndex); err != nil { - return nil, nil, fmt.Errorf("error recording relevant logs: %w", err) - } - batch := &mel.SequencerInboxBatch{ BlockHash: log.BlockHash, ParentChainBlockNumber: log.BlockNumber, diff --git a/arbnode/mel/extraction/delayed_message_lookup.go b/arbnode/mel/extraction/delayed_message_lookup.go index d94a6d2f93..c0cc92b29d 100644 --- a/arbnode/mel/extraction/delayed_message_lookup.go +++ b/arbnode/mel/extraction/delayed_message_lookup.go @@ -39,11 +39,6 @@ func parseDelayedMessagesFromBlock( // On Arbitrum One, this is the bridge contract which emits a MessageDelivered event. if log.Address == melState.DelayedMessagePostingTargetAddress { relevantLogs = append(relevantLogs, log) - // Record this log for MEL validation. This is a very cheap operation in native mode - // and is optimized for recording mode as well. - if _, err := logsFetcher.LogsForTxIndex(ctx, parentChainHeader.Hash(), log.TxIndex); err != nil { - return nil, fmt.Errorf("error recording relevant logs: %w", err) - } } } if len(relevantLogs) > 0 { @@ -83,11 +78,6 @@ func parseDelayedMessagesFromBlock( return nil, err } messageData[common.BigToHash(msgNum)] = msg - // Record this log for MEL validation. This is a very cheap operation in native mode - // and is optimized for recording mode as well. - if _, err := logsFetcher.LogsForTxIndex(ctx, parentChainHeader.Hash(), inboxMsgLog.TxIndex); err != nil { - return nil, fmt.Errorf("error recording relevant logs: %w", err) - } } for i, parsedLog := range messageDeliveredEvents { msgKey := common.BigToHash(parsedLog.MessageIndex) diff --git a/arbnode/mel/extraction/message_extraction_function.go b/arbnode/mel/extraction/message_extraction_function.go index 38a895dd0c..3437841783 100644 --- a/arbnode/mel/extraction/message_extraction_function.go +++ b/arbnode/mel/extraction/message_extraction_function.go @@ -233,10 +233,10 @@ func extractMessagesImpl( } for _, msg := range messagesInBatch { messages = append(messages, msg) - state.MsgCount += 1 if err = state.AccumulateMessage(msg); err != nil { return nil, nil, nil, nil, fmt.Errorf("failed to accumulate message: %w", err) } + state.MsgCount += 1 } state.BatchCount += 1 batchMetas = append(batchMetas, &mel.BatchMetadata{ diff --git a/arbnode/mel/recording/delayed_msg_database.go b/arbnode/mel/recording/delayed_msg_database.go index cf6b980050..ea67c4177e 100644 --- a/arbnode/mel/recording/delayed_msg_database.go +++ b/arbnode/mel/recording/delayed_msg_database.go @@ -68,7 +68,8 @@ func (r *DelayedMsgDatabase) ReadDelayedMessage(ctx context.Context, state *mel. return nil, err } hashDelayedHash := crypto.Keccak256(delayed.Hash().Bytes()) - r.preimages[common.BytesToHash(hashDelayedHash)] = delayedMsgBytes + r.preimages[common.BytesToHash(hashDelayedHash)] = delayed.Hash().Bytes() + r.preimages[delayed.Hash()] = delayedMsgBytes return delayed, nil } diff --git a/arbnode/mel/recording/receipt_recorder.go b/arbnode/mel/recording/receipt_recorder.go index c5dc8d8cfb..ee2b7e119f 100644 --- a/arbnode/mel/recording/receipt_recorder.go +++ b/arbnode/mel/recording/receipt_recorder.go @@ -1,184 +1,118 @@ package melrecording import ( - "bytes" "context" "errors" "fmt" "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/core/rawdb" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/crypto" - "github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/trie" - "github.com/ethereum/go-ethereum/trie/trienode" - "github.com/ethereum/go-ethereum/triedb" + melextraction "github.com/offchainlabs/nitro/arbnode/mel/extraction" "github.com/offchainlabs/nitro/arbutil" "github.com/offchainlabs/nitro/daprovider" - "github.com/offchainlabs/nitro/mel-replay" ) -// ReceiptRecorder records preimages corresponding to the receipts of a parent chain block -// needed during the message extraction. These preimages are needed for MEL validation and -// is used in creation of the validation entries by the MEL validator -type ReceiptRecorder struct { - parentChainReader BlockReader - parentChainBlockHash common.Hash - parentChainBlockNumber uint64 - recordPreimages daprovider.PreimageRecorder - receipts []*types.Receipt - logs []*types.Log - relevantLogsTxIndexes map[uint]struct{} - trieDB *triedb.Database - blockReceiptHash common.Hash +// Implements a hasher that captures preimages of hashes as it computes them. +type preimageRecordingHasher struct { + trie *trie.StackTrie + recordPreimages daprovider.PreimageRecorder } -// NewReceiptRecorder returns ReceiptRecorder that records -// the receipt preimages into the given preimages map -func NewReceiptRecorder( - parentChainReader BlockReader, - parentChainBlockHash common.Hash, - preimages daprovider.PreimagesMap, -) (*ReceiptRecorder, error) { - if preimages == nil { - return nil, errors.New("preimages recording destination cannot be nil") +func newRecordingHasher(recordPreimages daprovider.PreimageRecorder) *preimageRecordingHasher { + h := &preimageRecordingHasher{ + recordPreimages: recordPreimages, } - return &ReceiptRecorder{ - parentChainReader: parentChainReader, - parentChainBlockHash: parentChainBlockHash, - recordPreimages: daprovider.RecordPreimagesTo(preimages), - relevantLogsTxIndexes: make(map[uint]struct{}), - }, nil + // OnTrieNode callback captures all trie nodes. + onTrieNode := func(path []byte, hash common.Hash, blob []byte) { + // Deep copy the blob since the callback warns contents may change, so this is required. + recordPreimages(hash, common.CopyBytes(blob), arbutil.Keccak256PreimageType) + } + + h.trie = trie.NewStackTrie(onTrieNode) + return h +} + +func (h *preimageRecordingHasher) Reset() { + onTrieNode := func(path []byte, hash common.Hash, blob []byte) { + h.recordPreimages(hash, common.CopyBytes(blob), arbutil.Keccak256PreimageType) + } + h.trie = trie.NewStackTrie(onTrieNode) +} + +func (h *preimageRecordingHasher) Update(key, value []byte) error { + valueHash := crypto.Keccak256Hash(value) + h.recordPreimages(valueHash, common.CopyBytes(value), arbutil.Keccak256PreimageType) + return h.trie.Update(key, value) +} + +func (h *preimageRecordingHasher) Hash() common.Hash { + return h.trie.Hash() } -// Initialize must be called first to setup the recording trie database and store all the -// block receipts into the triedb. Without this, preimage recording is not possible and -// the other functions will error out if called beforehand -func (rr *ReceiptRecorder) Initialize(ctx context.Context) error { - block, err := rr.parentChainReader.BlockByHash(ctx, rr.parentChainBlockHash) +// recordedLogsFetcher holds the logs of recorded receipt preimages. These preimages are +// needed for MEL validation and is used in creation of the validation entries by the MEL validator +type recordedLogsFetcher struct { + parentChainBlockHash common.Hash + receipts []*types.Receipt + logs []*types.Log +} + +// RecordReceipts records preimages of all the receipts in a block and returns a LogsFetcher +// that provides these logs during message extraction +func RecordReceipts(ctx context.Context, parentChainReader BlockReader, parentChainBlockHash common.Hash, preimages daprovider.PreimagesMap) (melextraction.LogsFetcher, error) { + if preimages == nil { + return nil, errors.New("preimages recording destination cannot be nil") + } + block, err := parentChainReader.BlockByHash(ctx, parentChainBlockHash) if err != nil { - return err + return nil, err } - tdb := triedb.NewDatabase(rawdb.NewMemoryDatabase(), &triedb.Config{ - Preimages: true, - }) - receiptsTrie := trie.NewEmpty(tdb) - var receipts []*types.Receipt txs := block.Body().Transactions - for i, tx := range txs { - receipt, err := rr.parentChainReader.TransactionReceipt(ctx, tx.Hash()) + var receipts []*types.Receipt + var logs []*types.Log + for _, tx := range txs { + receipt, err := parentChainReader.TransactionReceipt(ctx, tx.Hash()) if err != nil { - return fmt.Errorf("error fetching receipt for tx: %v, blockHash: %v", tx.Hash(), block.Hash()) + return nil, fmt.Errorf("error fetching receipt for tx: %v, blockHash: %v", tx.Hash(), block.Hash()) } receipts = append(receipts, receipt) - rr.logs = append(rr.logs, receipt.Logs...) - // #nosec G115 - indexBytes, err := rlp.EncodeToBytes(uint64(i)) - if err != nil { - return fmt.Errorf("failed to encode index %d: %w", i, err) - } - receiptBytes, err := receipt.MarshalBinary() - if err != nil { - return fmt.Errorf("failed to marshal receipt %d: %w", i, err) - } - if err := receiptsTrie.Update(indexBytes, receiptBytes); err != nil { - return fmt.Errorf("failed to update trie at index %d: %w", i, err) - } + logs = append(logs, receipt.Logs...) } - root, nodes := receiptsTrie.Commit(false) - if root != block.ReceiptHash() { - return fmt.Errorf("computed root %s doesn't match header root %s", - root.Hex(), block.ReceiptHash().Hex()) + hasher := newRecordingHasher(daprovider.RecordPreimagesTo(preimages)) + receiptsRoot := types.DeriveSha(types.Receipts(receipts), hasher) + if receiptsRoot != block.ReceiptHash() { + return nil, fmt.Errorf("computed root %s doesn't match header root %s", receiptsRoot.Hex(), block.ReceiptHash().Hex()) } - if nodes != nil { - if err := tdb.Update(root, types.EmptyRootHash, 0, trienode.NewWithNodeSet(nodes), nil); err != nil { - return fmt.Errorf("failed to commit trie nodes: %w", err) - } - } - if err := tdb.Commit(root, false); err != nil { - return fmt.Errorf("failed to commit database: %w", err) - } - rr.receipts = receipts - rr.trieDB = tdb - rr.blockReceiptHash = root - rr.parentChainBlockNumber = block.NumberU64() + return &recordedLogsFetcher{ + logs: logs, + receipts: receipts, + parentChainBlockHash: parentChainBlockHash, + }, nil +} + +// RecordPreimages records preimages corresponding to all the receipts in a block using preimageRecordingHasher +func (rr *recordedLogsFetcher) RecordPreimages(ctx context.Context) error { + return nil } -func (rr *ReceiptRecorder) LogsForTxIndex(ctx context.Context, parentChainBlockHash common.Hash, txIndex uint) ([]*types.Log, error) { - if rr.trieDB == nil { - return nil, errors.New("TransactionRecorder not initialized") - } +func (rr *recordedLogsFetcher) LogsForTxIndex(ctx context.Context, parentChainBlockHash common.Hash, txIndex uint) ([]*types.Log, error) { if rr.parentChainBlockHash != parentChainBlockHash { return nil, fmt.Errorf("parentChainBlockHash mismatch. expected: %v got: %v", rr.parentChainBlockHash, parentChainBlockHash) } - if _, recorded := rr.relevantLogsTxIndexes[txIndex]; recorded { - return rr.receipts[txIndex].Logs, nil - } // #nosec G115 if int(txIndex) >= len(rr.receipts) { return nil, fmt.Errorf("index out of range: %d", txIndex) } - recordingDB := &TxsAndReceiptsDatabase{ - underlying: rr.trieDB, - recorder: rr.recordPreimages, // RecordingDB will record relevant preimages into the given preimagesmap - } - recordingTDB := triedb.NewDatabase(recordingDB, nil) - receiptsTrie, err := trie.New(trie.TrieID(rr.blockReceiptHash), recordingTDB) - if err != nil { - return nil, fmt.Errorf("failed to create trie: %w", err) - } - indexBytes, err := rlp.EncodeToBytes(txIndex) - if err != nil { - return nil, fmt.Errorf("failed to encode index: %w", err) - } - receiptBytes, err := receiptsTrie.Get(indexBytes) - if err != nil { - return nil, fmt.Errorf("failed to get receipt from trie: %w", err) - } - receipt := new(types.Receipt) - if err = receipt.UnmarshalBinary(receiptBytes); err != nil { - return nil, fmt.Errorf("failed to unmarshal receipt: %w", err) - } - // Add the receipt marshaled binary by hash to the preimages map - rr.recordPreimages(crypto.Keccak256Hash(receiptBytes), receiptBytes, arbutil.Keccak256PreimageType) - // Fill in the TxIndex (give as input to this method) into the logs so that Tx recording - // is possible. This field is one of the derived fields of Log hence won't be stored in trie. - // - // We use this same trick in validation as well in order to link a tx with its logs - for _, log := range receipt.Logs { - log.TxIndex = txIndex - log.BlockHash = parentChainBlockHash - log.BlockNumber = rr.parentChainBlockNumber - } - rr.relevantLogsTxIndexes[txIndex] = struct{}{} - return receipt.Logs, nil + return rr.receipts[txIndex].Logs, nil } -func (rr *ReceiptRecorder) LogsForBlockHash(ctx context.Context, parentChainBlockHash common.Hash) ([]*types.Log, error) { - if rr.trieDB == nil { - return nil, errors.New("TransactionRecorder not initialized") - } +func (rr *recordedLogsFetcher) LogsForBlockHash(ctx context.Context, parentChainBlockHash common.Hash) ([]*types.Log, error) { if rr.parentChainBlockHash != parentChainBlockHash { return nil, fmt.Errorf("parentChainBlockHash mismatch. expected: %v got: %v", rr.parentChainBlockHash, parentChainBlockHash) } return rr.logs, nil } - -// CollectTxIndicesPreimage adds the array of relevant tx indexes to the preimages map as a value -// to the key represented by parentChainBlockHash. -func (rr *ReceiptRecorder) CollectTxIndicesPreimage() error { - var relevantLogsTxIndexes []uint - for k := range rr.relevantLogsTxIndexes { - relevantLogsTxIndexes = append(relevantLogsTxIndexes, k) - } - var buf bytes.Buffer - if err := rlp.Encode(&buf, relevantLogsTxIndexes); err != nil { - return err - } - relevantTxIndicesKey := melreplay.RelevantTxIndexesKey(rr.parentChainBlockHash) - rr.recordPreimages(relevantTxIndicesKey, buf.Bytes(), arbutil.Keccak256PreimageType) - return nil -} diff --git a/arbnode/mel/recording/tx_recorder.go b/arbnode/mel/recording/tx_recorder.go index d86786a54c..f4cdb9fb5a 100644 --- a/arbnode/mel/recording/tx_recorder.go +++ b/arbnode/mel/recording/tx_recorder.go @@ -109,7 +109,7 @@ func (tr *TransactionRecorder) TransactionByLog(ctx context.Context, log *types. if int(log.TxIndex) >= len(tr.txs) { return nil, fmt.Errorf("index out of range: %d", log.TxIndex) } - recordingDB := &TxsAndReceiptsDatabase{ + recordingDB := &TxsRecordingDatabase{ underlying: tr.trieDB, recorder: tr.recordPreimages, // RecordingDB will record relevant preimages into the given preimagesmap } diff --git a/arbnode/mel/recording/txs_and_receipts_database.go b/arbnode/mel/recording/txs_and_receipts_database.go deleted file mode 100644 index 1c5aaaef2b..0000000000 --- a/arbnode/mel/recording/txs_and_receipts_database.go +++ /dev/null @@ -1,110 +0,0 @@ -package melrecording - -import ( - "fmt" - - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/ethdb" - "github.com/ethereum/go-ethereum/triedb" - - "github.com/offchainlabs/nitro/arbutil" - "github.com/offchainlabs/nitro/daprovider" -) - -type TxsAndReceiptsDatabase struct { - underlying *triedb.Database - recorder daprovider.PreimageRecorder -} - -func (rdb *TxsAndReceiptsDatabase) Get(key []byte) ([]byte, error) { - hash := common.BytesToHash(key) - value, err := rdb.underlying.Node(hash) - if err != nil { - return nil, err - } - if rdb.recorder != nil { - rdb.recorder(hash, value, arbutil.Keccak256PreimageType) - } - - return value, nil -} -func (rdb *TxsAndReceiptsDatabase) Has(key []byte) (bool, error) { - hash := common.BytesToHash(key) - _, err := rdb.underlying.Node(hash) - return err == nil, nil -} -func (rdb *TxsAndReceiptsDatabase) Put(key []byte, value []byte) error { - return fmt.Errorf("Put not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) Delete(key []byte) error { - return fmt.Errorf("Delete not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) DeleteRange(start, end []byte) error { - return fmt.Errorf("DeleteRange not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) ReadAncients(fn func(ethdb.AncientReaderOp) error) (err error) { - return fmt.Errorf("ReadAncients not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) ModifyAncients(func(ethdb.AncientWriteOp) error) (int64, error) { - return 0, fmt.Errorf("ReadAncients not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) SyncAncient() error { - return fmt.Errorf("SyncAncient not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) TruncateHead(n uint64) (uint64, error) { - return 0, fmt.Errorf("TruncateHead not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) TruncateTail(n uint64) (uint64, error) { - return 0, fmt.Errorf("TruncateTail not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) Append(kind string, number uint64, item interface{}) error { - return fmt.Errorf("Append not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) AppendRaw(kind string, number uint64, item []byte) error { - return fmt.Errorf("AppendRaw not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) AncientDatadir() (string, error) { - return "", fmt.Errorf("AncientDatadir not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) Ancient(kind string, number uint64) ([]byte, error) { - return nil, fmt.Errorf("Ancient not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) AncientRange(kind string, start, count, maxBytes uint64) ([][]byte, error) { - return nil, fmt.Errorf("AncientRange not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) AncientBytes(kind string, id, offset, length uint64) ([]byte, error) { - return nil, fmt.Errorf("AncientBytes not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) Ancients() (uint64, error) { - return 0, fmt.Errorf("Ancients not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) Tail() (uint64, error) { - return 0, fmt.Errorf("Tail not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) AncientSize(kind string) (uint64, error) { - return 0, fmt.Errorf("AncientSize not supported on recording DB") -} -func (rdb *TxsAndReceiptsDatabase) Compact(start []byte, limit []byte) error { - return nil -} -func (rdb *TxsAndReceiptsDatabase) SyncKeyValue() error { - return nil -} -func (rdb *TxsAndReceiptsDatabase) Stat() (string, error) { - return "", nil -} -func (rdb *TxsAndReceiptsDatabase) WasmDataBase() ethdb.KeyValueStore { - return nil -} -func (rdb *TxsAndReceiptsDatabase) NewBatch() ethdb.Batch { - return nil -} -func (rdb *TxsAndReceiptsDatabase) NewBatchWithSize(size int) ethdb.Batch { - return nil -} -func (rdb *TxsAndReceiptsDatabase) NewIterator(prefix []byte, start []byte) ethdb.Iterator { - return nil -} -func (rdb *TxsAndReceiptsDatabase) Close() error { - return nil -} diff --git a/arbnode/mel/recording/txs_recording_database.go b/arbnode/mel/recording/txs_recording_database.go new file mode 100644 index 0000000000..05a96bed20 --- /dev/null +++ b/arbnode/mel/recording/txs_recording_database.go @@ -0,0 +1,110 @@ +package melrecording + +import ( + "fmt" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/ethdb" + "github.com/ethereum/go-ethereum/triedb" + + "github.com/offchainlabs/nitro/arbutil" + "github.com/offchainlabs/nitro/daprovider" +) + +type TxsRecordingDatabase struct { + underlying *triedb.Database + recorder daprovider.PreimageRecorder +} + +func (rdb *TxsRecordingDatabase) Get(key []byte) ([]byte, error) { + hash := common.BytesToHash(key) + value, err := rdb.underlying.Node(hash) + if err != nil { + return nil, err + } + if rdb.recorder != nil { + rdb.recorder(hash, value, arbutil.Keccak256PreimageType) + } + + return value, nil +} +func (rdb *TxsRecordingDatabase) Has(key []byte) (bool, error) { + hash := common.BytesToHash(key) + _, err := rdb.underlying.Node(hash) + return err == nil, nil +} +func (rdb *TxsRecordingDatabase) Put(key []byte, value []byte) error { + return fmt.Errorf("Put not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) Delete(key []byte) error { + return fmt.Errorf("Delete not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) DeleteRange(start, end []byte) error { + return fmt.Errorf("DeleteRange not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) ReadAncients(fn func(ethdb.AncientReaderOp) error) (err error) { + return fmt.Errorf("ReadAncients not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) ModifyAncients(func(ethdb.AncientWriteOp) error) (int64, error) { + return 0, fmt.Errorf("ReadAncients not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) SyncAncient() error { + return fmt.Errorf("SyncAncient not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) TruncateHead(n uint64) (uint64, error) { + return 0, fmt.Errorf("TruncateHead not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) TruncateTail(n uint64) (uint64, error) { + return 0, fmt.Errorf("TruncateTail not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) Append(kind string, number uint64, item interface{}) error { + return fmt.Errorf("Append not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) AppendRaw(kind string, number uint64, item []byte) error { + return fmt.Errorf("AppendRaw not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) AncientDatadir() (string, error) { + return "", fmt.Errorf("AncientDatadir not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) Ancient(kind string, number uint64) ([]byte, error) { + return nil, fmt.Errorf("Ancient not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) AncientRange(kind string, start, count, maxBytes uint64) ([][]byte, error) { + return nil, fmt.Errorf("AncientRange not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) AncientBytes(kind string, id, offset, length uint64) ([]byte, error) { + return nil, fmt.Errorf("AncientBytes not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) Ancients() (uint64, error) { + return 0, fmt.Errorf("Ancients not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) Tail() (uint64, error) { + return 0, fmt.Errorf("Tail not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) AncientSize(kind string) (uint64, error) { + return 0, fmt.Errorf("AncientSize not supported on recording DB") +} +func (rdb *TxsRecordingDatabase) Compact(start []byte, limit []byte) error { + return nil +} +func (rdb *TxsRecordingDatabase) SyncKeyValue() error { + return nil +} +func (rdb *TxsRecordingDatabase) Stat() (string, error) { + return "", nil +} +func (rdb *TxsRecordingDatabase) WasmDataBase() ethdb.KeyValueStore { + return nil +} +func (rdb *TxsRecordingDatabase) NewBatch() ethdb.Batch { + return nil +} +func (rdb *TxsRecordingDatabase) NewBatchWithSize(size int) ethdb.Batch { + return nil +} +func (rdb *TxsRecordingDatabase) NewIterator(prefix []byte, start []byte) ethdb.Iterator { + return nil +} +func (rdb *TxsRecordingDatabase) Close() error { + return nil +} diff --git a/arbnode/mel/state.go b/arbnode/mel/state.go index 0d3e612ce8..331679c495 100644 --- a/arbnode/mel/state.go +++ b/arbnode/mel/state.go @@ -146,16 +146,11 @@ func (s *State) Clone() *State { DelayedMessageMerklePartials: delayedMessageMerklePartials, delayedMessageBacklog: delayedMessageBacklog, readCountFromBacklog: s.readCountFromBacklog, - // we copy msgPreimagesDest as is to continue recordng of msg preimages + // we pass along msgPreimagesDest to continue recordng of msg preimages msgPreimagesDest: s.msgPreimagesDest, } } -func (s *State) ReadMessage(msgIdx uint64) (*arbostypes.MessageWithMetadata, error) { - // TODO: Unimplemented. - return &arbostypes.MessageWithMetadata{}, nil -} - func (s *State) AccumulateMessage(msg *arbostypes.MessageWithMetadata) error { if s.msgsAcc == nil { log.Debug("Initializing MelState's msgsAcc") @@ -240,10 +235,6 @@ func getPartialsAndRoot(acc *merkleAccumulator.MerkleAccumulator) ([]common.Hash return partials, root, err } -func (s *State) SetMsgsAcc(acc *merkleAccumulator.MerkleAccumulator) { - s.msgsAcc = acc -} - func (s *State) GetSeenDelayedMsgsAcc() *merkleAccumulator.MerkleAccumulator { return s.seenDelayedMsgsAcc } diff --git a/arbos/merkleAccumulator/merkleAccumulator.go b/arbos/merkleAccumulator/merkleAccumulator.go index 3a8db6c996..06a1ca196b 100644 --- a/arbos/merkleAccumulator/merkleAccumulator.go +++ b/arbos/merkleAccumulator/merkleAccumulator.go @@ -180,9 +180,9 @@ func (acc *MerkleAccumulator) Append(itemHash common.Hash, itemBytesToRecord ... events = append(events, MerkleTreeNodeEvent{level, size - 1, common.BytesToHash(soFar)}) } if acc.recordPreimages != nil && itemBytesToRecord != nil { - keyBytes := crypto.Keccak256(itemHash.Bytes()) - key := common.BytesToHash(keyBytes) - acc.recordPreimages(key, itemBytesToRecord) + hashItemHash := crypto.Keccak256(itemHash.Bytes()) + acc.recordPreimages(common.BytesToHash(hashItemHash), itemHash.Bytes()) + acc.recordPreimages(itemHash, itemBytesToRecord) } return events, nil } diff --git a/cmd/unified-replay/main.go b/cmd/unified-replay/main.go index 608ae78406..f1bdfc3c61 100644 --- a/cmd/unified-replay/main.go +++ b/cmd/unified-replay/main.go @@ -17,10 +17,10 @@ import ( "github.com/ethereum/go-ethereum/rlp" "github.com/offchainlabs/nitro/arbnode/mel" - melextraction "github.com/offchainlabs/nitro/arbnode/mel/extraction" + "github.com/offchainlabs/nitro/arbnode/mel/extraction" "github.com/offchainlabs/nitro/arbutil" "github.com/offchainlabs/nitro/daprovider" - melreplay "github.com/offchainlabs/nitro/mel-replay" + "github.com/offchainlabs/nitro/mel-replay" "github.com/offchainlabs/nitro/melwavmio" "github.com/offchainlabs/nitro/wavmio" ) @@ -54,15 +54,13 @@ func main() { positionInMEL := melwavmio.GetPositionInMEL() if melState.MsgCount > positionInMEL { - nextMsg, err := melState.ReadMessage(positionInMEL) + resolver := &wavmPreimageResolver{} + msgReader := melreplay.NewMessageReader(resolver) + nextMsg, err := msgReader.Read(context.Background(), melState, positionInMEL) if err != nil { panic(fmt.Errorf("error reading message idx %d: %w", positionInMEL, err)) } - msgHash, err := mel.MessageHash(nextMsg) - if err != nil { - panic(fmt.Errorf("error hashing message idx %d: %w", positionInMEL, err)) - } - melwavmio.SetMELMsgHash(msgHash) + melwavmio.SetMELMsgHash(nextMsg.Hash()) } else { melwavmio.SetMELMsgHash(common.Hash{}) } diff --git a/cmd/unified-replay/preimage_resolver.go b/cmd/unified-replay/preimage_resolver.go index f24af85047..27b100a766 100644 --- a/cmd/unified-replay/preimage_resolver.go +++ b/cmd/unified-replay/preimage_resolver.go @@ -9,14 +9,11 @@ import ( "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/crypto/kzg4844" + "github.com/offchainlabs/nitro/arbutil" "github.com/offchainlabs/nitro/melwavmio" ) -type preimageResolver interface { - ResolveTypedPreimage(preimageType arbutil.PreimageType, hash common.Hash) ([]byte, error) -} - type wavmPreimageResolver struct{} func (w *wavmPreimageResolver) ResolveTypedPreimage( diff --git a/mel-replay/merkle_tree_fetcher.go b/mel-replay/merkle_tree_fetcher.go index 8bf7eea396..804815532a 100644 --- a/mel-replay/merkle_tree_fetcher.go +++ b/mel-replay/merkle_tree_fetcher.go @@ -44,7 +44,11 @@ func fetchObjectFromMerkleTree[T any](merkleRoot common.Hash, merkleDepth int, m currentDepth-- } // At this point, currentHash should be the hash of the object. - objectBytes, err := preimageResolver.ResolveTypedPreimage(arbutil.Keccak256PreimageType, currentHash) + objectHashBytes, err := preimageResolver.ResolveTypedPreimage(arbutil.Keccak256PreimageType, currentHash) + if err != nil { + return nil, err + } + objectBytes, err := preimageResolver.ResolveTypedPreimage(arbutil.Keccak256PreimageType, common.BytesToHash(objectHashBytes)) if err != nil { return nil, err } diff --git a/mel-replay/merkle_tree_fetcher_test.go b/mel-replay/merkle_tree_fetcher_test.go index 856b45bb9b..3669a1cff6 100644 --- a/mel-replay/merkle_tree_fetcher_test.go +++ b/mel-replay/merkle_tree_fetcher_test.go @@ -180,7 +180,9 @@ func buildMerkleTree(t *testing.T, messages []*mel.DelayedInboxMessage) (map[com require.NoError(t, err) hash := crypto.Keccak256Hash(encoded) preimages[hash] = encoded - leafHashes[i] = hash + hashOfHash := crypto.Keccak256Hash(hash.Bytes()) + preimages[hashOfHash] = hash.Bytes() + leafHashes[i] = hashOfHash } currentLevel := leafHashes diff --git a/mel-replay/message_reader_test.go b/mel-replay/message_reader_test.go index 75ca64a3e5..e3b6f86697 100644 --- a/mel-replay/message_reader_test.go +++ b/mel-replay/message_reader_test.go @@ -32,14 +32,20 @@ func TestRecordingMessagePreimagesAndReadingMessages(t *testing.T) { }) } state := &mel.State{} - // Simulate extracting of Messages in native mode to record preimages - preimages := make(daprovider.PreimagesMap) - require.NoError(t, state.RecordMsgPreimagesTo(preimages)) - for i := range numMsgs { + for i := range uint64(5) { require.NoError(t, state.AccumulateMessage(messages[i])) state.MsgCount++ } require.NoError(t, state.GenerateMessageMerklePartialsAndRoot()) + // Simulate extracting of Messages in native mode to record preimages + preimages := make(daprovider.PreimagesMap) + require.NoError(t, state.RecordMsgPreimagesTo(preimages)) + newState := state.Clone() + for i := uint64(5); i < numMsgs; i++ { + require.NoError(t, newState.AccumulateMessage(messages[i])) + newState.MsgCount++ + } + require.NoError(t, newState.GenerateMessageMerklePartialsAndRoot()) // Test reading in wasm mode msgReader := melreplay.NewMessageReader( @@ -48,8 +54,8 @@ func TestRecordingMessagePreimagesAndReadingMessages(t *testing.T) { preimages, ), ) - for i := range numMsgs { - msg, err := msgReader.Read(ctx, state, i) + for i := uint64(5); i < numMsgs; i++ { + msg, err := msgReader.Read(ctx, newState, i) require.NoError(t, err) require.Equal(t, msg.Hash(), messages[i].Hash()) } diff --git a/mel-replay/receipt_fetcher.go b/mel-replay/receipt_fetcher.go index 5567ebc09d..82c207312f 100644 --- a/mel-replay/receipt_fetcher.go +++ b/mel-replay/receipt_fetcher.go @@ -16,7 +16,6 @@ import ( "github.com/ethereum/go-ethereum/triedb" "github.com/offchainlabs/nitro/arbnode/mel/extraction" - "github.com/offchainlabs/nitro/arbutil" ) type receiptFetcherForBlock struct { @@ -24,7 +23,10 @@ type receiptFetcherForBlock struct { preimageResolver PreimageResolver } -func NewLogsFetcher(header *types.Header, preimageResolver PreimageResolver) melextraction.LogsFetcher { +func NewLogsFetcher( + header *types.Header, + preimageResolver PreimageResolver, +) melextraction.LogsFetcher { return &receiptFetcherForBlock{header, preimageResolver} } @@ -48,34 +50,8 @@ func (rf *receiptFetcherForBlock) LogsForTxIndex(ctx context.Context, parentChai return receipt.Logs, nil } -// LogsForBlockHash first gets the txIndexes corresponding to the relevant logs by reading -// the key `parentChainBlockHash` from the preimages and then fetches logs for each of these txIndexes -func (rf *receiptFetcherForBlock) LogsForBlockHash(ctx context.Context, parentChainBlockHash common.Hash) ([]*types.Log, error) { - if rf.header.Hash() != parentChainBlockHash { - return nil, errors.New("parentChainBlockHash mismatch") - } - relevantTxIndicesKey := RelevantTxIndexesKey(rf.header.Hash()) - txIndexData, err := rf.preimageResolver.ResolveTypedPreimage(arbutil.Keccak256PreimageType, relevantTxIndicesKey) - if err != nil { - return nil, err - } - var txIndexes []uint - if err := rlp.DecodeBytes(txIndexData, &txIndexes); err != nil { - return nil, err - } - var relevantLogs []*types.Log - for _, txIndex := range txIndexes { - logs, err := rf.LogsForTxIndex(ctx, parentChainBlockHash, txIndex) - if err != nil { - return nil, err - } - relevantLogs = append(relevantLogs, logs...) - } - return relevantLogs, nil -} - // TODO: LogsForBlockHashAllLogs is kept, in case we go with an implementation of returning all logs present in a block -func (rf *receiptFetcherForBlock) LogsForBlockHashAllLogs(ctx context.Context, parentChainBlockHash common.Hash) ([]*types.Log, error) { +func (rf *receiptFetcherForBlock) LogsForBlockHash(ctx context.Context, parentChainBlockHash common.Hash) ([]*types.Log, error) { if rf.header.Hash() != parentChainBlockHash { return nil, errors.New("parentChainBlockHash mismatch") } @@ -87,20 +63,30 @@ func (rf *receiptFetcherForBlock) LogsForBlockHashAllLogs(ctx context.Context, p if err != nil { return nil, err } - entries, indices := collectTrieEntries(receiptsTrie) + entries, indices, maxIndex := collectTrieEntries(receiptsTrie) + if len(indices) != 0 && uint64(len(indices)) != maxIndex+1 { + return nil, fmt.Errorf("incorrect number of receipts in trie, want: %d, have: %d", maxIndex+1, len(indices)) + } rawReceipts := reconstructOrderedData(entries, indices) receipts, err := decodeReceiptsData(rawReceipts) if err != nil { return nil, err } var relevantLogs []*types.Log - for _, receipt := range receipts { - relevantLogs = append(relevantLogs, receipt.Logs...) + for i, receipt := range receipts { + // This is needed to enable fetching corresponding tx from the txFetcher + for _, log := range receipt.Logs { + // #nosec G115 + log.TxIndex = uint(i) + log.BlockHash = rf.header.Hash() + log.BlockNumber = rf.header.Number.Uint64() + relevantLogs = append(relevantLogs, log) + } } return relevantLogs, nil } -func collectTrieEntries(txTrie *trie.Trie) ([][]byte, []uint64) { +func collectTrieEntries(txTrie *trie.Trie) ([][]byte, []uint64, uint64) { nodeIterator, iterErr := txTrie.NodeIterator(nil) if iterErr != nil { panic(iterErr) @@ -108,6 +94,7 @@ func collectTrieEntries(txTrie *trie.Trie) ([][]byte, []uint64) { var rawValues [][]byte var indexKeys []uint64 + var maxIndex uint64 for nodeIterator.Next(true) { if !nodeIterator.Leaf() { @@ -123,10 +110,11 @@ func collectTrieEntries(txTrie *trie.Trie) ([][]byte, []uint64) { } indexKeys = append(indexKeys, decodedIndex) + maxIndex = max(maxIndex, decodedIndex) rawValues = append(rawValues, nodeIterator.LeafBlob()) } - return rawValues, indexKeys + return rawValues, indexKeys, maxIndex } func reconstructOrderedData(rawValues [][]byte, indices []uint64) []hexutil.Bytes { diff --git a/mel-replay/receipt_recorder_and_receipt_fetcher_test.go b/mel-replay/receipt_recorder_and_receipt_fetcher_test.go index cfff7fa156..52cfa6fb8e 100644 --- a/mel-replay/receipt_recorder_and_receipt_fetcher_test.go +++ b/mel-replay/receipt_recorder_and_receipt_fetcher_test.go @@ -7,7 +7,6 @@ import ( "context" "fmt" "math/big" - "strings" "testing" "github.com/stretchr/testify/require" @@ -32,7 +31,8 @@ func TestRecordingOfReceiptPreimagesAndFetchingLogsFromPreimages(t *testing.T) { blockHeader := &types.Header{} receipts := []*types.Receipt{} txs := make([]*types.Transaction, 0) - for i := range uint64(50) { + numTxs := uint64(50) + for i := range numTxs { txData := &types.DynamicFeeTx{ To: &toAddr, Nonce: i, @@ -74,15 +74,12 @@ func TestRecordingOfReceiptPreimagesAndFetchingLogsFromPreimages(t *testing.T) { } } preimages := make(daprovider.PreimagesMap) - recorder, err := melrecording.NewReceiptRecorder(blockReader, block.Hash(), preimages) + recordedLogsFetcher, err := melrecording.RecordReceipts(ctx, blockReader, block.Hash(), preimages) require.NoError(t, err) - require.NoError(t, recorder.Initialize(ctx)) // Test recording of preimages - recordStart := uint(6) - recordEnd := uint(20) - for i := recordStart; i <= recordEnd; i++ { - logs, err := recorder.LogsForTxIndex(ctx, block.Hash(), i) + for i := range numTxs { + logs, err := recordedLogsFetcher.LogsForTxIndex(ctx, block.Hash(), uint(i)) require.NoError(t, err) have, err := logs[0].MarshalJSON() require.NoError(t, err) @@ -92,8 +89,6 @@ func TestRecordingOfReceiptPreimagesAndFetchingLogsFromPreimages(t *testing.T) { } // Test reading of logs from the recorded preimages - require.NoError(t, recorder.CollectTxIndicesPreimage()) - require.NoError(t, err) receiptFetcher := melreplay.NewLogsFetcher( block.Header(), melreplay.NewTypeBasedPreimageResolver( @@ -105,8 +100,8 @@ func TestRecordingOfReceiptPreimagesAndFetchingLogsFromPreimages(t *testing.T) { logs, err := receiptFetcher.LogsForBlockHash(ctx, block.Hash()) require.NoError(t, err) // #nosec G115 - if len(logs) != int(recordEnd-recordStart+1) { - t.Fatalf("number of logs from LogsForBlockHash mismatch. Want: %d, Got: %d", recordEnd-recordStart+1, len(logs)) + if len(logs) != int(numTxs) { + t.Fatalf("number of logs from LogsForBlockHash mismatch. Want: %d, Got: %d", numTxs, len(logs)) } for _, log := range logs { have, err := log.MarshalJSON() @@ -116,8 +111,8 @@ func TestRecordingOfReceiptPreimagesAndFetchingLogsFromPreimages(t *testing.T) { require.Equal(t, want, have) } // Test LogsForTxIndex - for i := recordStart; i <= recordEnd; i++ { - logs, err := receiptFetcher.LogsForTxIndex(ctx, block.Hash(), i) + for i := range numTxs { + logs, err := receiptFetcher.LogsForTxIndex(ctx, block.Hash(), uint(i)) require.NoError(t, err) have, err := logs[0].MarshalJSON() require.NoError(t, err) @@ -125,10 +120,4 @@ func TestRecordingOfReceiptPreimagesAndFetchingLogsFromPreimages(t *testing.T) { require.NoError(t, err) require.Equal(t, want, have) } - - // Logs fetching should fail for not recorded ones - _, err = receiptFetcher.LogsForTxIndex(ctx, block.Hash(), recordStart-1) - if err == nil || !strings.Contains(err.Error(), "preimage not found for hash") { - t.Fatalf("failed with unexpected error: %v", err) - } } diff --git a/melwavmio/stub.go b/melwavmio/stub.go index cecac79c68..c6d280a73d 100644 --- a/melwavmio/stub.go +++ b/melwavmio/stub.go @@ -24,15 +24,18 @@ var ( melMsgHash = common.Hash{} endMelStateHash = common.Hash{} // This is set by the stubbed SetEndMELStateHash function endParentChainBlockHash = common.Hash{} // This is set by the stubbed GetEndParentChainBlockHash function + positionInMEL = uint64(0) ) func StubInit() { endParentChainBlockHashFlag := flag.String("end-parent-chain-block-hash", "0000000000000000000000000000000000000000000000000000000000000000", "endParentChainBlockHash") startMelRootFlag := flag.String("start-mel-state-hash", "0000000000000000000000000000000000000000000000000000000000000000", "startMelHash") preimagesPath := flag.String("preimages", "", "file to load preimages from") + positionInMELFlag := flag.Uint64("position-in-mel", 0, "positionInMEL") flag.Parse() endParentChainBlockHash = common.HexToHash(*endParentChainBlockHashFlag) startMelStateHash = common.HexToHash(*startMelRootFlag) + positionInMEL = *positionInMELFlag fileBytes, err := os.ReadFile(*preimagesPath) if err != nil { panic(err) @@ -70,10 +73,11 @@ func SetEndMELRoot(hash common.Hash) { } func GetPositionInMEL() uint64 { - return 0 + return positionInMEL } func IncreasePositionInMEL() { + positionInMEL++ } func ResolveTypedPreimage(ty arbutil.PreimageType, hash common.Hash) ([]byte, error) { diff --git a/staker/mel_validator.go b/staker/mel_validator.go index b095a10110..e6d28867d8 100644 --- a/staker/mel_validator.go +++ b/staker/mel_validator.go @@ -353,6 +353,9 @@ func (mv *MELValidator) CreateNextValidationEntry(ctx context.Context, lastValid if err != nil { return nil, 0, err } + if err := currentState.RecordMsgPreimagesTo(preimages); err != nil { + return nil, 0, err + } var endState *mel.State for i := lastValidatedParentChainBlock + 1; ; i++ { header, err := mv.l1Client.HeaderByNumber(ctx, new(big.Int).SetUint64(i)) @@ -371,14 +374,11 @@ func (mv *MELValidator) CreateNextValidationEntry(ctx context.Context, lastValid if err := txsRecorder.Initialize(ctx); err != nil { return nil, 0, err } - receiptsRecorder, err := melrecording.NewReceiptRecorder(mv.l1Client, header.Hash(), preimages) + recordedLogsFetcher, err := melrecording.RecordReceipts(ctx, mv.l1Client, header.Hash(), preimages) if err != nil { return nil, 0, err } - if err := receiptsRecorder.Initialize(ctx); err != nil { - return nil, 0, err - } - endState, _, _, _, err = melextraction.ExtractMessages(ctx, currentState, header, recordingDAPReaders, delayedMsgRecordingDB, txsRecorder, receiptsRecorder, nil) + endState, _, _, _, err = melextraction.ExtractMessages(ctx, currentState, header, recordingDAPReaders, delayedMsgRecordingDB, txsRecorder, recordedLogsFetcher, nil) if err != nil { return nil, 0, fmt.Errorf("error calling melextraction.ExtractMessages in recording mode: %w", err) } @@ -389,9 +389,6 @@ func (mv *MELValidator) CreateNextValidationEntry(ctx context.Context, lastValid if endState.Hash() != wantState.Hash() { return nil, 0, fmt.Errorf("calculated MEL state hash in recording mode doesn't match the one computed in native mode, parentchainBlocknumber: %d", i) } - if err := receiptsRecorder.CollectTxIndicesPreimage(); err != nil { - return nil, 0, err - } if endState.MsgCount >= toValidateMsgExtractionCount { break } @@ -405,7 +402,7 @@ func (mv *MELValidator) CreateNextValidationEntry(ctx context.Context, lastValid MELStateHash: initialState.Hash(), MELMsgHash: common.Hash{}, Batch: 0, - PosInBatch: 0, + PosInBatch: initialState.MsgCount, }, End: validator.GoGlobalState{ BlockHash: common.Hash{}, diff --git a/system_tests/message_extraction_layer_validation_test.go b/system_tests/message_extraction_layer_validation_test.go index 7ab4280207..309d371118 100644 --- a/system_tests/message_extraction_layer_validation_test.go +++ b/system_tests/message_extraction_layer_validation_test.go @@ -11,10 +11,10 @@ import ( "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/crypto/kzg4844" - melextraction "github.com/offchainlabs/nitro/arbnode/mel/extraction" + "github.com/offchainlabs/nitro/arbnode/mel/extraction" "github.com/offchainlabs/nitro/arbutil" "github.com/offchainlabs/nitro/daprovider" - melreplay "github.com/offchainlabs/nitro/mel-replay" + "github.com/offchainlabs/nitro/mel-replay" "github.com/offchainlabs/nitro/staker" "github.com/offchainlabs/nitro/validator" "github.com/offchainlabs/nitro/validator/server_arb" @@ -71,7 +71,16 @@ func TestMELValidator_Recording_RunsUnifiedReplayBinary(t *testing.T) { Require(t, err) t.Log(entry.Preimages) - locator, err := server_common.NewMachineLocator("target/machines") + // jsonPreimages, err := json.Marshal(entry.Preimages) + // Require(t, err) + // Require(t, os.WriteFile("/tmp/mypreimages.json", jsonPreimages, os.ModePerm)) + // t.Log("MELStateHash", entry.Start.MELStateHash.Hex()) + // t.Log("EndParentChainBlockHash", entry.EndParentChainBlockHash.Hex()) + // initialMELState, err := builder.L2.ConsensusNode.MessageExtractor.GetState(ctx, startBlock) + // Require(t, err) + // t.Log("PositionInMEL", initialMELState.MsgCount) // Because we only recorded preimages for starting from this l2 block + + locator, err := server_common.NewMachineLocator(builder.valnodeConfig.Wasm.RootPath) Require(t, err) arbConfigFetcher := func() *server_arb.ArbitratorSpawnerConfig { return &server_arb.DefaultArbitratorSpawnerConfig @@ -79,7 +88,7 @@ func TestMELValidator_Recording_RunsUnifiedReplayBinary(t *testing.T) { arbSpawner, err := server_arb.NewArbitratorSpawner(locator, arbConfigFetcher) Require(t, err) Require(t, arbSpawner.Start(ctx)) - wasmModuleRoot := common.HexToHash("0x5bb0a3fcc8a1f7cabda7489b005d04291ab30603551976b528c365a025d24092") + wasmModuleRoot := common.HexToHash("0x680289b92d5944850d04919b2c7c0076b43944d210368066651008c7aaadc908") execRunPromise := arbSpawner.CreateExecutionRun( wasmModuleRoot, &validator.ValidationInput{