From 877cf11c00b134e85b25eaaea8b48b4b6d1cecb0 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Mon, 20 Mar 2023 16:07:12 +0200 Subject: [PATCH 01/34] improve separation of arb_interface --- arbnode/inbox_tracker.go | 43 +++++++++ arbnode/node.go | 15 +-- arbnode/transaction_streamer.go | 8 ++ cmd/nitro/nitro.go | 2 +- execution/gethexec/arb_interface.go | 17 ++-- .../gethexec}/classicMessage.go | 2 +- execution/gethexec/executionengine.go | 4 + execution/gethexec/node.go | 23 ++++- execution/interface.go | 2 + .../nodeInterface}/NodeInterface.go | 95 ++++++------------- .../nodeInterface}/NodeInterfaceDebug.go | 0 .../nodeInterface}/virtual-contracts.go | 8 +- staker/l1_validator.go | 2 +- staker/stateless_block_validator.go | 40 +------- system_tests/common_test.go | 2 +- 15 files changed, 129 insertions(+), 134 deletions(-) rename {arbnode => execution/gethexec}/classicMessage.go (99%) rename {nodeInterface => execution/nodeInterface}/NodeInterface.go (87%) rename {nodeInterface => execution/nodeInterface}/NodeInterfaceDebug.go (100%) rename {nodeInterface => execution/nodeInterface}/virtual-contracts.go (96%) diff --git a/arbnode/inbox_tracker.go b/arbnode/inbox_tracker.go index 38bf83bff5..efc38485b8 100644 --- a/arbnode/inbox_tracker.go +++ b/arbnode/inbox_tracker.go @@ -182,6 +182,11 @@ func (t *InboxTracker) GetBatchMessageCount(seqNum uint64) (arbutil.MessageIndex return metadata.MessageCount, err } +func (t *InboxTracker) GetBatchL1Block(seqNum uint64) (uint64, error) { + metadata, err := t.GetBatchMetadata(seqNum) + return metadata.L1Block, err +} + // GetBatchAcc is a convenience function wrapping GetBatchMetadata func (t *InboxTracker) GetBatchAcc(seqNum uint64) (common.Hash, error) { metadata, err := t.GetBatchMetadata(seqNum) @@ -201,6 +206,44 @@ func (t *InboxTracker) GetBatchCount() (uint64, error) { return count, nil } +func (t *InboxTracker) FindL1BatchForMessage(pos arbutil.MessageIndex) (uint64, error) { + batchCount, err := t.GetBatchCount() + if err != nil { + return 0, err + } + low := uint64(0) + high := batchCount - 1 + // Iteration preconditions: + // - high >= low + // - msgCount(low - 1) <= pos implies low <= target + // - msgCount(high) > pos implies high >= target + // Therefore, if low == high, then low == high == target + for { + // Due to integer rounding, mid >= low && mid < high + mid := (low + high) / 2 + count, err := t.GetBatchMessageCount(mid) + if err != nil { + return 0, err + } + if count < pos { + // Must narrow as mid >= low, therefore mid + 1 > low, therefore newLow > oldLow + // Keeps low precondition as msgCount(mid) < pos + low = mid + 1 + } else if count == pos { + return mid + 1, err + } else if count == pos+1 || mid == low { // implied: count > pos + return mid, nil + } else { // implied: count > pos + 1 + // Must narrow as mid < high, therefore newHigh < lowHigh + // Keeps high precondition as msgCount(mid) > pos + high = mid + } + if high == low { + return high, err + } + } +} + func (t *InboxTracker) populateFeedBacklog(broadcastServer *broadcaster.Broadcaster) error { batchCount, err := t.GetBatchCount() if err != nil { diff --git a/arbnode/node.go b/arbnode/node.go index 822cbcbccb..472b2e5844 100644 --- a/arbnode/node.go +++ b/arbnode/node.go @@ -537,7 +537,6 @@ type Node struct { SeqCoordinator *SeqCoordinator MaintenanceRunner *MaintenanceRunner DASLifecycleManager *das.LifecycleManager - ClassicOutboxRetriever *ClassicOutboxRetriever SyncMonitor *SyncMonitor configFetcher ConfigFetcher ctx context.Context @@ -617,16 +616,6 @@ func createNodeImpl( // config.Dangerous.ReorgToBlock >= 0 { syncMonitor := NewSyncMonitor(&config.SyncMonitor) - var classicOutbox *ClassicOutboxRetriever - classicMsgDb, err := stack.OpenDatabase("classic-msg", 0, 0, "", true) - if err != nil { - if l2Config.ArbitrumChainParams.GenesisBlockNum > 0 { - log.Warn("Classic Msg Database not found", "err", err) - } - classicOutbox = nil - } else { - classicOutbox = NewClassicOutboxRetriever(classicMsgDb) - } var l1Reader *headerreader.HeaderReader if config.L1Reader.Enable { @@ -717,7 +706,6 @@ func createNodeImpl( coordinator, maintenanceRunner, nil, - classicOutbox, syncMonitor, configFetcher, ctx, @@ -894,7 +882,6 @@ func createNodeImpl( coordinator, maintenanceRunner, dasLifecycleManager, - classicOutbox, syncMonitor, configFetcher, ctx, @@ -954,7 +941,7 @@ func (n *Node) Start(ctx context.Context) error { execClient = nil } if execClient != nil { - err := execClient.Initialize(ctx, n, n.SyncMonitor) + err := execClient.Initialize(ctx, n.SyncMonitor) if err != nil { return fmt.Errorf("error initializing exec client: %w", err) } diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index 6773f4d821..f9cf7f9b56 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -752,6 +752,14 @@ func (s *TransactionStreamer) FetchBatch(batchNum uint64) ([]byte, error) { return s.inboxReader.GetSequencerMessageBytes(context.TODO(), batchNum) } +func (s *TransactionStreamer) FindL1BatchForMessage(pos arbutil.MessageIndex) (uint64, error) { + return s.inboxReader.tracker.FindL1BatchForMessage(pos) +} + +func (s *TransactionStreamer) GetBatchL1Block(seqNum uint64) (uint64, error) { + return s.inboxReader.tracker.GetBatchL1Block(seqNum) +} + // The caller must hold the insertionMutex func (s *TransactionStreamer) ExpectChosenSequencer() error { if s.coordinator != nil { diff --git a/cmd/nitro/nitro.go b/cmd/nitro/nitro.go index ec827c61d8..2027c101b1 100644 --- a/cmd/nitro/nitro.go +++ b/cmd/nitro/nitro.go @@ -50,7 +50,7 @@ import ( "github.com/offchainlabs/nitro/cmd/util" "github.com/offchainlabs/nitro/cmd/util/confighelpers" "github.com/offchainlabs/nitro/execution/gethexec" - _ "github.com/offchainlabs/nitro/nodeInterface" + _ "github.com/offchainlabs/nitro/execution/nodeInterface" "github.com/offchainlabs/nitro/staker" "github.com/offchainlabs/nitro/util/colors" "github.com/offchainlabs/nitro/util/headerreader" diff --git a/execution/gethexec/arb_interface.go b/execution/gethexec/arb_interface.go index 049dc4e240..1d07012528 100644 --- a/execution/gethexec/arb_interface.go +++ b/execution/gethexec/arb_interface.go @@ -20,30 +20,31 @@ type TransactionPublisher interface { } type ArbInterface struct { - exec *ExecutionEngine + blockchain *core.BlockChain + node *ExecutionNode txPublisher TransactionPublisher - arbNode interface{} } -func NewArbInterface(exec *ExecutionEngine, txPublisher TransactionPublisher) (*ArbInterface, error) { +func NewArbInterface(blockchain *core.BlockChain, txPublisher TransactionPublisher) (*ArbInterface, error) { return &ArbInterface{ - exec: exec, + blockchain: blockchain, txPublisher: txPublisher, }, nil } -func (a *ArbInterface) Initialize(arbnode interface{}) { - a.arbNode = arbnode +func (a *ArbInterface) Initialize(node *ExecutionNode) { + a.node = node } func (a *ArbInterface) PublishTransaction(ctx context.Context, tx *types.Transaction) error { return a.txPublisher.PublishTransaction(ctx, tx) } +// might be used before Initialize func (a *ArbInterface) BlockChain() *core.BlockChain { - return a.exec.bc + return a.blockchain } func (a *ArbInterface) ArbNode() interface{} { - return a.arbNode + return a.node } diff --git a/arbnode/classicMessage.go b/execution/gethexec/classicMessage.go similarity index 99% rename from arbnode/classicMessage.go rename to execution/gethexec/classicMessage.go index f03ef5bd45..df749b98b4 100644 --- a/arbnode/classicMessage.go +++ b/execution/gethexec/classicMessage.go @@ -1,7 +1,7 @@ // Copyright 2022, Offchain Labs, Inc. // For license information, see https://github.com/nitro/blob/master/LICENSE -package arbnode +package gethexec import ( "encoding/binary" diff --git a/execution/gethexec/executionengine.go b/execution/gethexec/executionengine.go index 07a54d0757..e025e5e607 100644 --- a/execution/gethexec/executionengine.go +++ b/execution/gethexec/executionengine.go @@ -81,6 +81,10 @@ func (s *ExecutionEngine) SetTransactionStreamer(streamer execution.TransactionS s.streamer = streamer } +func (s *ExecutionEngine) GetBatchFetcher() execution.BatchFetcher { + return s.streamer +} + func (s *ExecutionEngine) Reorg(count arbutil.MessageIndex, newMessages []arbostypes.MessageWithMetadata, oldMessages []*arbostypes.MessageWithMetadata) error { if count == 0 { return errors.New("cannot reorg out genesis") diff --git a/execution/gethexec/node.go b/execution/gethexec/node.go index 8114f9c623..87d7e3925c 100644 --- a/execution/gethexec/node.go +++ b/execution/gethexec/node.go @@ -116,6 +116,8 @@ type ExecutionNode struct { Sequencer *Sequencer // either nil or same as TxPublisher TxPublisher TransactionPublisher ConfigFetcher ConfigFetcher + L1Reader *headerreader.HeaderReader + ClassicOutbox *ClassicOutboxRetriever } func CreateExecutionNode( @@ -162,7 +164,7 @@ func CreateExecutionNode( strictnessFetcher := func() uint { return configFetcher().TxPreCheckerStrictness } txPublisher = NewTxPreChecker(txPublisher, l2BlockChain, strictnessFetcher) - arbInterface, err := NewArbInterface(execEngine, txPublisher) + arbInterface, err := NewArbInterface(l2BlockChain, txPublisher) if err != nil { return nil, err } @@ -175,6 +177,17 @@ func CreateExecutionNode( return nil, err } + var classicOutbox *ClassicOutboxRetriever + classicMsgDb, err := stack.OpenDatabase("classic-msg", 0, 0, "", true) + if err != nil { + if l2BlockChain.Config().ArbitrumChainParams.GenesisBlockNum > 0 { + log.Warn("Classic Msg Database not found", "err", err) + } + classicOutbox = nil + } else { + classicOutbox = NewClassicOutboxRetriever(classicMsgDb) + } + apis := []rpc.API{{ Namespace: "arb", Version: "1.0", @@ -218,11 +231,13 @@ func CreateExecutionNode( sequencer, txPublisher, configFetcher, + l1Reader, + classicOutbox, }, nil } -func (n *ExecutionNode) Initialize(ctx context.Context, arbnode interface{}, sync arbitrum.SyncProgressBackend) error { +func (n *ExecutionNode) Initialize(ctx context.Context, sync arbitrum.SyncProgressBackend) error { n.ArbInterface.Initialize(n) err := n.Backend.Start() if err != nil { @@ -320,15 +335,19 @@ func (n *ExecutionNode) PrepareForRecord(ctx context.Context, start, end arbutil func (n *ExecutionNode) Pause() { n.Sequencer.Pause() } + func (n *ExecutionNode) Activate() { n.Sequencer.Activate() } + func (n *ExecutionNode) ForwardTo(url string) error { return n.Sequencer.ForwardTo(url) } + func (n *ExecutionNode) SetTransactionStreamer(streamer execution.TransactionStreamer) { n.ExecEngine.SetTransactionStreamer(streamer) } + func (n *ExecutionNode) MessageIndexToBlockNumber(messageNum arbutil.MessageIndex) uint64 { return n.ExecEngine.MessageIndexToBlockNumber(messageNum) } diff --git a/execution/interface.go b/execution/interface.go index dc1551d8cf..b1de97e2ba 100644 --- a/execution/interface.go +++ b/execution/interface.go @@ -71,6 +71,8 @@ type FullExecutionClient interface { // not implemented in execution, used as input type BatchFetcher interface { FetchBatch(batchNum uint64) ([]byte, error) + FindL1BatchForMessage(message arbutil.MessageIndex) (uint64, error) + GetBatchL1Block(seqNum uint64) (uint64, error) } type TransactionStreamer interface { diff --git a/nodeInterface/NodeInterface.go b/execution/nodeInterface/NodeInterface.go similarity index 87% rename from nodeInterface/NodeInterface.go rename to execution/nodeInterface/NodeInterface.go index 097f0dcca2..c0233e4d3a 100644 --- a/nodeInterface/NodeInterface.go +++ b/execution/nodeInterface/NodeInterface.go @@ -6,7 +6,6 @@ package nodeInterface import ( "context" "errors" - "fmt" "math/big" "sort" @@ -18,13 +17,10 @@ import ( "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/rpc" - "github.com/offchainlabs/nitro/arbnode" "github.com/offchainlabs/nitro/arbos" "github.com/offchainlabs/nitro/arbos/l1pricing" "github.com/offchainlabs/nitro/arbos/retryables" "github.com/offchainlabs/nitro/arbos/util" - "github.com/offchainlabs/nitro/arbutil" - "github.com/offchainlabs/nitro/staker" "github.com/offchainlabs/nitro/util/arbmath" "github.com/offchainlabs/nitro/util/merkletree" ) @@ -50,74 +46,69 @@ var merkleTopic common.Hash var l2ToL1TxTopic common.Hash var l2ToL1TransactionTopic common.Hash -var blockInGenesis = errors.New("") -var blockAfterLatestBatch = errors.New("") - func (n NodeInterface) NitroGenesisBlock(c ctx) (huge, error) { block := n.backend.ChainConfig().ArbitrumChainParams.GenesisBlockNum return arbmath.UintToBig(block), nil } func (n NodeInterface) FindBatchContainingBlock(c ctx, evm mech, blockNum uint64) (uint64, error) { - node, err := arbNodeFromNodeInterfaceBackend(n.backend) + node, err := gethExecFromNodeInterfaceBackend(n.backend) if err != nil { return 0, err } - genesis, err := node.TxStreamer.GetGenesisBlockNumber() + msgIndex, err := node.ExecEngine.BlockNumberToMessageIndex(blockNum) if err != nil { return 0, err } - return findBatchContainingBlock(node, genesis, blockNum) + fetcher := node.ExecEngine.GetBatchFetcher() + if fetcher == nil { + return 0, errors.New("batch fetcher not set") + } + batch, err := fetcher.FindL1BatchForMessage(msgIndex) + return batch, err } func (n NodeInterface) GetL1Confirmations(c ctx, evm mech, blockHash bytes32) (uint64, error) { - node, err := arbNodeFromNodeInterfaceBackend(n.backend) + node, err := gethExecFromNodeInterfaceBackend(n.backend) if err != nil { return 0, err } - if node.InboxReader == nil { - return 0, nil - } - bc, err := blockchainFromNodeInterfaceBackend(n.backend) + blockchain, err := blockchainFromNodeInterfaceBackend(n.backend) if err != nil { return 0, err } - header := bc.GetHeaderByHash(blockHash) + header := blockchain.GetHeaderByHash(blockHash) if header == nil { return 0, errors.New("unknown block hash") } - blockNum := header.Number.Uint64() - genesis, err := node.TxStreamer.GetGenesisBlockNumber() + l2BlockNum := header.Number.Uint64() + canonicalHash := blockchain.GetCanonicalHash(l2BlockNum) + if canonicalHash != header.Hash() { + return 0, errors.New("block hash is non-canonical") + } + batchNum, err := n.FindBatchContainingBlock(c, evm, l2BlockNum) if err != nil { return 0, err } - batch, err := findBatchContainingBlock(node, genesis, blockNum) + blockNum, err := node.ExecEngine.GetBatchFetcher().GetBatchL1Block(batchNum) if err != nil { - if errors.Is(err, blockInGenesis) { - batch = 0 - } else if errors.Is(err, blockAfterLatestBatch) { - return 0, nil - } else { - return 0, err - } + return 0, err } - latestL1Block, latestBatchCount := node.InboxReader.GetLastReadBlockAndBatchCount() - if latestBatchCount <= batch { - return 0, nil // batch was reorg'd out? + if node.L1Reader == nil { + return 0, nil } - meta, err := node.InboxTracker.GetBatchMetadata(batch) + latestHeader, err := node.L1Reader.LastHeaderWithError() if err != nil { return 0, err } - if latestL1Block < meta.L1Block || arbutil.BlockNumberToMessageCount(blockNum, genesis) > meta.MessageCount { - return 0, nil + if latestHeader == nil { + return 0, errors.New("no headers read from l1") } - canonicalHash := bc.GetCanonicalHash(header.Number.Uint64()) - if canonicalHash != header.Hash() { - return 0, errors.New("block hash is non-canonical") + latestBlockNum := latestHeader.Number.Uint64() + if latestBlockNum < blockNum { + return 0, nil } - confs := (latestL1Block - meta.L1Block) + 1 + node.InboxReader.GetDelayBlocks() - return confs, nil + return (latestBlockNum - blockNum), nil } func (n NodeInterface) EstimateRetryableTicket( @@ -537,42 +528,18 @@ func (n NodeInterface) GasEstimateComponents( return total, gasForL1, baseFee, l1BaseFeeEstimate, nil } -func findBatchContainingBlock(node *arbnode.Node, genesis uint64, block uint64) (uint64, error) { - if block <= genesis { - return 0, fmt.Errorf("%wblock %v is part of genesis", blockInGenesis, block) - } - pos := arbutil.BlockNumberToMessageCount(block, genesis) - 1 - high, err := node.InboxTracker.GetBatchCount() - if err != nil { - return 0, err - } - high-- - latestCount, err := node.InboxTracker.GetBatchMessageCount(high) - if err != nil { - return 0, err - } - latestBlock := arbutil.MessageCountToBlockNumber(latestCount, genesis) - if int64(block) > latestBlock { - return 0, fmt.Errorf( - "%wrequested block %v is after latest on-chain block %v published in batch %v", - blockAfterLatestBatch, block, latestBlock, high, - ) - } - return staker.FindBatchContainingMessageIndex(node.InboxTracker, pos, high) -} - func (n NodeInterface) LegacyLookupMessageBatchProof(c ctx, evm mech, batchNum huge, index uint64) ( proof []bytes32, path huge, l2Sender addr, l1Dest addr, l2Block huge, l1Block huge, timestamp huge, amount huge, calldataForL1 []byte, err error) { - node, err := arbNodeFromNodeInterfaceBackend(n.backend) + node, err := gethExecFromNodeInterfaceBackend(n.backend) if err != nil { return } - if node.ClassicOutboxRetriever == nil { + if node.ClassicOutbox == nil { err = errors.New("this node doesnt support classicLookupMessageBatchProof") return } - msg, err := node.ClassicOutboxRetriever.GetMsg(batchNum, index) + msg, err := node.ClassicOutbox.GetMsg(batchNum, index) if err != nil { return } diff --git a/nodeInterface/NodeInterfaceDebug.go b/execution/nodeInterface/NodeInterfaceDebug.go similarity index 100% rename from nodeInterface/NodeInterfaceDebug.go rename to execution/nodeInterface/NodeInterfaceDebug.go diff --git a/nodeInterface/virtual-contracts.go b/execution/nodeInterface/virtual-contracts.go similarity index 96% rename from nodeInterface/virtual-contracts.go rename to execution/nodeInterface/virtual-contracts.go index 50eb1f47bb..3cb8ad53d1 100644 --- a/nodeInterface/virtual-contracts.go +++ b/execution/nodeInterface/virtual-contracts.go @@ -15,9 +15,9 @@ import ( "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/core/vm" "github.com/ethereum/go-ethereum/log" - "github.com/offchainlabs/nitro/arbnode" "github.com/offchainlabs/nitro/arbos/arbosState" "github.com/offchainlabs/nitro/arbos/l1pricing" + "github.com/offchainlabs/nitro/execution/gethexec" "github.com/offchainlabs/nitro/gethhook" "github.com/offchainlabs/nitro/precompiles" "github.com/offchainlabs/nitro/solgen/go/node_interfacegen" @@ -170,16 +170,16 @@ func init() { merkleTopic = arbSys.Events["SendMerkleUpdate"].ID } -func arbNodeFromNodeInterfaceBackend(backend BackendAPI) (*arbnode.Node, error) { +func gethExecFromNodeInterfaceBackend(backend BackendAPI) (*gethexec.ExecutionNode, error) { apiBackend, ok := backend.(*arbitrum.APIBackend) if !ok { return nil, errors.New("API backend isn't Arbitrum") } - arbNode, ok := apiBackend.GetArbitrumNode().(*arbnode.Node) + exec, ok := apiBackend.GetArbitrumNode().(*gethexec.ExecutionNode) if !ok { return nil, errors.New("failed to get Arbitrum Node from backend") } - return arbNode, nil + return exec, nil } func blockchainFromNodeInterfaceBackend(backend BackendAPI) (*core.BlockChain, error) { diff --git a/staker/l1_validator.go b/staker/l1_validator.go index e9029bf265..cb046c7d63 100644 --- a/staker/l1_validator.go +++ b/staker/l1_validator.go @@ -302,7 +302,7 @@ func (v *L1Validator) generateNodeAction(ctx context.Context, stakerInfo *OurSta batchNum = localBatchCount - 1 validatedCount = messageCount } else { - batchNum, err = FindBatchContainingMessageIndex(v.inboxTracker, validatedCount-1, localBatchCount) + batchNum, err = v.inboxTracker.FindL1BatchForMessage(validatedCount - 1) if err != nil { return nil, false, err } diff --git a/staker/stateless_block_validator.go b/staker/stateless_block_validator.go index 9ad3f3629c..ae04d6f5d0 100644 --- a/staker/stateless_block_validator.go +++ b/staker/stateless_block_validator.go @@ -54,6 +54,7 @@ type InboxTrackerInterface interface { GetBatchMessageCount(seqNum uint64) (arbutil.MessageIndex, error) GetBatchAcc(seqNum uint64) (common.Hash, error) GetBatchCount() (uint64, error) + FindL1BatchForMessage(pos arbutil.MessageIndex) (uint64, error) } type TransactionStreamerInterface interface { @@ -110,39 +111,6 @@ func GlobalStatePositionsAtCount( return startPos, GlobalStatePosition{batch, posInBatch + 1}, nil } -func FindBatchContainingMessageIndex( - tracker InboxTrackerInterface, pos arbutil.MessageIndex, high uint64, -) (uint64, error) { - var low uint64 - // Iteration preconditions: - // - high >= low - // - msgCount(low - 1) <= pos implies low <= target - // - msgCount(high) > pos implies high >= target - // Therefore, if low == high, then low == high == target - for high > low { - // Due to integer rounding, mid >= low && mid < high - mid := (low + high) / 2 - count, err := tracker.GetBatchMessageCount(mid) - if err != nil { - return 0, err - } - if count < pos { - // Must narrow as mid >= low, therefore mid + 1 > low, therefore newLow > oldLow - // Keeps low precondition as msgCount(mid) < pos - low = mid + 1 - } else if count == pos { - return mid + 1, nil - } else if count == pos+1 || mid == low { // implied: count > pos - return mid, nil - } else { // implied: count > pos + 1 - // Must narrow as mid < high, therefore newHigh < lowHigh - // Keeps high precondition as msgCount(mid) > pos - high = mid - } - } - return low, nil -} - type ValidationEntryStage uint32 const ( @@ -331,11 +299,7 @@ func (v *StatelessBlockValidator) GlobalStatePositionsAtCount(count arbutil.Mess if count == 0 { return GlobalStatePosition{}, GlobalStatePosition{1, 0}, nil } - batchCount, err := v.inboxTracker.GetBatchCount() - if err != nil { - return GlobalStatePosition{}, GlobalStatePosition{}, err - } - batch, err := FindBatchContainingMessageIndex(v.inboxTracker, count-1, batchCount) + batch, err := v.inboxTracker.FindL1BatchForMessage(count - 1) if err != nil { return GlobalStatePosition{}, GlobalStatePosition{}, err } diff --git a/system_tests/common_test.go b/system_tests/common_test.go index dd16b2081c..5f07e9a22f 100644 --- a/system_tests/common_test.go +++ b/system_tests/common_test.go @@ -42,7 +42,7 @@ import ( "github.com/offchainlabs/nitro/arbnode" "github.com/offchainlabs/nitro/arbos" "github.com/offchainlabs/nitro/arbutil" - _ "github.com/offchainlabs/nitro/nodeInterface" + _ "github.com/offchainlabs/nitro/execution/nodeInterface" "github.com/offchainlabs/nitro/solgen/go/bridgegen" "github.com/offchainlabs/nitro/solgen/go/mocksgen" "github.com/offchainlabs/nitro/solgen/go/precompilesgen" From bf3d0f0088f37a092196a3f4201719a6b61c039a Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Mon, 20 Mar 2023 16:26:24 +0200 Subject: [PATCH 02/34] remove unused var lastBlockRead --- arbnode/inbox_reader.go | 8 ++------ arbnode/sync_monitor.go | 2 +- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/arbnode/inbox_reader.go b/arbnode/inbox_reader.go index 7c06467d29..99a2c08d4b 100644 --- a/arbnode/inbox_reader.go +++ b/arbnode/inbox_reader.go @@ -93,7 +93,6 @@ type InboxReader struct { // Behind the mutex lastReadMutex sync.RWMutex - lastReadBlock uint64 lastReadBatchCount uint64 } @@ -348,7 +347,6 @@ func (r *InboxReader) run(ctx context.Context, hadError bool) error { from = arbmath.BigAddByUint(currentHeight, 1) blocksToFetch = config.DefaultBlocksToRead r.lastReadMutex.Lock() - r.lastReadBlock = currentHeight.Uint64() r.lastReadBatchCount = checkingBatchCount r.lastReadMutex.Unlock() storeSeenBatchCount() @@ -483,7 +481,6 @@ func (r *InboxReader) run(ctx context.Context, hadError bool) error { if len(sequencerBatches) > 0 { readAnyBatches = true r.lastReadMutex.Lock() - r.lastReadBlock = to.Uint64() r.lastReadBatchCount = sequencerBatches[len(sequencerBatches)-1].SequenceNumber + 1 r.lastReadMutex.Unlock() storeSeenBatchCount() @@ -513,7 +510,6 @@ func (r *InboxReader) run(ctx context.Context, hadError bool) error { if !readAnyBatches { r.lastReadMutex.Lock() - r.lastReadBlock = currentHeight.Uint64() r.lastReadBatchCount = checkingBatchCount r.lastReadMutex.Unlock() storeSeenBatchCount() @@ -585,10 +581,10 @@ func (r *InboxReader) GetSequencerMessageBytes(ctx context.Context, seqNum uint6 return nil, fmt.Errorf("sequencer batch %v not found in L1 block %v (found batches %v)", seqNum, metadata.L1Block, seenBatches) } -func (r *InboxReader) GetLastReadBlockAndBatchCount() (uint64, uint64) { +func (r *InboxReader) GetLastReadBatchCount() uint64 { r.lastReadMutex.RLock() defer r.lastReadMutex.RUnlock() - return r.lastReadBlock, r.lastReadBatchCount + return r.lastReadBatchCount } // GetLastSeenBatchCount returns how many sequencer batches the inbox reader has read in from L1. diff --git a/arbnode/sync_monitor.go b/arbnode/sync_monitor.go index bd9b24529c..6f0cee1882 100644 --- a/arbnode/sync_monitor.go +++ b/arbnode/sync_monitor.go @@ -90,7 +90,7 @@ func (s *SyncMonitor) SyncProgressMap() map[string]interface{} { if s.inboxReader != nil { batchSeen := s.inboxReader.GetLastSeenBatchCount() - _, batchProcessed := s.inboxReader.GetLastReadBlockAndBatchCount() + batchProcessed := s.inboxReader.GetLastReadBatchCount() if (batchSeen == 0) || // error or not yet read inbox (batchProcessed < batchSeen) { // unprocessed inbox messages From ad926edd464c8bb2f5006e3e45b510aad304c1da Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Mon, 20 Mar 2023 16:27:28 +0200 Subject: [PATCH 03/34] remove unused code --- arbnode/transaction_streamer.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index f9cf7f9b56..8b8e5fbf41 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -807,10 +807,6 @@ func (s *TransactionStreamer) WriteMessageFromSequencer(pos arbutil.MessageIndex return nil } -func (s *TransactionStreamer) GetGenesisBlockNumber() (uint64, error) { - return s.chainConfig.ArbitrumChainParams.GenesisBlockNum, nil -} - // PauseReorgs until a matching call to ResumeReorgs (may be called concurrently) func (s *TransactionStreamer) PauseReorgs() { s.reorgMutex.RLock() From a96898a579721cea37666d4c41be586155c733af Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Tue, 21 Mar 2023 22:11:43 +0200 Subject: [PATCH 04/34] readLastBatchCount atomic instead of mutex --- arbnode/inbox_reader.go | 20 ++++---------------- 1 file changed, 4 insertions(+), 16 deletions(-) diff --git a/arbnode/inbox_reader.go b/arbnode/inbox_reader.go index 99a2c08d4b..1130aaccc2 100644 --- a/arbnode/inbox_reader.go +++ b/arbnode/inbox_reader.go @@ -10,7 +10,6 @@ import ( "math" "math/big" "strings" - "sync" "sync/atomic" "time" @@ -90,9 +89,6 @@ type InboxReader struct { // Atomic lastSeenBatchCount uint64 - - // Behind the mutex - lastReadMutex sync.RWMutex lastReadBatchCount uint64 } @@ -346,9 +342,7 @@ func (r *InboxReader) run(ctx context.Context, hadError bool) error { // There's nothing to do from = arbmath.BigAddByUint(currentHeight, 1) blocksToFetch = config.DefaultBlocksToRead - r.lastReadMutex.Lock() - r.lastReadBatchCount = checkingBatchCount - r.lastReadMutex.Unlock() + atomic.StoreUint64(&r.lastReadBatchCount, checkingBatchCount) storeSeenBatchCount() if !r.caughtUp { r.caughtUp = true @@ -480,9 +474,7 @@ func (r *InboxReader) run(ctx context.Context, hadError bool) error { } if len(sequencerBatches) > 0 { readAnyBatches = true - r.lastReadMutex.Lock() - r.lastReadBatchCount = sequencerBatches[len(sequencerBatches)-1].SequenceNumber + 1 - r.lastReadMutex.Unlock() + atomic.StoreUint64(&r.lastReadBatchCount, sequencerBatches[len(sequencerBatches)-1].SequenceNumber+1) storeSeenBatchCount() } } @@ -509,9 +501,7 @@ func (r *InboxReader) run(ctx context.Context, hadError bool) error { } if !readAnyBatches { - r.lastReadMutex.Lock() - r.lastReadBatchCount = checkingBatchCount - r.lastReadMutex.Unlock() + atomic.StoreUint64(&r.lastReadBatchCount, checkingBatchCount) storeSeenBatchCount() } } @@ -582,9 +572,7 @@ func (r *InboxReader) GetSequencerMessageBytes(ctx context.Context, seqNum uint6 } func (r *InboxReader) GetLastReadBatchCount() uint64 { - r.lastReadMutex.RLock() - defer r.lastReadMutex.RUnlock() - return r.lastReadBatchCount + return atomic.LoadUint64(&r.lastReadBatchCount) } // GetLastSeenBatchCount returns how many sequencer batches the inbox reader has read in from L1. From 45ffab0c18141532d3bb359d1a48843027aeb245 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Tue, 21 Mar 2023 22:14:34 +0200 Subject: [PATCH 05/34] split sync_monitor between consensus and execution --- arbnode/node.go | 49 ++++++- arbnode/sync_monitor.go | 195 +++++++++++++++----------- arbnode/transaction_streamer.go | 28 ++-- execution/gethexec/block_recorder.go | 2 +- execution/gethexec/executionengine.go | 28 ++-- execution/gethexec/node.go | 27 ++-- execution/gethexec/sequencer.go | 2 +- execution/gethexec/sync_monitor.go | 70 +++++++++ execution/interface.go | 25 +++- 9 files changed, 297 insertions(+), 129 deletions(-) create mode 100644 execution/gethexec/sync_monitor.go diff --git a/arbnode/node.go b/arbnode/node.go index 472b2e5844..391f840492 100644 --- a/arbnode/node.go +++ b/arbnode/node.go @@ -24,6 +24,7 @@ import ( "github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/rpc" + "github.com/offchainlabs/nitro/arbos/arbostypes" "github.com/offchainlabs/nitro/arbutil" "github.com/offchainlabs/nitro/broadcastclient" "github.com/offchainlabs/nitro/broadcastclients" @@ -614,8 +615,10 @@ func createNodeImpl( //TODO: // var reorgingToBlock *types.Block // config.Dangerous.ReorgToBlock >= 0 { - - syncMonitor := NewSyncMonitor(&config.SyncMonitor) + syncConfigFetcher := func() *SyncMonitorConfig { + return &configFetcher.Get().SyncMonitor + } + syncMonitor := NewSyncMonitor(syncConfigFetcher) var l1Reader *headerreader.HeaderReader if config.L1Reader.Enable { @@ -941,17 +944,18 @@ func (n *Node) Start(ctx context.Context) error { execClient = nil } if execClient != nil { - err := execClient.Initialize(ctx, n.SyncMonitor) + err := execClient.Initialize(ctx) if err != nil { return fmt.Errorf("error initializing exec client: %w", err) } } - n.SyncMonitor.Initialize(n.InboxReader, n.TxStreamer, n.SeqCoordinator, n.Execution) + n.SyncMonitor.Initialize(n.InboxReader, n.TxStreamer, n.SeqCoordinator) err := n.Stack.Start() if err != nil { return fmt.Errorf("error starting geth stack: %w", err) } if execClient != nil { + execClient.SetConsensusClient(n) err := execClient.Start(ctx) if err != nil { return fmt.Errorf("error starting exec client: %w", err) @@ -1113,3 +1117,40 @@ func (n *Node) StopAndWait() { log.Error("error on stak close", "err", err) } } + +func (n *Node) FetchBatch(ctx context.Context, batchNum uint64) ([]byte, error) { + return n.InboxReader.GetSequencerMessageBytes(ctx, batchNum) +} + +func (n *Node) FindL1BatchForMessage(message arbutil.MessageIndex) (uint64, error) { + return n.InboxTracker.FindL1BatchForMessage(message) +} + +func (n *Node) GetBatchL1Block(seqNum uint64) (uint64, error) { + return n.InboxTracker.GetBatchL1Block(seqNum) +} + +func (n *Node) SyncProgressMap() map[string]interface{} { + return n.SyncMonitor.SyncProgressMap() +} + +func (n *Node) GetDelayedMaxMessageCount() arbutil.MessageIndex { + return n.SyncMonitor.GetDelayedMaxMessageCount() +} + +// TODO: switch from pulling to pushing safe/finalized +func (n *Node) GetSafeMsgCount(ctx context.Context) (arbutil.MessageIndex, error) { + return n.InboxReader.GetSafeMsgCount(ctx) +} + +func (n *Node) GetFinalizedMsgCount(ctx context.Context) (arbutil.MessageIndex, error) { + return n.InboxReader.GetFinalizedMsgCount(ctx) +} + +func (n *Node) WriteMessageFromSequencer(pos arbutil.MessageIndex, msgWithMeta arbostypes.MessageWithMetadata) error { + return n.TxStreamer.WriteMessageFromSequencer(pos, msgWithMeta) +} + +func (n *Node) ExpectChosenSequencer() error { + return n.TxStreamer.ExpectChosenSequencer() +} diff --git a/arbnode/sync_monitor.go b/arbnode/sync_monitor.go index 6f0cee1882..a88aa9678f 100644 --- a/arbnode/sync_monitor.go +++ b/arbnode/sync_monitor.go @@ -2,112 +2,146 @@ package arbnode import ( "context" - "errors" - "sync/atomic" + "sync" + "time" + "github.com/ethereum/go-ethereum/log" "github.com/offchainlabs/nitro/arbutil" - "github.com/offchainlabs/nitro/execution" + "github.com/offchainlabs/nitro/util/stopwaiter" flag "github.com/spf13/pflag" ) type SyncMonitor struct { - config *SyncMonitorConfig + stopwaiter.StopWaiter + config func() *SyncMonitorConfig inboxReader *InboxReader txStreamer *TransactionStreamer coordinator *SeqCoordinator - exec execution.FullExecutionClient initialized bool + + maxMsgLock sync.Mutex + lastMaxMessageCount arbutil.MessageIndex + prevMaxMessageCount arbutil.MessageIndex } -func NewSyncMonitor(config *SyncMonitorConfig) *SyncMonitor { +func NewSyncMonitor(config func() *SyncMonitorConfig) *SyncMonitor { return &SyncMonitor{ config: config, } } type SyncMonitorConfig struct { - BlockBuildLag uint64 `koanf:"block-build-lag"` - BlockBuildSequencerInboxLag uint64 `koanf:"block-build-sequencer-inbox-lag"` - CoordinatorMsgLag uint64 `koanf:"coordinator-msg-lag"` + MsgLag time.Duration `koanf:"msg-lag"` } var DefaultSyncMonitorConfig = SyncMonitorConfig{ - BlockBuildLag: 20, - BlockBuildSequencerInboxLag: 0, - CoordinatorMsgLag: 15, + MsgLag: time.Second, } func SyncMonitorConfigAddOptions(prefix string, f *flag.FlagSet) { - f.Uint64(prefix+".block-build-lag", DefaultSyncMonitorConfig.BlockBuildLag, "allowed lag between messages read and blocks built") - f.Uint64(prefix+".block-build-sequencer-inbox-lag", DefaultSyncMonitorConfig.BlockBuildSequencerInboxLag, "allowed lag between messages read from sequencer inbox and blocks built") - f.Uint64(prefix+".coordinator-msg-lag", DefaultSyncMonitorConfig.CoordinatorMsgLag, "allowed lag between local and remote messages") + f.Duration(prefix+".msg-lag", DefaultSyncMonitorConfig.MsgLag, "allowed msg lag while still considered in sync") } -func (s *SyncMonitor) Initialize(inboxReader *InboxReader, txStreamer *TransactionStreamer, coordinator *SeqCoordinator, exec execution.FullExecutionClient) { +func (s *SyncMonitor) Initialize(inboxReader *InboxReader, txStreamer *TransactionStreamer, coordinator *SeqCoordinator) { s.inboxReader = inboxReader s.txStreamer = txStreamer s.coordinator = coordinator - s.exec = exec s.initialized = true } +func (s *SyncMonitor) updateDelayedMaxMessageCount(ctx context.Context) time.Duration { + maxMsg, err := s.maxMessageCount() + if err != nil { + log.Warn("failed readin max msg count", "err", err) + return s.config().MsgLag + } + s.maxMsgLock.Lock() + defer s.maxMsgLock.Unlock() + s.prevMaxMessageCount = s.lastMaxMessageCount + s.lastMaxMessageCount = maxMsg + return s.config().MsgLag +} + +func (s *SyncMonitor) GetDelayedMaxMessageCount() arbutil.MessageIndex { + s.maxMsgLock.Lock() + defer s.maxMsgLock.Unlock() + return s.prevMaxMessageCount +} + +func (s *SyncMonitor) maxMessageCount() (arbutil.MessageIndex, error) { + msgCount, err := s.txStreamer.GetMessageCount() + if err != nil { + return 0, err + } + + pending := s.txStreamer.FeedPendingMessageCount() + if pending > msgCount { + msgCount = pending + } + + if s.inboxReader != nil { + batchProcessed := s.inboxReader.GetLastReadBatchCount() + + if batchProcessed > 0 { + batchMsgCount, err := s.inboxReader.Tracker().GetBatchMessageCount(batchProcessed - 1) + if err != nil { + return msgCount, err + } + if batchMsgCount > msgCount { + msgCount = batchMsgCount + } + } + } + + if s.coordinator != nil { + coordinatorMessageCount, err := s.coordinator.GetRemoteMsgCount() //NOTE: this creates a remote call + if err != nil { + return msgCount, err + } + if coordinatorMessageCount > msgCount { + msgCount = coordinatorMessageCount + } + } + + return msgCount, nil +} + func (s *SyncMonitor) SyncProgressMap() map[string]interface{} { - syncing := false res := make(map[string]interface{}) - if !s.initialized { - res["err"] = "uninitialized" + if s.Synced() { return res } - broadcasterQueuedMessagesPos := atomic.LoadUint64(&(s.txStreamer.broadcasterQueuedMessagesPos)) - - if broadcasterQueuedMessagesPos != 0 { // unprocessed feed - syncing = true + if !s.initialized { + res["err"] = "uninitialized" + return res } - res["broadcasterQueuedMessagesPos"] = broadcasterQueuedMessagesPos - builtMessageCount, err := s.txStreamer.exec.HeadMessageNumber() - if err != nil { - res["blockMessageToMessageCountError"] = err.Error() - syncing = true - builtMessageCount = 0 - } else { - builtMessageCount++ - res["messageOfLastBlock"] = builtMessageCount - } + delayedMax := s.GetDelayedMaxMessageCount() + res["delayedMaxMsgCount"] = delayedMax msgCount, err := s.txStreamer.GetMessageCount() if err != nil { res["msgCountError"] = err.Error() - syncing = true - } else { - res["msgCount"] = msgCount - if builtMessageCount+arbutil.MessageIndex(s.config.BlockBuildLag) < msgCount { - syncing = true - } + return res } + res["msgCount"] = msgCount + + res["feedPendingMessageCount"] = s.txStreamer.FeedPendingMessageCount() if s.inboxReader != nil { batchSeen := s.inboxReader.GetLastSeenBatchCount() - batchProcessed := s.inboxReader.GetLastReadBatchCount() - - if (batchSeen == 0) || // error or not yet read inbox - (batchProcessed < batchSeen) { // unprocessed inbox messages - syncing = true - } res["batchSeen"] = batchSeen + + batchProcessed := s.inboxReader.GetLastReadBatchCount() res["batchProcessed"] = batchProcessed - processedMetadata, err := s.inboxReader.Tracker().GetBatchMetadata(batchProcessed - 1) + processedBatchMsgs, err := s.inboxReader.Tracker().GetBatchMessageCount(batchProcessed - 1) if err != nil { res["batchMetadataError"] = err.Error() - syncing = true } else { - res["messageOfProcessedBatch"] = processedMetadata.MessageCount - if builtMessageCount+arbutil.MessageIndex(s.config.BlockBuildSequencerInboxLag) < processedMetadata.MessageCount { - syncing = true - } + res["messageOfProcessedBatch"] = processedBatchMsgs } l1reader := s.inboxReader.l1Reader @@ -127,46 +161,47 @@ func (s *SyncMonitor) SyncProgressMap() map[string]interface{} { coordinatorMessageCount, err := s.coordinator.GetRemoteMsgCount() //NOTE: this creates a remote call if err != nil { res["coordinatorMsgCountError"] = err.Error() - syncing = true } else { res["coordinatorMessageCount"] = coordinatorMessageCount - if msgCount+arbutil.MessageIndex(s.config.CoordinatorMsgLag) < coordinatorMessageCount { - syncing = true - } } } - if !syncing { - return make(map[string]interface{}) - } - return res } -func (s *SyncMonitor) SafeBlockNumber(ctx context.Context) (uint64, error) { - if s.inboxReader == nil || !s.initialized { - return 0, errors.New("not set up for safeblock") - } - msg, err := s.inboxReader.GetSafeMsgCount(ctx) - if err != nil { - return 0, err - } - block := s.exec.MessageIndexToBlockNumber(msg - 1) - return block, nil +func (s *SyncMonitor) Start(ctx_in context.Context) { + s.StopWaiter.Start(ctx_in, s) + s.CallIteratively(s.updateDelayedMaxMessageCount) } -func (s *SyncMonitor) FinalizedBlockNumber(ctx context.Context) (uint64, error) { - if s.inboxReader == nil || !s.initialized { - return 0, errors.New("not set up for safeblock") +func (s *SyncMonitor) Synced() bool { + if !s.initialized { + return false } - msg, err := s.inboxReader.GetFinalizedMsgCount(ctx) + if !s.Started() { + return false + } + delayedMax := s.GetDelayedMaxMessageCount() + + msgCount, err := s.txStreamer.GetMessageCount() if err != nil { - return 0, err + return false } - block := s.exec.MessageIndexToBlockNumber(msg - 1) - return block, nil -} -func (s *SyncMonitor) Synced() bool { - return len(s.SyncProgressMap()) == 0 + if delayedMax > msgCount { + return false + } + + if s.inboxReader != nil { + batchSeen := s.inboxReader.GetLastSeenBatchCount() + if batchSeen == 0 { + return false + } + batchProcessed := s.inboxReader.GetLastReadBatchCount() + + if batchProcessed < batchSeen { + return false + } + } + return true } diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index 8b8e5fbf41..85d283551c 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -100,7 +100,6 @@ func NewTransactionStreamer( fatalErrChan: fatalErrChan, config: config, } - streamer.exec.SetTransactionStreamer(streamer) err := streamer.cleanupInconsistentState() if err != nil { return nil, err @@ -369,6 +368,21 @@ func (s *TransactionStreamer) AddMessages(pos arbutil.MessageIndex, messagesAreC return s.AddMessagesAndEndBatch(pos, messagesAreConfirmed, messages, nil) } +func (s *TransactionStreamer) FeedPendingMessageCount() arbutil.MessageIndex { + pos := atomic.LoadUint64(&s.broadcasterQueuedMessagesPos) + if pos == 0 { + return 0 + } + + s.insertionMutex.Lock() + defer s.insertionMutex.Unlock() + pos = atomic.LoadUint64(&s.broadcasterQueuedMessagesPos) + if pos == 0 { + return 0 + } + return arbutil.MessageIndex(pos + uint64(len(s.broadcasterQueuedMessages))) +} + func (s *TransactionStreamer) AddBroadcastMessages(feedMessages []*broadcaster.BroadcastFeedMessage) error { if len(feedMessages) == 0 { return nil @@ -748,18 +762,6 @@ func (s *TransactionStreamer) addMessagesAndEndBatchImpl(messageStartPos arbutil return nil } -func (s *TransactionStreamer) FetchBatch(batchNum uint64) ([]byte, error) { - return s.inboxReader.GetSequencerMessageBytes(context.TODO(), batchNum) -} - -func (s *TransactionStreamer) FindL1BatchForMessage(pos arbutil.MessageIndex) (uint64, error) { - return s.inboxReader.tracker.FindL1BatchForMessage(pos) -} - -func (s *TransactionStreamer) GetBatchL1Block(seqNum uint64) (uint64, error) { - return s.inboxReader.tracker.GetBatchL1Block(seqNum) -} - // The caller must hold the insertionMutex func (s *TransactionStreamer) ExpectChosenSequencer() error { if s.coordinator != nil { diff --git a/execution/gethexec/block_recorder.go b/execution/gethexec/block_recorder.go index 745dc0d2fe..513bca0833 100644 --- a/execution/gethexec/block_recorder.go +++ b/execution/gethexec/block_recorder.go @@ -119,7 +119,7 @@ func (r *BlockRecorder) RecordBlockCreation( var readBatchInfo []validator.BatchInfo if msg != nil { batchFetcher := func(batchNum uint64) ([]byte, error) { - data, err := r.execEngine.streamer.FetchBatch(batchNum) + data, err := r.execEngine.consensus.FetchBatch(ctx, batchNum) if err != nil { return nil, err } diff --git a/execution/gethexec/executionengine.go b/execution/gethexec/executionengine.go index e025e5e607..398c2a9fdf 100644 --- a/execution/gethexec/executionengine.go +++ b/execution/gethexec/executionengine.go @@ -27,9 +27,9 @@ import ( type ExecutionEngine struct { stopwaiter.StopWaiter - bc *core.BlockChain - streamer execution.TransactionStreamer - recorder *BlockRecorder + bc *core.BlockChain + consensus execution.FullConsensusClient + recorder *BlockRecorder resequenceChan chan []*arbostypes.MessageWithMetadata createBlocksMutex sync.Mutex @@ -71,18 +71,18 @@ func (s *ExecutionEngine) EnableReorgSequencing() { s.reorgSequencing = true } -func (s *ExecutionEngine) SetTransactionStreamer(streamer execution.TransactionStreamer) { +func (s *ExecutionEngine) SetTransactionStreamer(consensus execution.FullConsensusClient) { if s.Started() { - panic("trying to set transaction streamer after start") + panic("trying to set transaction consensus after start") } - if s.streamer != nil { - panic("trying to set transaction streamer when already set") + if s.consensus != nil { + panic("trying to set transaction consensus when already set") } - s.streamer = streamer + s.consensus = consensus } func (s *ExecutionEngine) GetBatchFetcher() execution.BatchFetcher { - return s.streamer + return s.consensus } func (s *ExecutionEngine) Reorg(count arbutil.MessageIndex, newMessages []arbostypes.MessageWithMetadata, oldMessages []*arbostypes.MessageWithMetadata) error { @@ -298,7 +298,7 @@ func (s *ExecutionEngine) sequenceTransactionsWithBlockMutex(header *arbostypes. return nil, err } - err = s.streamer.WriteMessageFromSequencer(pos, msgWithMeta) + err = s.consensus.WriteMessageFromSequencer(pos, msgWithMeta) if err != nil { return nil, err } @@ -344,7 +344,7 @@ func (s *ExecutionEngine) sequenceDelayedMessageWithBlockMutex(message *arbostyp DelayedMessagesRead: delayedSeqNum + 1, } - err = s.streamer.WriteMessageFromSequencer(lastMsg+1, messageWithMeta) + err = s.consensus.WriteMessageFromSequencer(lastMsg+1, messageWithMeta) if err != nil { return err } @@ -395,6 +395,10 @@ func (s *ExecutionEngine) createBlockFromNextMessage(msg *arbostypes.MessageWith statedb.StartPrefetcher("TransactionStreamer") defer statedb.StopPrefetcher() + batchFetcher := func(num uint64) ([]byte, error) { + return s.consensus.FetchBatch(s.GetContext(), num) + } + block, receipts, err := arbos.ProduceBlock( msg.Message, msg.DelayedMessagesRead, @@ -402,7 +406,7 @@ func (s *ExecutionEngine) createBlockFromNextMessage(msg *arbostypes.MessageWith statedb, s.bc, s.bc.Config(), - s.streamer.FetchBatch, + batchFetcher, ) return block, statedb, receipts, err diff --git a/execution/gethexec/node.go b/execution/gethexec/node.go index 87d7e3925c..11748b7e31 100644 --- a/execution/gethexec/node.go +++ b/execution/gethexec/node.go @@ -116,6 +116,7 @@ type ExecutionNode struct { Sequencer *Sequencer // either nil or same as TxPublisher TxPublisher TransactionPublisher ConfigFetcher ConfigFetcher + SyncMonitor *SyncMonitor L1Reader *headerreader.HeaderReader ClassicOutbox *ClassicOutboxRetriever } @@ -177,6 +178,8 @@ func CreateExecutionNode( return nil, err } + syncMon := NewSyncMonitor(execEngine) + var classicOutbox *ClassicOutboxRetriever classicMsgDb, err := stack.OpenDatabase("classic-msg", 0, 0, "", true) if err != nil { @@ -231,13 +234,14 @@ func CreateExecutionNode( sequencer, txPublisher, configFetcher, + syncMon, l1Reader, classicOutbox, }, nil } -func (n *ExecutionNode) Initialize(ctx context.Context, sync arbitrum.SyncProgressBackend) error { +func (n *ExecutionNode) Initialize(ctx context.Context) error { n.ArbInterface.Initialize(n) err := n.Backend.Start() if err != nil { @@ -247,7 +251,7 @@ func (n *ExecutionNode) Initialize(ctx context.Context, sync arbitrum.SyncProgre if err != nil { return fmt.Errorf("error initializing transaction publisher: %w", err) } - err = n.Backend.APIBackend().SetSyncBackend(sync) + err = n.Backend.APIBackend().SetSyncBackend(n.SyncMonitor) if err != nil { return fmt.Errorf("error setting sync backend: %w", err) } @@ -265,10 +269,9 @@ func (n *ExecutionNode) Start(ctx context.Context) error { if err != nil { return fmt.Errorf("error starting transaction puiblisher: %w", err) } - // TODO after separation - // if n.L1Reader != nil { - // n.L1Reader.Start(ctx) - // } + if n.L1Reader != nil { + n.L1Reader.Start(ctx) + } return nil } @@ -279,10 +282,9 @@ func (n *ExecutionNode) StopAndWait() { n.TxPublisher.StopAndWait() } n.Recorder.OrderlyShutdown() - // TODO after separation - // if n.L1Reader != nil && n.L1Reader.Started() { - // n.L1Reader.StopAndWait() - // } + if n.L1Reader != nil && n.L1Reader.Started() { + n.L1Reader.StopAndWait() + } if n.ExecEngine.Started() { n.ExecEngine.StopAndWait() } @@ -344,8 +346,9 @@ func (n *ExecutionNode) ForwardTo(url string) error { return n.Sequencer.ForwardTo(url) } -func (n *ExecutionNode) SetTransactionStreamer(streamer execution.TransactionStreamer) { - n.ExecEngine.SetTransactionStreamer(streamer) +func (n *ExecutionNode) SetConsensusClient(consensus execution.FullConsensusClient) { + n.ExecEngine.SetTransactionStreamer(consensus) + n.SyncMonitor.SetConsensusInfo(consensus) } func (n *ExecutionNode) MessageIndexToBlockNumber(messageNum arbutil.MessageIndex) uint64 { diff --git a/execution/gethexec/sequencer.go b/execution/gethexec/sequencer.go index e8333e5443..d0dc3de175 100644 --- a/execution/gethexec/sequencer.go +++ b/execution/gethexec/sequencer.go @@ -438,7 +438,7 @@ func (s *Sequencer) CheckHealth(ctx context.Context) error { if pauseChan != nil { return nil } - return s.execEngine.streamer.ExpectChosenSequencer() + return s.execEngine.consensus.ExpectChosenSequencer() } func (s *Sequencer) ForwardTarget() string { diff --git a/execution/gethexec/sync_monitor.go b/execution/gethexec/sync_monitor.go new file mode 100644 index 0000000000..83c8d545af --- /dev/null +++ b/execution/gethexec/sync_monitor.go @@ -0,0 +1,70 @@ +package gethexec + +import ( + "context" + + "github.com/offchainlabs/nitro/execution" + "github.com/pkg/errors" +) + +type SyncMonitor struct { + consensus execution.ConsensusInfo + exec *ExecutionEngine +} + +func NewSyncMonitor(exec *ExecutionEngine) *SyncMonitor { + return &SyncMonitor{ + exec: exec, + } +} + +func (s *SyncMonitor) SyncProgressMap() map[string]interface{} { + res := s.consensus.SyncProgressMap() + consensusDelayedMax := s.consensus.GetDelayedMaxMessageCount() + + built, err := s.exec.HeadMessageNumber() + if err != nil { + res["headMsgNumberError"] = err + } + + if built+1 >= consensusDelayedMax && len(res) == 0 { + return res + } + + res["builtBlock"] = built + res["consensusDelayedMax"] = consensusDelayedMax + + return res +} + +func (s *SyncMonitor) SafeBlockNumber(ctx context.Context) (uint64, error) { + if s.consensus == nil { + return 0, errors.New("not set up for safeblock") + } + msg, err := s.consensus.GetSafeMsgCount(ctx) + if err != nil { + return 0, err + } + block := s.exec.MessageIndexToBlockNumber(msg - 1) + return block, nil +} + +func (s *SyncMonitor) FinalizedBlockNumber(ctx context.Context) (uint64, error) { + if s.consensus == nil { + return 0, errors.New("not set up for safeblock") + } + msg, err := s.consensus.GetFinalizedMsgCount(ctx) + if err != nil { + return 0, err + } + block := s.exec.MessageIndexToBlockNumber(msg - 1) + return block, nil +} + +func (s *SyncMonitor) Synced() bool { + return len(s.SyncProgressMap()) == 0 +} + +func (s *SyncMonitor) SetConsensusInfo(consensus execution.ConsensusInfo) { + s.consensus = consensus +} diff --git a/execution/interface.go b/execution/interface.go index b1de97e2ba..119c909c51 100644 --- a/execution/interface.go +++ b/execution/interface.go @@ -54,7 +54,6 @@ type ExecutionSequencer interface { Pause() Activate() ForwardTo(url string) error - SetTransactionStreamer(streamer TransactionStreamer) } type FullExecutionClient interface { @@ -64,19 +63,33 @@ type FullExecutionClient interface { Maintenance() error - // TODO: only used to get safe/finalized block numbers - MessageIndexToBlockNumber(messageNum arbutil.MessageIndex) uint64 + SetConsensusClient(consensus FullConsensusClient) } // not implemented in execution, used as input +// BatchFetcher is required for any execution node type BatchFetcher interface { - FetchBatch(batchNum uint64) ([]byte, error) + FetchBatch(ctx context.Context, batchNum uint64) ([]byte, error) FindL1BatchForMessage(message arbutil.MessageIndex) (uint64, error) GetBatchL1Block(seqNum uint64) (uint64, error) } -type TransactionStreamer interface { - BatchFetcher +type ConsensusInfo interface { + SyncProgressMap() map[string]interface{} + GetDelayedMaxMessageCount() arbutil.MessageIndex + + // TODO: switch from pulling to pushing safe/finalized + GetSafeMsgCount(ctx context.Context) (arbutil.MessageIndex, error) + GetFinalizedMsgCount(ctx context.Context) (arbutil.MessageIndex, error) +} + +type ConsensusSequencer interface { WriteMessageFromSequencer(pos arbutil.MessageIndex, msgWithMeta arbostypes.MessageWithMetadata) error ExpectChosenSequencer() error } + +type FullConsensusClient interface { + BatchFetcher + ConsensusInfo + ConsensusSequencer +} From b2a7e670985eeffc0ea8020c90bd3ec0f6c4f3d3 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Thu, 23 Mar 2023 18:47:20 +0200 Subject: [PATCH 06/34] test more node_interface functions --- execution/gethexec/node.go | 3 +- system_tests/full_challenge_impl_test.go | 31 +++-- ...timation_test.go => nodeinterface_test.go} | 117 ++++++++++++++++++ 3 files changed, 137 insertions(+), 14 deletions(-) rename system_tests/{estimation_test.go => nodeinterface_test.go} (59%) diff --git a/execution/gethexec/node.go b/execution/gethexec/node.go index 11748b7e31..ba3131876e 100644 --- a/execution/gethexec/node.go +++ b/execution/gethexec/node.go @@ -80,6 +80,7 @@ func ConfigAddOptions(prefix string, f *flag.FlagSet) { } var ConfigDefault = Config{ + L1Reader: headerreader.DefaultConfig, RPC: arbitrum.DefaultConfig, Sequencer: DefaultSequencerConfig, ForwardingTargetImpl: "", @@ -100,7 +101,7 @@ func ConfigDefaultNonSequencerTest() *Config { func ConfigDefaultTest() *Config { config := ConfigDefault config.Sequencer = TestSequencerConfig - + config.L1Reader = headerreader.TestConfig return &config } diff --git a/system_tests/full_challenge_impl_test.go b/system_tests/full_challenge_impl_test.go index 8b28832a3a..bb8144141a 100644 --- a/system_tests/full_challenge_impl_test.go +++ b/system_tests/full_challenge_impl_test.go @@ -32,6 +32,7 @@ import ( "github.com/offchainlabs/nitro/solgen/go/mocksgen" "github.com/offchainlabs/nitro/solgen/go/ospgen" "github.com/offchainlabs/nitro/staker" + "github.com/offchainlabs/nitro/util/signature" "github.com/offchainlabs/nitro/validator" "github.com/offchainlabs/nitro/validator/server_common" "github.com/offchainlabs/nitro/validator/valnode" @@ -222,6 +223,16 @@ func setupSequencerInboxStub(ctx context.Context, t *testing.T, l1Info *Blockcha return bridgeAddr, seqInbox, seqInboxAddr } +func createL2Nodes(t *testing.T, ctx context.Context, conf *arbnode.Config, chainConfig *params.ChainConfig, l1Client arbutil.L1Interface, l2info *BlockchainTestInfo, rollupAddresses *arbnode.RollupAddresses, txOpts *bind.TransactOpts, signer signature.DataSignerFunc, fatalErrChan chan error) (*arbnode.Node, *gethexec.ExecutionNode) { + _, stack, l2ChainDb, l2ArbDb, l2Blockchain := createL2BlockChain(t, l2info, "", chainConfig) + execNode, err := gethexec.CreateExecutionNode(stack, l2ChainDb, l2Blockchain, l1Client, gethexec.ConfigDefaultTest) + Require(t, err) + consensusNode, err := arbnode.CreateNode(ctx, stack, execNode, l2ArbDb, conf, chainConfig, l1Client, rollupAddresses, txOpts, signer, fatalErrChan) + Require(t, err) + + return consensusNode, execNode +} + func RunChallengeTest(t *testing.T, asserterIsCorrect bool, useStubs bool, challengeMsgIdx int64) { glogger := log.NewGlogHandler(log.StreamHandler(os.Stderr, log.TerminalFormat(false))) glogger.Verbosity(log.LvlInfo) @@ -264,24 +275,18 @@ func RunChallengeTest(t *testing.T, asserterIsCorrect bool, useStubs bool, chall asserterBridgeAddr, asserterSeqInbox, asserterSeqInboxAddr := setupSequencerInboxStub(ctx, t, l1Info, l1Backend, chainConfig) challengerBridgeAddr, challengerSeqInbox, challengerSeqInboxAddr := setupSequencerInboxStub(ctx, t, l1Info, l1Backend, chainConfig) - asserterL2Info, asserterL2Stack, asserterL2ChainDb, asserterL2ArbDb, asserterL2Blockchain := createL2BlockChain(t, nil, "", chainConfig) asserterRollupAddresses.Bridge = asserterBridgeAddr asserterRollupAddresses.SequencerInbox = asserterSeqInboxAddr - asserterExec, err := gethexec.CreateExecutionNode(asserterL2Stack, asserterL2ChainDb, asserterL2Blockchain, l1Backend, gethexec.ConfigDefaultTest) - Require(t, err) - asserterL2, err := arbnode.CreateNode(ctx, asserterL2Stack, asserterExec, asserterL2ArbDb, conf, chainConfig, l1Backend, asserterRollupAddresses, nil, nil, fatalErrChan) - Require(t, err) - err = asserterL2.Start(ctx) + asserterL2Info := NewArbTestInfo(t, chainConfig.ChainID) + asserterL2, asserterExec := createL2Nodes(t, ctx, conf, chainConfig, l1Backend, asserterL2Info, asserterRollupAddresses, nil, nil, fatalErrChan) + err := asserterL2.Start(ctx) Require(t, err) - challengerL2Info, challengerL2Stack, challengerL2ChainDb, challengerL2ArbDb, challengerL2Blockchain := createL2BlockChain(t, nil, "", chainConfig) challengerRollupAddresses := *asserterRollupAddresses challengerRollupAddresses.Bridge = challengerBridgeAddr challengerRollupAddresses.SequencerInbox = challengerSeqInboxAddr - challengerExec, err := gethexec.CreateExecutionNode(challengerL2Stack, challengerL2ChainDb, challengerL2Blockchain, l1Backend, gethexec.ConfigDefaultTest) - Require(t, err) - challengerL2, err := arbnode.CreateNode(ctx, challengerL2Stack, challengerExec, challengerL2ArbDb, conf, chainConfig, l1Backend, &challengerRollupAddresses, nil, nil, fatalErrChan) - Require(t, err) + challengerL2Info := NewArbTestInfo(t, chainConfig.ChainID) + challengerL2, challengerExec := createL2Nodes(t, ctx, conf, chainConfig, l1Backend, challengerL2Info, &challengerRollupAddresses, nil, nil, fatalErrChan) err = challengerL2.Start(ctx) Require(t, err) @@ -369,7 +374,7 @@ func RunChallengeTest(t *testing.T, asserterIsCorrect bool, useStubs bool, chall confirmLatestBlock(ctx, t, l1Info, l1Backend) - asserterValidator, err := staker.NewStatelessBlockValidator(asserterL2.InboxReader, asserterL2.InboxTracker, asserterL2.TxStreamer, asserterExec.Recorder, asserterL2ArbDb, nil, &conf.BlockValidator) + asserterValidator, err := staker.NewStatelessBlockValidator(asserterL2.InboxReader, asserterL2.InboxTracker, asserterL2.TxStreamer, asserterExec.Recorder, asserterL2.ArbDB, nil, &conf.BlockValidator) if err != nil { Fail(t, err) } @@ -386,7 +391,7 @@ func RunChallengeTest(t *testing.T, asserterIsCorrect bool, useStubs bool, chall if err != nil { Fail(t, err) } - challengerValidator, err := staker.NewStatelessBlockValidator(challengerL2.InboxReader, challengerL2.InboxTracker, challengerL2.TxStreamer, challengerExec.Recorder, challengerL2ArbDb, nil, &conf.BlockValidator) + challengerValidator, err := staker.NewStatelessBlockValidator(challengerL2.InboxReader, challengerL2.InboxTracker, challengerL2.TxStreamer, challengerExec.Recorder, challengerL2.ArbDB, nil, &conf.BlockValidator) if err != nil { Fail(t, err) } diff --git a/system_tests/estimation_test.go b/system_tests/nodeinterface_test.go similarity index 59% rename from system_tests/estimation_test.go rename to system_tests/nodeinterface_test.go index a1c411d7d5..82328d505e 100644 --- a/system_tests/estimation_test.go +++ b/system_tests/nodeinterface_test.go @@ -9,10 +9,13 @@ import ( "testing" "github.com/ethereum/go-ethereum" + "github.com/ethereum/go-ethereum/accounts/abi" "github.com/ethereum/go-ethereum/accounts/abi/bind" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/ethclient" "github.com/ethereum/go-ethereum/params" + "github.com/offchainlabs/nitro/arbnode" "github.com/offchainlabs/nitro/arbos/l1pricing" "github.com/offchainlabs/nitro/solgen/go/mocksgen" "github.com/offchainlabs/nitro/solgen/go/node_interfacegen" @@ -219,3 +222,117 @@ func TestComponentEstimate(t *testing.T) { Fail(t, l2Estimate, l2Used) } } + +func callFindBatchContainig(t *testing.T, ctx context.Context, client *ethclient.Client, nodeAbi *abi.ABI, blockNum uint64) uint64 { + findBatch := nodeAbi.Methods["findBatchContainingBlock"] + callData := append([]byte{}, findBatch.ID...) + packed, err := findBatch.Inputs.Pack(blockNum) + Require(t, err) + callData = append(callData, packed...) + msg := ethereum.CallMsg{ + To: &types.NodeInterfaceAddress, + Data: callData, + } + returnData, err := client.CallContract(ctx, msg, nil) + Require(t, err) + outputs, err := findBatch.Outputs.Unpack(returnData) + Require(t, err) + if len(outputs) != 1 { + Fail(t, "expected 1 output from findBatchContainingBlock, got", len(outputs)) + } + gotBatchNum, ok := outputs[0].(uint64) + if !ok { + Fail(t, "bad output from findBatchContainingBlock") + } + return gotBatchNum +} + +func callGetL1Confirmations(t *testing.T, ctx context.Context, client *ethclient.Client, nodeAbi *abi.ABI, blockHash common.Hash) uint64 { + getConfirmations := nodeAbi.Methods["getL1Confirmations"] + callData := append([]byte{}, getConfirmations.ID...) + packed, err := getConfirmations.Inputs.Pack(blockHash) + Require(t, err) + callData = append(callData, packed...) + msg := ethereum.CallMsg{ + To: &types.NodeInterfaceAddress, + Data: callData, + } + returnData, err := client.CallContract(ctx, msg, nil) + Require(t, err) + outputs, err := getConfirmations.Outputs.Unpack(returnData) + Require(t, err) + if len(outputs) != 1 { + Fail(t, "expected 1 output from findBatchContainingBlock, got", len(outputs)) + } + confirmations, ok := outputs[0].(uint64) + if !ok { + Fail(t, "bad output from findBatchContainingBlock") + } + return confirmations +} + +func TestFindBatch(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + l1Info := NewL1TestInfo(t) + initialBalance := new(big.Int).Lsh(big.NewInt(1), 200) + l1Info.GenerateGenesisAccount("deployer", initialBalance) + l1Info.GenerateGenesisAccount("asserter", initialBalance) + l1Info.GenerateGenesisAccount("challenger", initialBalance) + l1Info.GenerateGenesisAccount("sequencer", initialBalance) + + l1Info, l1Backend, _, _ := createTestL1BlockChain(t, l1Info) + conf := arbnode.ConfigDefaultL1Test() + conf.BlockValidator.Enable = false + conf.BatchPoster.Enable = false + + chainConfig := params.ArbitrumDevTestChainConfig() + fatalErrChan := make(chan error, 10) + rollupAddresses := DeployOnTestL1(t, ctx, l1Info, l1Backend, chainConfig.ChainID) + + bridgeAddr, seqInbox, seqInboxAddr := setupSequencerInboxStub(ctx, t, l1Info, l1Backend, chainConfig) + + rollupAddresses.Bridge = bridgeAddr + rollupAddresses.SequencerInbox = seqInboxAddr + l2Info := NewArbTestInfo(t, chainConfig.ChainID) + consensus, _ := createL2Nodes(t, ctx, conf, chainConfig, l1Backend, l2Info, rollupAddresses, nil, nil, fatalErrChan) + err := consensus.Start(ctx) + Require(t, err) + + l2Client := ClientForStack(t, consensus.Stack) + nodeAbi, err := node_interfacegen.NodeInterfaceMetaData.GetAbi() + Require(t, err) + sequencerTxOpts := l1Info.GetDefaultTransactOpts("sequencer", ctx) + + l2Info.GenerateAccount("Destination") + makeBatch(t, consensus, l2Info, l1Backend, &sequencerTxOpts, seqInbox, seqInboxAddr, -1) + makeBatch(t, consensus, l2Info, l1Backend, &sequencerTxOpts, seqInbox, seqInboxAddr, -1) + makeBatch(t, consensus, l2Info, l1Backend, &sequencerTxOpts, seqInbox, seqInboxAddr, -1) + + for blockNum := uint64(0); blockNum < uint64(MsgPerBatch)*3; blockNum++ { + gotBatchNum := callFindBatchContainig(t, ctx, l2Client, nodeAbi, blockNum) + expBatchNum := uint64(0) + if blockNum > 0 { + expBatchNum = 1 + (blockNum-1)/uint64(MsgPerBatch) + } + if expBatchNum != gotBatchNum { + Fail(t, "wrong result from findBatchContainingBlock. blocknum ", blockNum, " expected ", expBatchNum, " got ", gotBatchNum) + } + batchL1Block, err := consensus.InboxTracker.GetBatchL1Block(gotBatchNum) + Require(t, err) + blockHeader, err := l2Client.HeaderByNumber(ctx, new(big.Int).SetUint64(blockNum)) + Require(t, err) + blockHash := blockHeader.Hash() + + minCurrentL1Block, err := l1Backend.BlockNumber(ctx) + Require(t, err) + gotConfirmations := callGetL1Confirmations(t, ctx, l2Client, nodeAbi, blockHash) + maxCurrentL1Block, err := l1Backend.BlockNumber(ctx) + Require(t, err) + + if gotConfirmations > (maxCurrentL1Block-batchL1Block) || gotConfirmations < (minCurrentL1Block-batchL1Block) { + Fail(t, "wrong number of confirmations. got ", gotConfirmations) + } + } +} From 8186d20f2805160faef32d53b0a9ecb7388ab6ae Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Sat, 25 Mar 2023 10:55:27 -0600 Subject: [PATCH 07/34] call start/stop for syncMonitor --- arbnode/node.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/arbnode/node.go b/arbnode/node.go index 391f840492..4f2b9ed5ec 100644 --- a/arbnode/node.go +++ b/arbnode/node.go @@ -1053,6 +1053,7 @@ func (n *Node) Start(ctx context.Context) error { if n.configFetcher != nil { n.configFetcher.Start(ctx) } + n.SyncMonitor.Start(ctx) return nil } @@ -1110,6 +1111,7 @@ func (n *Node) StopAndWait() { // Just stops the redis client (most other stuff was stopped earlier) n.SeqCoordinator.StopAndWait() } + n.SyncMonitor.StopAndWait() if n.DASLifecycleManager != nil { n.DASLifecycleManager.StopAndWaitUntil(2 * time.Second) } From aeed89aab0f719ef241324570c88b83436fca348 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Sat, 25 Mar 2023 14:14:49 -0600 Subject: [PATCH 08/34] fix TestRedisForwarder --- system_tests/forwarder_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/system_tests/forwarder_test.go b/system_tests/forwarder_test.go index 6595eefb61..d278d72737 100644 --- a/system_tests/forwarder_test.go +++ b/system_tests/forwarder_test.go @@ -214,7 +214,8 @@ func TestRedisForwarder(t *testing.T) { if err == nil { break } - time.Sleep(gethexec.DefaultTestForwarderConfig.UpdateInterval / 2) + // takes > MsgLag for node to know it's in sync and become primary + time.Sleep(arbnode.DefaultSyncMonitorConfig.MsgLag / 2) } testhelpers.RequireImpl(t, err) _, err = EnsureTxSucceeded(ctx, clients[i], tx) From fe74bc9c202df83e61d6a339a5a4da6bc7b9ec9d Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Sun, 26 Mar 2023 11:04:26 -0600 Subject: [PATCH 09/34] Sequencer price adjusts: make non-parallel These tests often fail in CI because the chain doesn't move fast enough. Doing it serially should help. --- system_tests/fees_test.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/system_tests/fees_test.go b/system_tests/fees_test.go index 3efe2bfdf3..51b727628f 100644 --- a/system_tests/fees_test.go +++ b/system_tests/fees_test.go @@ -118,8 +118,6 @@ func TestSequencerFeePaid(t *testing.T) { } func testSequencerPriceAdjustsFrom(t *testing.T, initialEstimate uint64) { - t.Parallel() - _ = os.Mkdir("test-data", 0766) path := filepath.Join("test-data", fmt.Sprintf("testSequencerPriceAdjustsFrom%v.csv", initialEstimate)) From 84fc418074bb838e60c908a93238427897fb85ee Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Sun, 26 Mar 2023 19:24:46 -0600 Subject: [PATCH 10/34] fix build of rce-detection tests --- system_tests/full_challenge_impl_test.go | 18 ------------------ system_tests/full_challenge_mock_test.go | 21 +++++++++++++++++++++ 2 files changed, 21 insertions(+), 18 deletions(-) create mode 100644 system_tests/full_challenge_mock_test.go diff --git a/system_tests/full_challenge_impl_test.go b/system_tests/full_challenge_impl_test.go index 19d7ec492f..ef0d6f9c51 100644 --- a/system_tests/full_challenge_impl_test.go +++ b/system_tests/full_challenge_impl_test.go @@ -1,10 +1,6 @@ // Copyright 2021-2022, Offchain Labs, Inc. // For license information, see https://github.com/nitro/blob/master/LICENSE -// race detection makes things slow and miss timeouts -//go:build !race -// +build !race - package arbtest import ( @@ -481,17 +477,3 @@ func RunChallengeTest(t *testing.T, asserterIsCorrect bool, useStubs bool, chall Fail(t, "challenge timed out without winner") } - -func TestMockChallengeManagerAsserterIncorrect(t *testing.T) { - t.Parallel() - for i := int64(1); i <= MsgPerBatch*3; i++ { - RunChallengeTest(t, false, true, i) - } -} - -func TestMockChallengeManagerAsserterCorrect(t *testing.T) { - t.Parallel() - for i := int64(1); i <= MsgPerBatch*3; i++ { - RunChallengeTest(t, true, true, i) - } -} diff --git a/system_tests/full_challenge_mock_test.go b/system_tests/full_challenge_mock_test.go new file mode 100644 index 0000000000..751c40ef07 --- /dev/null +++ b/system_tests/full_challenge_mock_test.go @@ -0,0 +1,21 @@ +// race detection makes things slow and miss timeouts +//go:build !race +// +build !race + +package arbtest + +import "testing" + +func TestMockChallengeManagerAsserterIncorrect(t *testing.T) { + t.Parallel() + for i := int64(1); i <= MsgPerBatch*3; i++ { + RunChallengeTest(t, false, true, i) + } +} + +func TestMockChallengeManagerAsserterCorrect(t *testing.T) { + t.Parallel() + for i := int64(1); i <= MsgPerBatch*3; i++ { + RunChallengeTest(t, true, true, i) + } +} From fc8bcb27446f1b07763b5d76fcde1da02101380b Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Wed, 29 Mar 2023 15:34:52 -0600 Subject: [PATCH 11/34] execution: only link to l1 if sequencer --- execution/gethexec/node.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/execution/gethexec/node.go b/execution/gethexec/node.go index b6a4ade2c5..2f2b951741 100644 --- a/execution/gethexec/node.go +++ b/execution/gethexec/node.go @@ -139,15 +139,17 @@ func CreateExecutionNode( var sequencer *Sequencer var l1Reader *headerreader.HeaderReader - if l1client != nil { - l1Reader = headerreader.New(l1client, func() *headerreader.Config { return &configFetcher().L1Reader }) - } fwTarget := config.ForwardingTarget() if config.Sequencer.Enable { if fwTarget != "" { return nil, errors.New("sequencer and forwarding target both set") } + if l1client != nil { + l1Reader = headerreader.New(l1client, func() *headerreader.Config { return &configFetcher().L1Reader }) + } else { + log.Warn("sequencer enabled without l1 client") + } seqConfigFetcher := func() *SequencerConfig { return &configFetcher().Sequencer } sequencer, err = NewSequencer(execEngine, l1Reader, seqConfigFetcher) if err != nil { From 696d91f8a2efcd9b0d418b3dd10c9b89241d97df Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Mon, 3 Apr 2023 09:51:56 -0600 Subject: [PATCH 12/34] syncMonitor: create Test config --- arbnode/node.go | 2 ++ arbnode/sync_monitor.go | 4 ++++ 2 files changed, 6 insertions(+) diff --git a/arbnode/node.go b/arbnode/node.go index 4f2b9ed5ec..49423c6de7 100644 --- a/arbnode/node.go +++ b/arbnode/node.go @@ -488,6 +488,7 @@ func ConfigDefaultL1NonSequencerTest() *Config { config.BatchPoster.Enable = false config.SeqCoordinator.Enable = false config.BlockValidator = staker.TestBlockValidatorConfig + config.SyncMonitor = TestSyncMonitorConfig return &config } @@ -500,6 +501,7 @@ func ConfigDefaultL2Test() *Config { config.Feed.Output.Signed = false config.SeqCoordinator.Signing.ECDSA.AcceptSequencer = false config.SeqCoordinator.Signing.ECDSA.Dangerous.AcceptMissing = true + config.SyncMonitor = TestSyncMonitorConfig return &config } diff --git a/arbnode/sync_monitor.go b/arbnode/sync_monitor.go index a88aa9678f..e7295a5828 100644 --- a/arbnode/sync_monitor.go +++ b/arbnode/sync_monitor.go @@ -38,6 +38,10 @@ var DefaultSyncMonitorConfig = SyncMonitorConfig{ MsgLag: time.Second, } +var TestSyncMonitorConfig = SyncMonitorConfig{ + MsgLag: time.Millisecond * 10, +} + func SyncMonitorConfigAddOptions(prefix string, f *flag.FlagSet) { f.Duration(prefix+".msg-lag", DefaultSyncMonitorConfig.MsgLag, "allowed msg lag while still considered in sync") } From 633b586f0b3354b792b4d3b9f5ec87c677227d95 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Mon, 3 Apr 2023 09:58:11 -0600 Subject: [PATCH 13/34] sync monitor: ewname API to SyncTarget DelayedMaxMessageCount is confusing with delayed inbox messages --- arbnode/node.go | 4 ++-- arbnode/sync_monitor.go | 36 +++++++++++++++--------------- execution/gethexec/sync_monitor.go | 6 ++--- execution/interface.go | 2 +- 4 files changed, 24 insertions(+), 24 deletions(-) diff --git a/arbnode/node.go b/arbnode/node.go index 49423c6de7..e2ba5f381a 100644 --- a/arbnode/node.go +++ b/arbnode/node.go @@ -1138,8 +1138,8 @@ func (n *Node) SyncProgressMap() map[string]interface{} { return n.SyncMonitor.SyncProgressMap() } -func (n *Node) GetDelayedMaxMessageCount() arbutil.MessageIndex { - return n.SyncMonitor.GetDelayedMaxMessageCount() +func (n *Node) SyncTargetMessageCount() arbutil.MessageIndex { + return n.SyncMonitor.SyncTargetMessageCount() } // TODO: switch from pulling to pushing safe/finalized diff --git a/arbnode/sync_monitor.go b/arbnode/sync_monitor.go index e7295a5828..3641a40790 100644 --- a/arbnode/sync_monitor.go +++ b/arbnode/sync_monitor.go @@ -19,9 +19,9 @@ type SyncMonitor struct { coordinator *SeqCoordinator initialized bool - maxMsgLock sync.Mutex - lastMaxMessageCount arbutil.MessageIndex - prevMaxMessageCount arbutil.MessageIndex + syncTargetLock sync.Mutex + nextSyncTarget arbutil.MessageIndex + syncTarget arbutil.MessageIndex } func NewSyncMonitor(config func() *SyncMonitorConfig) *SyncMonitor { @@ -53,23 +53,23 @@ func (s *SyncMonitor) Initialize(inboxReader *InboxReader, txStreamer *Transacti s.initialized = true } -func (s *SyncMonitor) updateDelayedMaxMessageCount(ctx context.Context) time.Duration { - maxMsg, err := s.maxMessageCount() +func (s *SyncMonitor) updateSyncTarget(ctx context.Context) time.Duration { + nextSyncTarget, err := s.maxMessageCount() if err != nil { log.Warn("failed readin max msg count", "err", err) return s.config().MsgLag } - s.maxMsgLock.Lock() - defer s.maxMsgLock.Unlock() - s.prevMaxMessageCount = s.lastMaxMessageCount - s.lastMaxMessageCount = maxMsg + s.syncTargetLock.Lock() + defer s.syncTargetLock.Unlock() + s.syncTarget = s.nextSyncTarget + s.nextSyncTarget = nextSyncTarget return s.config().MsgLag } -func (s *SyncMonitor) GetDelayedMaxMessageCount() arbutil.MessageIndex { - s.maxMsgLock.Lock() - defer s.maxMsgLock.Unlock() - return s.prevMaxMessageCount +func (s *SyncMonitor) SyncTargetMessageCount() arbutil.MessageIndex { + s.syncTargetLock.Lock() + defer s.syncTargetLock.Unlock() + return s.syncTarget } func (s *SyncMonitor) maxMessageCount() (arbutil.MessageIndex, error) { @@ -122,8 +122,8 @@ func (s *SyncMonitor) SyncProgressMap() map[string]interface{} { return res } - delayedMax := s.GetDelayedMaxMessageCount() - res["delayedMaxMsgCount"] = delayedMax + syncTarget := s.SyncTargetMessageCount() + res["syncTargetMsgCount"] = syncTarget msgCount, err := s.txStreamer.GetMessageCount() if err != nil { @@ -175,7 +175,7 @@ func (s *SyncMonitor) SyncProgressMap() map[string]interface{} { func (s *SyncMonitor) Start(ctx_in context.Context) { s.StopWaiter.Start(ctx_in, s) - s.CallIteratively(s.updateDelayedMaxMessageCount) + s.CallIteratively(s.updateSyncTarget) } func (s *SyncMonitor) Synced() bool { @@ -185,14 +185,14 @@ func (s *SyncMonitor) Synced() bool { if !s.Started() { return false } - delayedMax := s.GetDelayedMaxMessageCount() + syncTarget := s.SyncTargetMessageCount() msgCount, err := s.txStreamer.GetMessageCount() if err != nil { return false } - if delayedMax > msgCount { + if syncTarget > msgCount { return false } diff --git a/execution/gethexec/sync_monitor.go b/execution/gethexec/sync_monitor.go index 83c8d545af..e790d9485a 100644 --- a/execution/gethexec/sync_monitor.go +++ b/execution/gethexec/sync_monitor.go @@ -20,19 +20,19 @@ func NewSyncMonitor(exec *ExecutionEngine) *SyncMonitor { func (s *SyncMonitor) SyncProgressMap() map[string]interface{} { res := s.consensus.SyncProgressMap() - consensusDelayedMax := s.consensus.GetDelayedMaxMessageCount() + consensusSyncTarget := s.consensus.SyncTargetMessageCount() built, err := s.exec.HeadMessageNumber() if err != nil { res["headMsgNumberError"] = err } - if built+1 >= consensusDelayedMax && len(res) == 0 { + if built+1 >= consensusSyncTarget && len(res) == 0 { return res } res["builtBlock"] = built - res["consensusDelayedMax"] = consensusDelayedMax + res["consensusSyncTarget"] = consensusSyncTarget return res } diff --git a/execution/interface.go b/execution/interface.go index a333926eca..ac4cde1f18 100644 --- a/execution/interface.go +++ b/execution/interface.go @@ -74,7 +74,7 @@ type BatchFetcher interface { type ConsensusInfo interface { SyncProgressMap() map[string]interface{} - GetDelayedMaxMessageCount() arbutil.MessageIndex + SyncTargetMessageCount() arbutil.MessageIndex // TODO: switch from pulling to pushing safe/finalized GetSafeMsgCount(ctx context.Context) (arbutil.MessageIndex, error) From e15dced51afebdfdfc5b5ec31eb1211353c5e73f Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Wed, 19 Apr 2023 15:36:19 -0600 Subject: [PATCH 14/34] fix merge --- execution/gethexec/executionengine.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/execution/gethexec/executionengine.go b/execution/gethexec/executionengine.go index 2dd4a703f4..6aebe6bb19 100644 --- a/execution/gethexec/executionengine.go +++ b/execution/gethexec/executionengine.go @@ -238,7 +238,7 @@ func (s *ExecutionEngine) sequencerWrapper(sequencerFunc func() (*types.Block, e } // We got SequencerInsertLockTaken // option 1: there was a race, we are no longer main sequencer - chosenErr := s.streamer.ExpectChosenSequencer() + chosenErr := s.consensus.ExpectChosenSequencer() if chosenErr != nil { return nil, chosenErr } From 4482c20e44f74470e3c5e92c1e37a99e4f318e37 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Wed, 14 Jun 2023 18:48:17 -0600 Subject: [PATCH 15/34] fix merge --- system_tests/nodeinterface_test.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/system_tests/nodeinterface_test.go b/system_tests/nodeinterface_test.go index 0fc21b3d9c..4a6ead5723 100644 --- a/system_tests/nodeinterface_test.go +++ b/system_tests/nodeinterface_test.go @@ -238,11 +238,11 @@ func callFindBatchContainig(t *testing.T, ctx context.Context, client *ethclient outputs, err := findBatch.Outputs.Unpack(returnData) Require(t, err) if len(outputs) != 1 { - Fail(t, "expected 1 output from findBatchContainingBlock, got", len(outputs)) + Fatal(t, "expected 1 output from findBatchContainingBlock, got", len(outputs)) } gotBatchNum, ok := outputs[0].(uint64) if !ok { - Fail(t, "bad output from findBatchContainingBlock") + Fatal(t, "bad output from findBatchContainingBlock") } return gotBatchNum } @@ -262,11 +262,11 @@ func callGetL1Confirmations(t *testing.T, ctx context.Context, client *ethclient outputs, err := getConfirmations.Outputs.Unpack(returnData) Require(t, err) if len(outputs) != 1 { - Fail(t, "expected 1 output from findBatchContainingBlock, got", len(outputs)) + Fatal(t, "expected 1 output from findBatchContainingBlock, got", len(outputs)) } confirmations, ok := outputs[0].(uint64) if !ok { - Fail(t, "bad output from findBatchContainingBlock") + Fatal(t, "bad output from findBatchContainingBlock") } return confirmations } @@ -317,7 +317,7 @@ func TestFindBatch(t *testing.T) { expBatchNum = 1 + (blockNum-1)/uint64(MsgPerBatch) } if expBatchNum != gotBatchNum { - Fail(t, "wrong result from findBatchContainingBlock. blocknum ", blockNum, " expected ", expBatchNum, " got ", gotBatchNum) + Fatal(t, "wrong result from findBatchContainingBlock. blocknum ", blockNum, " expected ", expBatchNum, " got ", gotBatchNum) } batchL1Block, err := consensus.InboxTracker.GetBatchL1Block(gotBatchNum) Require(t, err) @@ -332,7 +332,7 @@ func TestFindBatch(t *testing.T) { Require(t, err) if gotConfirmations > (maxCurrentL1Block-batchL1Block) || gotConfirmations < (minCurrentL1Block-batchL1Block) { - Fail(t, "wrong number of confirmations. got ", gotConfirmations) + Fatal(t, "wrong number of confirmations. got ", gotConfirmations) } } } From d8498b0616e18c21b32c5cced3e1dfea729ae505 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Mon, 10 Jul 2023 17:17:49 -0600 Subject: [PATCH 16/34] execution: update interface to support L3 --- arbnode/inbox_tracker.go | 4 ++-- arbnode/node.go | 4 ++-- execution/interface.go | 2 +- execution/nodeInterface/NodeInterface.go | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/arbnode/inbox_tracker.go b/arbnode/inbox_tracker.go index 16f7bb41d0..0fc5b11486 100644 --- a/arbnode/inbox_tracker.go +++ b/arbnode/inbox_tracker.go @@ -194,9 +194,9 @@ func (t *InboxTracker) GetBatchMessageCount(seqNum uint64) (arbutil.MessageIndex return metadata.MessageCount, err } -func (t *InboxTracker) GetBatchL1Block(seqNum uint64) (uint64, error) { +func (t *InboxTracker) GetBatchParentChainBlock(seqNum uint64) (uint64, error) { metadata, err := t.GetBatchMetadata(seqNum) - return metadata.L1Block, err + return metadata.ParentChainBlock, err } // GetBatchAcc is a convenience function wrapping GetBatchMetadata diff --git a/arbnode/node.go b/arbnode/node.go index e3a39a09e8..8c1fd288a8 100644 --- a/arbnode/node.go +++ b/arbnode/node.go @@ -1090,8 +1090,8 @@ func (n *Node) FindL1BatchForMessage(message arbutil.MessageIndex) (uint64, erro return n.InboxTracker.FindL1BatchForMessage(message) } -func (n *Node) GetBatchL1Block(seqNum uint64) (uint64, error) { - return n.InboxTracker.GetBatchL1Block(seqNum) +func (n *Node) GetBatchParentChainBlock(seqNum uint64) (uint64, error) { + return n.InboxTracker.GetBatchParentChainBlock(seqNum) } func (n *Node) SyncProgressMap() map[string]interface{} { diff --git a/execution/interface.go b/execution/interface.go index ac4cde1f18..8bc9a44d2d 100644 --- a/execution/interface.go +++ b/execution/interface.go @@ -69,7 +69,7 @@ type FullExecutionClient interface { type BatchFetcher interface { FetchBatch(ctx context.Context, batchNum uint64) ([]byte, error) FindL1BatchForMessage(message arbutil.MessageIndex) (uint64, error) - GetBatchL1Block(seqNum uint64) (uint64, error) + GetBatchParentChainBlock(seqNum uint64) (uint64, error) } type ConsensusInfo interface { diff --git a/execution/nodeInterface/NodeInterface.go b/execution/nodeInterface/NodeInterface.go index 8dc0b195d6..54252959f2 100644 --- a/execution/nodeInterface/NodeInterface.go +++ b/execution/nodeInterface/NodeInterface.go @@ -90,7 +90,7 @@ func (n NodeInterface) GetL1Confirmations(c ctx, evm mech, blockHash bytes32) (u if err != nil { return 0, err } - blockNum, err := node.ExecEngine.GetBatchFetcher().GetBatchL1Block(batchNum) + blockNum, err := node.ExecEngine.GetBatchFetcher().GetBatchParentChainBlock(batchNum) if err != nil { return 0, err } From 47ece30b94640be67b8cd7d066040c5f4e816833 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Mon, 10 Jul 2023 17:18:29 -0600 Subject: [PATCH 17/34] system_tests: handle initMsg from Deploy --- system_tests/full_challenge_impl_test.go | 9 +++++---- system_tests/nodeinterface_test.go | 6 +++--- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/system_tests/full_challenge_impl_test.go b/system_tests/full_challenge_impl_test.go index 60ae5c2def..7d3df4d297 100644 --- a/system_tests/full_challenge_impl_test.go +++ b/system_tests/full_challenge_impl_test.go @@ -25,6 +25,7 @@ import ( "github.com/offchainlabs/nitro/arbcompress" "github.com/offchainlabs/nitro/arbnode" "github.com/offchainlabs/nitro/arbos" + "github.com/offchainlabs/nitro/arbos/arbostypes" "github.com/offchainlabs/nitro/arbstate" "github.com/offchainlabs/nitro/arbutil" "github.com/offchainlabs/nitro/cmd/chaininfo" @@ -224,8 +225,8 @@ func setupSequencerInboxStub(ctx context.Context, t *testing.T, l1Info *Blockcha return bridgeAddr, seqInbox, seqInboxAddr } -func createL2Nodes(t *testing.T, ctx context.Context, conf *arbnode.Config, chainConfig *params.ChainConfig, l1Client arbutil.L1Interface, l2info *BlockchainTestInfo, rollupAddresses *chaininfo.RollupAddresses, txOpts *bind.TransactOpts, signer signature.DataSignerFunc, fatalErrChan chan error) (*arbnode.Node, *gethexec.ExecutionNode) { - _, stack, l2ChainDb, l2ArbDb, l2Blockchain := createL2BlockChain(t, l2info, "", chainConfig) +func createL2Nodes(t *testing.T, ctx context.Context, conf *arbnode.Config, chainConfig *params.ChainConfig, l1Client arbutil.L1Interface, l2info *BlockchainTestInfo, rollupAddresses *chaininfo.RollupAddresses, initMsg *arbostypes.ParsedInitMessage, txOpts *bind.TransactOpts, signer signature.DataSignerFunc, fatalErrChan chan error) (*arbnode.Node, *gethexec.ExecutionNode) { + _, stack, l2ChainDb, l2ArbDb, l2Blockchain := createL2BlockChainWithStackConfig(t, l2info, "", chainConfig, initMsg, nil) execNode, err := gethexec.CreateExecutionNode(ctx, stack, l2ChainDb, l2Blockchain, l1Client, gethexec.ConfigDefaultTest) Require(t, err) consensusNode, err := arbnode.CreateNode(ctx, stack, execNode, l2ArbDb, NewFetcherFromConfig(conf), chainConfig, l1Client, rollupAddresses, txOpts, txOpts, signer, fatalErrChan) @@ -279,7 +280,7 @@ func RunChallengeTest(t *testing.T, asserterIsCorrect bool, useStubs bool, chall asserterRollupAddresses.Bridge = asserterBridgeAddr asserterRollupAddresses.SequencerInbox = asserterSeqInboxAddr asserterL2Info := NewArbTestInfo(t, chainConfig.ChainID) - asserterL2, asserterExec := createL2Nodes(t, ctx, conf, chainConfig, l1Backend, asserterL2Info, asserterRollupAddresses, nil, nil, fatalErrChan) + asserterL2, asserterExec := createL2Nodes(t, ctx, conf, chainConfig, l1Backend, asserterL2Info, asserterRollupAddresses, initMessage, nil, nil, fatalErrChan) err := asserterL2.Start(ctx) Require(t, err) @@ -287,7 +288,7 @@ func RunChallengeTest(t *testing.T, asserterIsCorrect bool, useStubs bool, chall challengerRollupAddresses.Bridge = challengerBridgeAddr challengerRollupAddresses.SequencerInbox = challengerSeqInboxAddr challengerL2Info := NewArbTestInfo(t, chainConfig.ChainID) - challengerL2, challengerExec := createL2Nodes(t, ctx, conf, chainConfig, l1Backend, challengerL2Info, &challengerRollupAddresses, nil, nil, fatalErrChan) + challengerL2, challengerExec := createL2Nodes(t, ctx, conf, chainConfig, l1Backend, challengerL2Info, &challengerRollupAddresses, initMessage, nil, nil, fatalErrChan) err = challengerL2.Start(ctx) Require(t, err) diff --git a/system_tests/nodeinterface_test.go b/system_tests/nodeinterface_test.go index 6636c5950e..06268290c6 100644 --- a/system_tests/nodeinterface_test.go +++ b/system_tests/nodeinterface_test.go @@ -289,14 +289,14 @@ func TestFindBatch(t *testing.T) { chainConfig := params.ArbitrumDevTestChainConfig() fatalErrChan := make(chan error, 10) - rollupAddresses := DeployOnTestL1(t, ctx, l1Info, l1Backend, chainConfig) + rollupAddresses, initMsg := DeployOnTestL1(t, ctx, l1Info, l1Backend, chainConfig) bridgeAddr, seqInbox, seqInboxAddr := setupSequencerInboxStub(ctx, t, l1Info, l1Backend, chainConfig) rollupAddresses.Bridge = bridgeAddr rollupAddresses.SequencerInbox = seqInboxAddr l2Info := NewArbTestInfo(t, chainConfig.ChainID) - consensus, _ := createL2Nodes(t, ctx, conf, chainConfig, l1Backend, l2Info, rollupAddresses, nil, nil, fatalErrChan) + consensus, _ := createL2Nodes(t, ctx, conf, chainConfig, l1Backend, l2Info, rollupAddresses, initMsg, nil, nil, fatalErrChan) err := consensus.Start(ctx) Require(t, err) @@ -319,7 +319,7 @@ func TestFindBatch(t *testing.T) { if expBatchNum != gotBatchNum { Fatal(t, "wrong result from findBatchContainingBlock. blocknum ", blockNum, " expected ", expBatchNum, " got ", gotBatchNum) } - batchL1Block, err := consensus.InboxTracker.GetBatchL1Block(gotBatchNum) + batchL1Block, err := consensus.InboxTracker.GetBatchParentChainBlock(gotBatchNum) Require(t, err) blockHeader, err := l2Client.HeaderByNumber(ctx, new(big.Int).SetUint64(blockNum)) Require(t, err) From 18616c4e024e45dfe732510fe568e78ba573a49a Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Wed, 4 Oct 2023 12:11:06 -0600 Subject: [PATCH 18/34] separate estimation tests from nodeinterface --- system_tests/estimation_test.go | 254 +++++++++++++++++++++++++++++ system_tests/nodeinterface_test.go | 237 --------------------------- 2 files changed, 254 insertions(+), 237 deletions(-) create mode 100644 system_tests/estimation_test.go diff --git a/system_tests/estimation_test.go b/system_tests/estimation_test.go new file mode 100644 index 0000000000..9f2db62dab --- /dev/null +++ b/system_tests/estimation_test.go @@ -0,0 +1,254 @@ +// Copyright 2021-2022, Offchain Labs, Inc. +// For license information, see https://github.com/nitro/blob/master/LICENSE + +package arbtest + +import ( + "context" + "math/big" + "testing" + + "github.com/ethereum/go-ethereum" + "github.com/ethereum/go-ethereum/accounts/abi/bind" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/params" + "github.com/offchainlabs/nitro/arbos/arbostypes" + "github.com/offchainlabs/nitro/solgen/go/mocksgen" + "github.com/offchainlabs/nitro/solgen/go/node_interfacegen" + "github.com/offchainlabs/nitro/solgen/go/precompilesgen" + "github.com/offchainlabs/nitro/util/arbmath" + "github.com/offchainlabs/nitro/util/colors" + "github.com/offchainlabs/nitro/util/testhelpers" +) + +func TestDeploy(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + l2info, node, client := CreateTestL2(t, ctx) + defer node.StopAndWait() + + auth := l2info.GetDefaultTransactOpts("Owner", ctx) + auth.GasMargin = 0 // don't adjust, we want to see if the estimate alone is sufficient + + _, simple := deploySimple(t, ctx, auth, client) + + tx, err := simple.Increment(&auth) + Require(t, err, "failed to call Increment()") + _, err = EnsureTxSucceeded(ctx, client, tx) + Require(t, err) + + counter, err := simple.Counter(&bind.CallOpts{}) + Require(t, err, "failed to get counter") + + if counter != 1 { + Fatal(t, "Unexpected counter value", counter) + } +} + +func TestEstimate(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + l2info, node, client := CreateTestL2(t, ctx) + defer node.StopAndWait() + + auth := l2info.GetDefaultTransactOpts("Owner", ctx) + auth.GasMargin = 0 // don't adjust, we want to see if the estimate alone is sufficient + + gasPrice := big.NewInt(params.GWei / 10) + + // set the gas price + arbOwner, err := precompilesgen.NewArbOwner(common.HexToAddress("0x70"), client) + Require(t, err, "could not deploy ArbOwner contract") + tx, err := arbOwner.SetMinimumL2BaseFee(&auth, gasPrice) + Require(t, err, "could not set L2 gas price") + _, err = EnsureTxSucceeded(ctx, client, tx) + Require(t, err) + + // connect to arbGasInfo precompile + arbGasInfo, err := precompilesgen.NewArbGasInfo(common.HexToAddress("0x6c"), client) + Require(t, err, "could not deploy contract") + + // wait for price to come to equilibrium + equilibrated := false + numTriesLeft := 20 + for !equilibrated && numTriesLeft > 0 { + // make an empty block to let the gas price update + l2info.GasPrice = new(big.Int).Mul(l2info.GasPrice, big.NewInt(2)) + TransferBalance(t, "Owner", "Owner", common.Big0, l2info, client, ctx) + + // check if the price has equilibrated + _, _, _, _, _, setPrice, err := arbGasInfo.GetPricesInWei(&bind.CallOpts{}) + Require(t, err, "could not get L2 gas price") + if gasPrice.Cmp(setPrice) == 0 { + equilibrated = true + } + numTriesLeft-- + } + if !equilibrated { + Fatal(t, "L2 gas price did not converge", gasPrice) + } + + initialBalance, err := client.BalanceAt(ctx, auth.From, nil) + Require(t, err, "could not get balance") + + // deploy a test contract + _, tx, simple, err := mocksgen.DeploySimple(&auth, client) + Require(t, err, "could not deploy contract") + receipt, err := EnsureTxSucceeded(ctx, client, tx) + Require(t, err) + + header, err := client.HeaderByNumber(ctx, receipt.BlockNumber) + Require(t, err, "could not get header") + if header.BaseFee.Cmp(gasPrice) != 0 { + Fatal(t, "Header has wrong basefee", header.BaseFee, gasPrice) + } + + balance, err := client.BalanceAt(ctx, auth.From, nil) + Require(t, err, "could not get balance") + expectedCost := receipt.GasUsed * gasPrice.Uint64() + observedCost := initialBalance.Uint64() - balance.Uint64() + if expectedCost != observedCost { + Fatal(t, "Expected deployment to cost", expectedCost, "instead of", observedCost) + } + + tx, err = simple.Increment(&auth) + Require(t, err, "failed to call Increment()") + _, err = EnsureTxSucceeded(ctx, client, tx) + Require(t, err) + + counter, err := simple.Counter(&bind.CallOpts{}) + Require(t, err, "failed to get counter") + + if counter != 1 { + Fatal(t, "Unexpected counter value", counter) + } +} + +func TestComponentEstimate(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + l2info, node, client := CreateTestL2(t, ctx) + defer node.StopAndWait() + + l1BaseFee := new(big.Int).Set(arbostypes.DefaultInitialL1BaseFee) + l2BaseFee := GetBaseFee(t, client, ctx) + + colors.PrintGrey("l1 basefee ", l1BaseFee) + colors.PrintGrey("l2 basefee ", l2BaseFee) + + userBalance := big.NewInt(1e16) + maxPriorityFeePerGas := big.NewInt(0) + maxFeePerGas := arbmath.BigMulByUfrac(l2BaseFee, 3, 2) + + l2info.GenerateAccount("User") + TransferBalance(t, "Owner", "User", userBalance, l2info, client, ctx) + + from := l2info.GetAddress("User") + to := testhelpers.RandomAddress() + gas := uint64(100000000) + calldata := []byte{0x00, 0x12} + value := big.NewInt(4096) + + nodeAbi, err := node_interfacegen.NodeInterfaceMetaData.GetAbi() + Require(t, err) + + nodeMethod := nodeAbi.Methods["gasEstimateComponents"] + estimateCalldata := append([]byte{}, nodeMethod.ID...) + packed, err := nodeMethod.Inputs.Pack(to, false, calldata) + Require(t, err) + estimateCalldata = append(estimateCalldata, packed...) + + msg := ethereum.CallMsg{ + From: from, + To: &types.NodeInterfaceAddress, + Gas: gas, + GasFeeCap: maxFeePerGas, + GasTipCap: maxPriorityFeePerGas, + Value: value, + Data: estimateCalldata, + } + returnData, err := client.CallContract(ctx, msg, nil) + Require(t, err) + + outputs, err := nodeMethod.Outputs.Unpack(returnData) + Require(t, err) + if len(outputs) != 4 { + Fatal(t, "expected 4 outputs from gasEstimateComponents, got", len(outputs)) + } + + gasEstimate, _ := outputs[0].(uint64) + gasEstimateForL1, _ := outputs[1].(uint64) + baseFee, _ := outputs[2].(*big.Int) + l1BaseFeeEstimate, _ := outputs[3].(*big.Int) + + execNode := getExecNode(t, node) + tx := l2info.SignTxAs("User", &types.DynamicFeeTx{ + ChainID: execNode.ArbInterface.BlockChain().Config().ChainID, + Nonce: 0, + GasTipCap: maxPriorityFeePerGas, + GasFeeCap: maxFeePerGas, + Gas: gasEstimate, + To: &to, + Value: value, + Data: calldata, + }) + + l2Estimate := gasEstimate - gasEstimateForL1 + + colors.PrintBlue("Est. ", gasEstimate, " - ", gasEstimateForL1, " = ", l2Estimate) + + if !arbmath.BigEquals(l1BaseFeeEstimate, l1BaseFee) { + Fatal(t, l1BaseFeeEstimate, l1BaseFee) + } + if !arbmath.BigEquals(baseFee, l2BaseFee) { + Fatal(t, baseFee, l2BaseFee.Uint64()) + } + + Require(t, client.SendTransaction(ctx, tx)) + receipt, err := EnsureTxSucceeded(ctx, client, tx) + Require(t, err) + + l2Used := receipt.GasUsed - receipt.GasUsedForL1 + colors.PrintMint("True ", receipt.GasUsed, " - ", receipt.GasUsedForL1, " = ", l2Used) + + if l2Estimate != l2Used { + Fatal(t, l2Estimate, l2Used) + } +} + +func TestDisableL1Charging(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + _, node, client := CreateTestL2(t, ctx) + defer node.StopAndWait() + addr := common.HexToAddress("0x12345678") + + gasWithL1Charging, err := client.EstimateGas(ctx, ethereum.CallMsg{To: &addr}) + Require(t, err) + + gasWithoutL1Charging, err := client.EstimateGas(ctx, ethereum.CallMsg{To: &addr, SkipL1Charging: true}) + Require(t, err) + + if gasWithL1Charging <= gasWithoutL1Charging { + Fatal(t, "SkipL1Charging didn't disable L1 charging") + } + if gasWithoutL1Charging != params.TxGas { + Fatal(t, "Incorrect gas estimate with disabled L1 charging") + } + + _, err = client.CallContract(ctx, ethereum.CallMsg{To: &addr, Gas: gasWithL1Charging}, nil) + Require(t, err) + + _, err = client.CallContract(ctx, ethereum.CallMsg{To: &addr, Gas: gasWithoutL1Charging}, nil) + if err == nil { + Fatal(t, "CallContract passed with insufficient gas") + } + + _, err = client.CallContract(ctx, ethereum.CallMsg{To: &addr, Gas: gasWithoutL1Charging, SkipL1Charging: true}, nil) + Require(t, err) +} diff --git a/system_tests/nodeinterface_test.go b/system_tests/nodeinterface_test.go index 0b060b2b99..20152a2ef4 100644 --- a/system_tests/nodeinterface_test.go +++ b/system_tests/nodeinterface_test.go @@ -16,214 +16,10 @@ import ( "github.com/ethereum/go-ethereum/ethclient" "github.com/ethereum/go-ethereum/params" "github.com/offchainlabs/nitro/arbnode" - "github.com/offchainlabs/nitro/arbos/arbostypes" "github.com/offchainlabs/nitro/arbos/util" - "github.com/offchainlabs/nitro/solgen/go/mocksgen" "github.com/offchainlabs/nitro/solgen/go/node_interfacegen" - "github.com/offchainlabs/nitro/solgen/go/precompilesgen" - "github.com/offchainlabs/nitro/util/arbmath" - "github.com/offchainlabs/nitro/util/colors" - "github.com/offchainlabs/nitro/util/testhelpers" ) -func TestDeploy(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - l2info, node, client := CreateTestL2(t, ctx) - defer node.StopAndWait() - - auth := l2info.GetDefaultTransactOpts("Owner", ctx) - auth.GasMargin = 0 // don't adjust, we want to see if the estimate alone is sufficient - - _, simple := deploySimple(t, ctx, auth, client) - - tx, err := simple.Increment(&auth) - Require(t, err, "failed to call Increment()") - _, err = EnsureTxSucceeded(ctx, client, tx) - Require(t, err) - - counter, err := simple.Counter(&bind.CallOpts{}) - Require(t, err, "failed to get counter") - - if counter != 1 { - Fatal(t, "Unexpected counter value", counter) - } -} - -func TestEstimate(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - l2info, node, client := CreateTestL2(t, ctx) - defer node.StopAndWait() - - auth := l2info.GetDefaultTransactOpts("Owner", ctx) - auth.GasMargin = 0 // don't adjust, we want to see if the estimate alone is sufficient - - gasPrice := big.NewInt(params.GWei / 10) - - // set the gas price - arbOwner, err := precompilesgen.NewArbOwner(common.HexToAddress("0x70"), client) - Require(t, err, "could not deploy ArbOwner contract") - tx, err := arbOwner.SetMinimumL2BaseFee(&auth, gasPrice) - Require(t, err, "could not set L2 gas price") - _, err = EnsureTxSucceeded(ctx, client, tx) - Require(t, err) - - // connect to arbGasInfo precompile - arbGasInfo, err := precompilesgen.NewArbGasInfo(common.HexToAddress("0x6c"), client) - Require(t, err, "could not deploy contract") - - // wait for price to come to equilibrium - equilibrated := false - numTriesLeft := 20 - for !equilibrated && numTriesLeft > 0 { - // make an empty block to let the gas price update - l2info.GasPrice = new(big.Int).Mul(l2info.GasPrice, big.NewInt(2)) - TransferBalance(t, "Owner", "Owner", common.Big0, l2info, client, ctx) - - // check if the price has equilibrated - _, _, _, _, _, setPrice, err := arbGasInfo.GetPricesInWei(&bind.CallOpts{}) - Require(t, err, "could not get L2 gas price") - if gasPrice.Cmp(setPrice) == 0 { - equilibrated = true - } - numTriesLeft-- - } - if !equilibrated { - Fatal(t, "L2 gas price did not converge", gasPrice) - } - - initialBalance, err := client.BalanceAt(ctx, auth.From, nil) - Require(t, err, "could not get balance") - - // deploy a test contract - _, tx, simple, err := mocksgen.DeploySimple(&auth, client) - Require(t, err, "could not deploy contract") - receipt, err := EnsureTxSucceeded(ctx, client, tx) - Require(t, err) - - header, err := client.HeaderByNumber(ctx, receipt.BlockNumber) - Require(t, err, "could not get header") - if header.BaseFee.Cmp(gasPrice) != 0 { - Fatal(t, "Header has wrong basefee", header.BaseFee, gasPrice) - } - - balance, err := client.BalanceAt(ctx, auth.From, nil) - Require(t, err, "could not get balance") - expectedCost := receipt.GasUsed * gasPrice.Uint64() - observedCost := initialBalance.Uint64() - balance.Uint64() - if expectedCost != observedCost { - Fatal(t, "Expected deployment to cost", expectedCost, "instead of", observedCost) - } - - tx, err = simple.Increment(&auth) - Require(t, err, "failed to call Increment()") - _, err = EnsureTxSucceeded(ctx, client, tx) - Require(t, err) - - counter, err := simple.Counter(&bind.CallOpts{}) - Require(t, err, "failed to get counter") - - if counter != 1 { - Fatal(t, "Unexpected counter value", counter) - } -} - -func TestComponentEstimate(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - l2info, node, client := CreateTestL2(t, ctx) - defer node.StopAndWait() - - l1BaseFee := new(big.Int).Set(arbostypes.DefaultInitialL1BaseFee) - l2BaseFee := GetBaseFee(t, client, ctx) - - colors.PrintGrey("l1 basefee ", l1BaseFee) - colors.PrintGrey("l2 basefee ", l2BaseFee) - - userBalance := big.NewInt(1e16) - maxPriorityFeePerGas := big.NewInt(0) - maxFeePerGas := arbmath.BigMulByUfrac(l2BaseFee, 3, 2) - - l2info.GenerateAccount("User") - TransferBalance(t, "Owner", "User", userBalance, l2info, client, ctx) - - from := l2info.GetAddress("User") - to := testhelpers.RandomAddress() - gas := uint64(100000000) - calldata := []byte{0x00, 0x12} - value := big.NewInt(4096) - - nodeAbi, err := node_interfacegen.NodeInterfaceMetaData.GetAbi() - Require(t, err) - - nodeMethod := nodeAbi.Methods["gasEstimateComponents"] - estimateCalldata := append([]byte{}, nodeMethod.ID...) - packed, err := nodeMethod.Inputs.Pack(to, false, calldata) - Require(t, err) - estimateCalldata = append(estimateCalldata, packed...) - - msg := ethereum.CallMsg{ - From: from, - To: &types.NodeInterfaceAddress, - Gas: gas, - GasFeeCap: maxFeePerGas, - GasTipCap: maxPriorityFeePerGas, - Value: value, - Data: estimateCalldata, - } - returnData, err := client.CallContract(ctx, msg, nil) - Require(t, err) - - outputs, err := nodeMethod.Outputs.Unpack(returnData) - Require(t, err) - if len(outputs) != 4 { - Fatal(t, "expected 4 outputs from gasEstimateComponents, got", len(outputs)) - } - - gasEstimate, _ := outputs[0].(uint64) - gasEstimateForL1, _ := outputs[1].(uint64) - baseFee, _ := outputs[2].(*big.Int) - l1BaseFeeEstimate, _ := outputs[3].(*big.Int) - - execNode := getExecNode(t, node) - tx := l2info.SignTxAs("User", &types.DynamicFeeTx{ - ChainID: execNode.ArbInterface.BlockChain().Config().ChainID, - Nonce: 0, - GasTipCap: maxPriorityFeePerGas, - GasFeeCap: maxFeePerGas, - Gas: gasEstimate, - To: &to, - Value: value, - Data: calldata, - }) - - l2Estimate := gasEstimate - gasEstimateForL1 - - colors.PrintBlue("Est. ", gasEstimate, " - ", gasEstimateForL1, " = ", l2Estimate) - - if !arbmath.BigEquals(l1BaseFeeEstimate, l1BaseFee) { - Fatal(t, l1BaseFeeEstimate, l1BaseFee) - } - if !arbmath.BigEquals(baseFee, l2BaseFee) { - Fatal(t, baseFee, l2BaseFee.Uint64()) - } - - Require(t, client.SendTransaction(ctx, tx)) - receipt, err := EnsureTxSucceeded(ctx, client, tx) - Require(t, err) - - l2Used := receipt.GasUsed - receipt.GasUsedForL1 - colors.PrintMint("True ", receipt.GasUsed, " - ", receipt.GasUsedForL1, " = ", l2Used) - - if l2Estimate != l2Used { - Fatal(t, l2Estimate, l2Used) - } -} - func callFindBatchContainig(t *testing.T, ctx context.Context, client *ethclient.Client, nodeAbi *abi.ABI, blockNum uint64) uint64 { findBatch := nodeAbi.Methods["findBatchContainingBlock"] callData := append([]byte{}, findBatch.ID...) @@ -338,39 +134,6 @@ func TestFindBatch(t *testing.T) { } } -func TestDisableL1Charging(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - _, node, client := CreateTestL2(t, ctx) - defer node.StopAndWait() - addr := common.HexToAddress("0x12345678") - - gasWithL1Charging, err := client.EstimateGas(ctx, ethereum.CallMsg{To: &addr}) - Require(t, err) - - gasWithoutL1Charging, err := client.EstimateGas(ctx, ethereum.CallMsg{To: &addr, SkipL1Charging: true}) - Require(t, err) - - if gasWithL1Charging <= gasWithoutL1Charging { - Fatal(t, "SkipL1Charging didn't disable L1 charging") - } - if gasWithoutL1Charging != params.TxGas { - Fatal(t, "Incorrect gas estimate with disabled L1 charging") - } - - _, err = client.CallContract(ctx, ethereum.CallMsg{To: &addr, Gas: gasWithL1Charging}, nil) - Require(t, err) - - _, err = client.CallContract(ctx, ethereum.CallMsg{To: &addr, Gas: gasWithoutL1Charging}, nil) - if err == nil { - Fatal(t, "CallContract passed with insufficient gas") - } - - _, err = client.CallContract(ctx, ethereum.CallMsg{To: &addr, Gas: gasWithoutL1Charging, SkipL1Charging: true}, nil) - Require(t, err) -} - func TestL2BlockRangeForL1(t *testing.T) { t.Parallel() ctx, cancel := context.WithCancel(context.Background()) From f804c7ff9bee496ea8e7185c15805919f36b7b93 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Fri, 6 Oct 2023 18:57:40 -0600 Subject: [PATCH 19/34] minor edits and renames --- arbnode/inbox_tracker.go | 2 +- arbnode/node.go | 4 ++-- execution/gethexec/executionengine.go | 2 +- execution/gethexec/node.go | 17 +++++++++-------- execution/interface.go | 2 +- execution/nodeInterface/NodeInterface.go | 2 +- staker/l1_validator.go | 2 +- staker/stateless_block_validator.go | 4 ++-- 8 files changed, 18 insertions(+), 17 deletions(-) diff --git a/arbnode/inbox_tracker.go b/arbnode/inbox_tracker.go index 0fc5b11486..09aee52d0a 100644 --- a/arbnode/inbox_tracker.go +++ b/arbnode/inbox_tracker.go @@ -218,7 +218,7 @@ func (t *InboxTracker) GetBatchCount() (uint64, error) { return count, nil } -func (t *InboxTracker) FindL1BatchForMessage(pos arbutil.MessageIndex) (uint64, error) { +func (t *InboxTracker) FindInboxBatchContainingMessage(pos arbutil.MessageIndex) (uint64, error) { batchCount, err := t.GetBatchCount() if err != nil { return 0, err diff --git a/arbnode/node.go b/arbnode/node.go index 98200163d8..1410ff1adc 100644 --- a/arbnode/node.go +++ b/arbnode/node.go @@ -1115,8 +1115,8 @@ func (n *Node) FetchBatch(ctx context.Context, batchNum uint64) ([]byte, error) return n.InboxReader.GetSequencerMessageBytes(ctx, batchNum) } -func (n *Node) FindL1BatchForMessage(message arbutil.MessageIndex) (uint64, error) { - return n.InboxTracker.FindL1BatchForMessage(message) +func (n *Node) FindInboxBatchContainingMessage(message arbutil.MessageIndex) (uint64, error) { + return n.InboxTracker.FindInboxBatchContainingMessage(message) } func (n *Node) GetBatchParentChainBlock(seqNum uint64) (uint64, error) { diff --git a/execution/gethexec/executionengine.go b/execution/gethexec/executionengine.go index 3cbc64bbdb..239f15ec9b 100644 --- a/execution/gethexec/executionengine.go +++ b/execution/gethexec/executionengine.go @@ -71,7 +71,7 @@ func (s *ExecutionEngine) EnableReorgSequencing() { s.reorgSequencing = true } -func (s *ExecutionEngine) SetTransactionStreamer(consensus execution.FullConsensusClient) { +func (s *ExecutionEngine) SetConsensus(consensus execution.FullConsensusClient) { if s.Started() { panic("trying to set transaction consensus after start") } diff --git a/execution/gethexec/node.go b/execution/gethexec/node.go index 49956ae418..94fd6d01e3 100644 --- a/execution/gethexec/node.go +++ b/execution/gethexec/node.go @@ -158,7 +158,7 @@ func CreateExecutionNode( if err != nil { return nil, err } - } else { + } else if config.Sequencer.Enable { log.Warn("sequencer enabled without l1 client") } @@ -198,14 +198,15 @@ func CreateExecutionNode( syncMon := NewSyncMonitor(execEngine) var classicOutbox *ClassicOutboxRetriever - classicMsgDb, err := stack.OpenDatabase("classic-msg", 0, 0, "", true) - if err != nil { - if l2BlockChain.Config().ArbitrumChainParams.GenesisBlockNum > 0 { + + if l2BlockChain.Config().ArbitrumChainParams.GenesisBlockNum > 0 { + classicMsgDb, err := stack.OpenDatabase("classic-msg", 0, 0, "", true) + if err != nil { log.Warn("Classic Msg Database not found", "err", err) + classicOutbox = nil + } else { + classicOutbox = NewClassicOutboxRetriever(classicMsgDb) } - classicOutbox = nil - } else { - classicOutbox = NewClassicOutboxRetriever(classicMsgDb) } apis := []rpc.API{{ @@ -379,7 +380,7 @@ func (n *ExecutionNode) ForwardTo(url string) error { } func (n *ExecutionNode) SetConsensusClient(consensus execution.FullConsensusClient) { - n.ExecEngine.SetTransactionStreamer(consensus) + n.ExecEngine.SetConsensus(consensus) n.SyncMonitor.SetConsensusInfo(consensus) } diff --git a/execution/interface.go b/execution/interface.go index c1674f0a6e..5fc5f1ddfe 100644 --- a/execution/interface.go +++ b/execution/interface.go @@ -71,7 +71,7 @@ type FullExecutionClient interface { // BatchFetcher is required for any execution node type BatchFetcher interface { FetchBatch(ctx context.Context, batchNum uint64) ([]byte, error) - FindL1BatchForMessage(message arbutil.MessageIndex) (uint64, error) + FindInboxBatchContainingMessage(message arbutil.MessageIndex) (uint64, error) GetBatchParentChainBlock(seqNum uint64) (uint64, error) } diff --git a/execution/nodeInterface/NodeInterface.go b/execution/nodeInterface/NodeInterface.go index 96989f4408..f85865f170 100644 --- a/execution/nodeInterface/NodeInterface.go +++ b/execution/nodeInterface/NodeInterface.go @@ -65,7 +65,7 @@ func (n NodeInterface) FindBatchContainingBlock(c ctx, evm mech, blockNum uint64 if fetcher == nil { return 0, errors.New("batch fetcher not set") } - batch, err := fetcher.FindL1BatchForMessage(msgIndex) + batch, err := fetcher.FindInboxBatchContainingMessage(msgIndex) return batch, err } diff --git a/staker/l1_validator.go b/staker/l1_validator.go index d2b6f57d80..ee24fc49d5 100644 --- a/staker/l1_validator.go +++ b/staker/l1_validator.go @@ -336,7 +336,7 @@ func (v *L1Validator) generateNodeAction( batchNum = localBatchCount - 1 validatedCount = messageCount } else { - batchNum, err = v.inboxTracker.FindL1BatchForMessage(validatedCount - 1) + batchNum, err = v.inboxTracker.FindInboxBatchContainingMessage(validatedCount - 1) if err != nil { return nil, false, err } diff --git a/staker/stateless_block_validator.go b/staker/stateless_block_validator.go index 26a79f067f..2c7046effb 100644 --- a/staker/stateless_block_validator.go +++ b/staker/stateless_block_validator.go @@ -56,7 +56,7 @@ type InboxTrackerInterface interface { GetBatchMessageCount(seqNum uint64) (arbutil.MessageIndex, error) GetBatchAcc(seqNum uint64) (common.Hash, error) GetBatchCount() (uint64, error) - FindL1BatchForMessage(pos arbutil.MessageIndex) (uint64, error) + FindInboxBatchContainingMessage(pos arbutil.MessageIndex) (uint64, error) } type TransactionStreamerInterface interface { @@ -297,7 +297,7 @@ func (v *StatelessBlockValidator) GlobalStatePositionsAtCount(count arbutil.Mess if count == 1 { return GlobalStatePosition{}, GlobalStatePosition{1, 0}, nil } - batch, err := v.inboxTracker.FindL1BatchForMessage(count - 1) + batch, err := v.inboxTracker.FindInboxBatchContainingMessage(count - 1) if err != nil { return GlobalStatePosition{}, GlobalStatePosition{}, err } From 8505119d4920b377ffff8686c66c6dddd1794ee1 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Wed, 1 Nov 2023 09:37:40 -0600 Subject: [PATCH 20/34] fix merge errors --- execution/gethexec/node.go | 1 - 1 file changed, 1 deletion(-) diff --git a/execution/gethexec/node.go b/execution/gethexec/node.go index b38ba30a7a..3154eb8bce 100644 --- a/execution/gethexec/node.go +++ b/execution/gethexec/node.go @@ -85,7 +85,6 @@ func ConfigAddOptions(prefix string, f *flag.FlagSet) { } var ConfigDefault = Config{ - ParentChainReader: headerreader.DefaultConfig, RPC: arbitrum.DefaultConfig, Sequencer: DefaultSequencerConfig, ParentChainReader: headerreader.DefaultConfig, From a3e275bfbbaaa1933197ba9a204d5745db405fd5 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Wed, 20 Mar 2024 10:44:41 -0600 Subject: [PATCH 21/34] wait for validation before safe or final --- arbnode/node.go | 9 ++++++- execution/gethexec/node.go | 4 ++- execution/gethexec/sync_monitor.go | 40 ++++++++++++++++++++++++++++-- execution/interface.go | 1 + 4 files changed, 50 insertions(+), 4 deletions(-) diff --git a/arbnode/node.go b/arbnode/node.go index b77715a0e3..dd379ba483 100644 --- a/arbnode/node.go +++ b/arbnode/node.go @@ -953,7 +953,7 @@ func (n *Node) StopAndWait() { } } -func (n *Node) FetchBatch(ctx context.Context, batchNum uint64) ([]byte, error) { +func (n *Node) FetchBatch(ctx context.Context, batchNum uint64) ([]byte, common.Hash, error) { return n.InboxReader.GetSequencerMessageBytes(ctx, batchNum) } @@ -989,3 +989,10 @@ func (n *Node) WriteMessageFromSequencer(pos arbutil.MessageIndex, msgWithMeta a func (n *Node) ExpectChosenSequencer() error { return n.TxStreamer.ExpectChosenSequencer() } + +func (n *Node) ValidatedMessageCount() (arbutil.MessageIndex, error) { + if n.BlockValidator == nil { + return 0, errors.New("validator not set up") + } + return n.BlockValidator.GetValidated(), nil +} diff --git a/execution/gethexec/node.go b/execution/gethexec/node.go index bb438c2119..d8ec944a30 100644 --- a/execution/gethexec/node.go +++ b/execution/gethexec/node.go @@ -51,6 +51,7 @@ type Config struct { TxLookupLimit uint64 `koanf:"tx-lookup-limit"` Dangerous DangerousConfig `koanf:"dangerous"` EnablePrefetchBlock bool `koanf:"enable-prefetch-block"` + SyncMonitor SyncMonitorConfig `koanf:"sync-monitor"` forwardingTarget string } @@ -83,6 +84,7 @@ func ConfigAddOptions(prefix string, f *flag.FlagSet) { AddOptionsForNodeForwarderConfig(prefix+".forwarder", f) TxPreCheckerConfigAddOptions(prefix+".tx-pre-checker", f) CachingConfigAddOptions(prefix+".caching", f) + SyncMonitorConfigAddOptions(prefix+"sync-monitor", f) f.Uint64(prefix+".tx-lookup-limit", ConfigDefault.TxLookupLimit, "retain the ability to lookup transactions by hash for the past N blocks (0 = all blocks)") DangerousConfigAddOptions(prefix+".dangerous", f) f.Bool(prefix+".enable-prefetch-block", ConfigDefault.EnablePrefetchBlock, "enable prefetching of blocks") @@ -210,7 +212,7 @@ func CreateExecutionNode( return nil, err } - syncMon := NewSyncMonitor(execEngine) + syncMon := NewSyncMonitor(&config.SyncMonitor, execEngine) var classicOutbox *ClassicOutboxRetriever diff --git a/execution/gethexec/sync_monitor.go b/execution/gethexec/sync_monitor.go index e790d9485a..22b8f4f91c 100644 --- a/execution/gethexec/sync_monitor.go +++ b/execution/gethexec/sync_monitor.go @@ -5,16 +5,34 @@ import ( "github.com/offchainlabs/nitro/execution" "github.com/pkg/errors" + flag "github.com/spf13/pflag" ) +type SyncMonitorConfig struct { + SafeBlockWaitForBlockValidator bool `koanf:"safe-block-wait-for-block-validator"` + FinalizedBlockWaitForBlockValidator bool `koanf:"finalized-block-wait-for-block-validator"` +} + +var DefaultSyncMonitorConfig = SyncMonitorConfig{ + SafeBlockWaitForBlockValidator: false, + FinalizedBlockWaitForBlockValidator: false, +} + +func SyncMonitorConfigAddOptions(prefix string, f *flag.FlagSet) { + f.Bool(prefix+".safe-block-wait-for-block-validator", DefaultSyncMonitorConfig.SafeBlockWaitForBlockValidator, "wait for block validator to complete before returning safe block number") + f.Bool(prefix+".finalized-block-wait-for-block-validator", DefaultSyncMonitorConfig.FinalizedBlockWaitForBlockValidator, "wait for block validator to complete before returning finalized block number") +} + type SyncMonitor struct { + config *SyncMonitorConfig consensus execution.ConsensusInfo exec *ExecutionEngine } -func NewSyncMonitor(exec *ExecutionEngine) *SyncMonitor { +func NewSyncMonitor(config *SyncMonitorConfig, exec *ExecutionEngine) *SyncMonitor { return &SyncMonitor{ - exec: exec, + config: config, + exec: exec, } } @@ -45,6 +63,15 @@ func (s *SyncMonitor) SafeBlockNumber(ctx context.Context) (uint64, error) { if err != nil { return 0, err } + if s.config.SafeBlockWaitForBlockValidator { + latestValidatedCount, err := s.consensus.ValidatedMessageCount() + if err != nil { + return 0, err + } + if msg > latestValidatedCount { + msg = latestValidatedCount + } + } block := s.exec.MessageIndexToBlockNumber(msg - 1) return block, nil } @@ -57,6 +84,15 @@ func (s *SyncMonitor) FinalizedBlockNumber(ctx context.Context) (uint64, error) if err != nil { return 0, err } + if s.config.FinalizedBlockWaitForBlockValidator { + latestValidatedCount, err := s.consensus.ValidatedMessageCount() + if err != nil { + return 0, err + } + if msg > latestValidatedCount { + msg = latestValidatedCount + } + } block := s.exec.MessageIndexToBlockNumber(msg - 1) return block, nil } diff --git a/execution/interface.go b/execution/interface.go index 6df28bfdd0..9aaf34e9ad 100644 --- a/execution/interface.go +++ b/execution/interface.go @@ -84,6 +84,7 @@ type ConsensusInfo interface { // TODO: switch from pulling to pushing safe/finalized GetSafeMsgCount(ctx context.Context) (arbutil.MessageIndex, error) GetFinalizedMsgCount(ctx context.Context) (arbutil.MessageIndex, error) + ValidatedMessageCount() (arbutil.MessageIndex, error) } type ConsensusSequencer interface { From 0ca597b65acbe92cfc1737cb3ee0cf0cadd5cf73 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Wed, 20 Mar 2024 11:02:13 -0600 Subject: [PATCH 22/34] fix more merge errors --- arbnode/sync_monitor.go | 2 -- arbnode/transaction_streamer.go | 15 --------------- execution/gethexec/node.go | 2 +- system_tests/full_challenge_impl_test.go | 2 +- 4 files changed, 2 insertions(+), 19 deletions(-) diff --git a/arbnode/sync_monitor.go b/arbnode/sync_monitor.go index b91991a182..3641a40790 100644 --- a/arbnode/sync_monitor.go +++ b/arbnode/sync_monitor.go @@ -44,8 +44,6 @@ var TestSyncMonitorConfig = SyncMonitorConfig{ func SyncMonitorConfigAddOptions(prefix string, f *flag.FlagSet) { f.Duration(prefix+".msg-lag", DefaultSyncMonitorConfig.MsgLag, "allowed msg lag while still considered in sync") - f.Bool(prefix+".safe-block-wait-for-block-validator", DefaultSyncMonitorConfig.SafeBlockWaitForBlockValidator, "wait for block validator to complete before returning safe block number") - f.Bool(prefix+".finalized-block-wait-for-block-validator", DefaultSyncMonitorConfig.FinalizedBlockWaitForBlockValidator, "wait for block validator to complete before returning finalized block number") } func (s *SyncMonitor) Initialize(inboxReader *InboxReader, txStreamer *TransactionStreamer, coordinator *SeqCoordinator) { diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index c265035afa..fa161db6c5 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -441,21 +441,6 @@ func (s *TransactionStreamer) FeedPendingMessageCount() arbutil.MessageIndex { return arbutil.MessageIndex(pos + uint64(len(s.broadcasterQueuedMessages))) } -func (s *TransactionStreamer) FeedPendingMessageCount() arbutil.MessageIndex { - pos := atomic.LoadUint64(&s.broadcasterQueuedMessagesPos) - if pos == 0 { - return 0 - } - - s.insertionMutex.Lock() - defer s.insertionMutex.Unlock() - pos = atomic.LoadUint64(&s.broadcasterQueuedMessagesPos) - if pos == 0 { - return 0 - } - return arbutil.MessageIndex(pos + uint64(len(s.broadcasterQueuedMessages))) -} - func (s *TransactionStreamer) AddBroadcastMessages(feedMessages []*m.BroadcastFeedMessage) error { if len(feedMessages) == 0 { return nil diff --git a/execution/gethexec/node.go b/execution/gethexec/node.go index d8ec944a30..5606b5ca35 100644 --- a/execution/gethexec/node.go +++ b/execution/gethexec/node.go @@ -84,7 +84,7 @@ func ConfigAddOptions(prefix string, f *flag.FlagSet) { AddOptionsForNodeForwarderConfig(prefix+".forwarder", f) TxPreCheckerConfigAddOptions(prefix+".tx-pre-checker", f) CachingConfigAddOptions(prefix+".caching", f) - SyncMonitorConfigAddOptions(prefix+"sync-monitor", f) + SyncMonitorConfigAddOptions(prefix+".sync-monitor", f) f.Uint64(prefix+".tx-lookup-limit", ConfigDefault.TxLookupLimit, "retain the ability to lookup transactions by hash for the past N blocks (0 = all blocks)") DangerousConfigAddOptions(prefix+".dangerous", f) f.Bool(prefix+".enable-prefetch-block", ConfigDefault.EnablePrefetchBlock, "enable prefetching of blocks") diff --git a/system_tests/full_challenge_impl_test.go b/system_tests/full_challenge_impl_test.go index 220179037b..7a47149244 100644 --- a/system_tests/full_challenge_impl_test.go +++ b/system_tests/full_challenge_impl_test.go @@ -242,7 +242,7 @@ func createL2Nodes(t *testing.T, ctx context.Context, conf *arbnode.Config, chai _, stack, l2ChainDb, l2ArbDb, l2Blockchain := createL2BlockChainWithStackConfig(t, l2info, "", chainConfig, initMsg, nil, nil) execNode, err := gethexec.CreateExecutionNode(ctx, stack, l2ChainDb, l2Blockchain, l1Client, gethexec.ConfigDefaultTest) Require(t, err) - consensusNode, err := arbnode.CreateNode(ctx, stack, execNode, l2ArbDb, NewFetcherFromConfig(conf), chainConfig, l1Client, rollupAddresses, txOpts, txOpts, signer, fatalErrChan) + consensusNode, err := arbnode.CreateNode(ctx, stack, execNode, l2ArbDb, NewFetcherFromConfig(conf), chainConfig, l1Client, rollupAddresses, txOpts, txOpts, signer, fatalErrChan, big.NewInt(1337), nil) Require(t, err) return consensusNode, execNode From 0b7a5901d7b22fff2d593060b4f297c0f597d901 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Wed, 20 Mar 2024 12:20:04 -0600 Subject: [PATCH 23/34] support counting validations for blocks not yet posted to parent --- arbnode/inbox_tracker.go | 24 ++++++---- arbnode/node.go | 2 +- execution/interface.go | 2 +- execution/nodeInterface/NodeInterface.go | 61 +++++++++++++++++------- staker/l1_validator.go | 6 ++- staker/stateless_block_validator.go | 7 ++- 6 files changed, 71 insertions(+), 31 deletions(-) diff --git a/arbnode/inbox_tracker.go b/arbnode/inbox_tracker.go index 61c31e6c7f..4d892ac4cc 100644 --- a/arbnode/inbox_tracker.go +++ b/arbnode/inbox_tracker.go @@ -228,13 +228,20 @@ func (t *InboxTracker) GetBatchCount() (uint64, error) { return count, nil } -func (t *InboxTracker) FindInboxBatchContainingMessage(pos arbutil.MessageIndex) (uint64, error) { +func (t *InboxTracker) FindInboxBatchContainingMessage(pos arbutil.MessageIndex) (uint64, bool, error) { batchCount, err := t.GetBatchCount() if err != nil { - return 0, err + return 0, false, err } low := uint64(0) high := batchCount - 1 + lastBatchMessageCount, err := t.GetBatchMessageCount(high) + if err != nil { + return 0, false, err + } + if lastBatchMessageCount <= pos { + return 0, false, nil + } // Iteration preconditions: // - high >= low // - msgCount(low - 1) <= pos implies low <= target @@ -245,23 +252,24 @@ func (t *InboxTracker) FindInboxBatchContainingMessage(pos arbutil.MessageIndex) mid := (low + high) / 2 count, err := t.GetBatchMessageCount(mid) if err != nil { - return 0, err + return 0, false, err } if count < pos { // Must narrow as mid >= low, therefore mid + 1 > low, therefore newLow > oldLow // Keeps low precondition as msgCount(mid) < pos low = mid + 1 } else if count == pos { - return mid + 1, err + return mid + 1, true, nil } else if count == pos+1 || mid == low { // implied: count > pos - return mid, nil - } else { // implied: count > pos + 1 - // Must narrow as mid < high, therefore newHigh < lowHigh + return mid, true, nil + } else { + // implied: count > pos + 1 + // Must narrow as mid < high, therefore newHigh < oldHigh // Keeps high precondition as msgCount(mid) > pos high = mid } if high == low { - return high, err + return high, true, nil } } } diff --git a/arbnode/node.go b/arbnode/node.go index dd379ba483..7086566408 100644 --- a/arbnode/node.go +++ b/arbnode/node.go @@ -957,7 +957,7 @@ func (n *Node) FetchBatch(ctx context.Context, batchNum uint64) ([]byte, common. return n.InboxReader.GetSequencerMessageBytes(ctx, batchNum) } -func (n *Node) FindInboxBatchContainingMessage(message arbutil.MessageIndex) (uint64, error) { +func (n *Node) FindInboxBatchContainingMessage(message arbutil.MessageIndex) (uint64, bool, error) { return n.InboxTracker.FindInboxBatchContainingMessage(message) } diff --git a/execution/interface.go b/execution/interface.go index 9aaf34e9ad..b7ffce7857 100644 --- a/execution/interface.go +++ b/execution/interface.go @@ -73,7 +73,7 @@ type FullExecutionClient interface { // BatchFetcher is required for any execution node type BatchFetcher interface { FetchBatch(ctx context.Context, batchNum uint64) ([]byte, common.Hash, error) - FindInboxBatchContainingMessage(message arbutil.MessageIndex) (uint64, error) + FindInboxBatchContainingMessage(message arbutil.MessageIndex) (uint64, bool, error) GetBatchParentChainBlock(seqNum uint64) (uint64, error) } diff --git a/execution/nodeInterface/NodeInterface.go b/execution/nodeInterface/NodeInterface.go index 9c072c1ebc..d1fb102ad2 100644 --- a/execution/nodeInterface/NodeInterface.go +++ b/execution/nodeInterface/NodeInterface.go @@ -24,6 +24,7 @@ import ( "github.com/offchainlabs/nitro/arbos/l1pricing" "github.com/offchainlabs/nitro/arbos/retryables" "github.com/offchainlabs/nitro/arbos/util" + "github.com/offchainlabs/nitro/arbutil" "github.com/offchainlabs/nitro/solgen/go/node_interfacegen" "github.com/offchainlabs/nitro/util/arbmath" "github.com/offchainlabs/nitro/util/merkletree" @@ -55,21 +56,51 @@ func (n NodeInterface) NitroGenesisBlock(c ctx) (huge, error) { return arbmath.UintToBig(block), nil } -func (n NodeInterface) FindBatchContainingBlock(c ctx, evm mech, blockNum uint64) (uint64, error) { +// returns 0 if blockNumbver is behind genesis +func (n NodeInterface) blockNumToMessageIndex(blockNum uint64) (arbutil.MessageIndex, bool, error) { node, err := gethExecFromNodeInterfaceBackend(n.backend) if err != nil { - return 0, err + return 0, false, err + } + blockchain, err := blockchainFromNodeInterfaceBackend(n.backend) + if err != nil { + return 0, false, err + } + if blockNum < blockchain.Config().ArbitrumChainParams.GenesisBlockNum { + return 0, true, nil } msgIndex, err := node.ExecEngine.BlockNumberToMessageIndex(blockNum) if err != nil { - return 0, err + return 0, false, err + } + return msgIndex, true, nil +} + +func (n NodeInterface) msgNumToInboxBatch(msgIndex arbutil.MessageIndex) (uint64, bool, error) { + node, err := gethExecFromNodeInterfaceBackend(n.backend) + if err != nil { + return 0, false, err } fetcher := node.ExecEngine.GetBatchFetcher() if fetcher == nil { - return 0, errors.New("batch fetcher not set") + return 0, false, errors.New("batch fetcher not set") } - batch, err := fetcher.FindInboxBatchContainingMessage(msgIndex) - return batch, err + return fetcher.FindInboxBatchContainingMessage(msgIndex) +} + +func (n NodeInterface) FindBatchContainingBlock(c ctx, evm mech, blockNum uint64) (uint64, error) { + msgIndex, found, err := n.blockNumToMessageIndex(blockNum) + if err != nil { + return 0, err + } + if !found { + return 0, fmt.Errorf("block %v is part of genesis", blockNum) + } + res, found, err := n.msgNumToInboxBatch(msgIndex) + if err == nil && !found { + return 0, errors.New("block not yet found on any block") + } + return res, err } func (n NodeInterface) GetL1Confirmations(c ctx, evm mech, blockHash bytes32) (uint64, error) { @@ -85,22 +116,16 @@ func (n NodeInterface) GetL1Confirmations(c ctx, evm mech, blockHash bytes32) (u if header == nil { return 0, errors.New("unknown block hash") } + blockNum := header.Number.Uint64() - l2BlockNum := header.Number.Uint64() - batchNum, err := n.FindBatchContainingBlock(c, evm, l2BlockNum) + msgNum, _, err := n.blockNumToMessageIndex(blockNum) if err != nil { return 0, err } - // TODO - // if err != nil { - // if errors.Is(err, blockInGenesis) { - // batch = 0 - // } else if errors.Is(err, blockAfterLatestBatch) { - // return 0, nil - // } else { - // return 0, err - // } - // } + batchNum, found, err := n.msgNumToInboxBatch(msgNum) + if err != nil || !found { + return 0, err + } parentChainBlockNum, err := node.ExecEngine.GetBatchFetcher().GetBatchParentChainBlock(batchNum) if err != nil { return 0, err diff --git a/staker/l1_validator.go b/staker/l1_validator.go index 72184f2167..56389ae80e 100644 --- a/staker/l1_validator.go +++ b/staker/l1_validator.go @@ -339,10 +339,14 @@ func (v *L1Validator) generateNodeAction( batchNum = localBatchCount - 1 validatedCount = messageCount } else { - batchNum, err = v.inboxTracker.FindInboxBatchContainingMessage(validatedCount - 1) + var found bool + batchNum, found, err = v.inboxTracker.FindInboxBatchContainingMessage(validatedCount - 1) if err != nil { return nil, false, err } + if !found { + return nil, false, errors.New("batch not found on L1") + } } execResult, err := v.txStreamer.ResultAtCount(validatedCount) if err != nil { diff --git a/staker/stateless_block_validator.go b/staker/stateless_block_validator.go index f25573e138..abfc08ec33 100644 --- a/staker/stateless_block_validator.go +++ b/staker/stateless_block_validator.go @@ -56,7 +56,7 @@ type InboxTrackerInterface interface { GetBatchMessageCount(seqNum uint64) (arbutil.MessageIndex, error) GetBatchAcc(seqNum uint64) (common.Hash, error) GetBatchCount() (uint64, error) - FindInboxBatchContainingMessage(pos arbutil.MessageIndex) (uint64, error) + FindInboxBatchContainingMessage(pos arbutil.MessageIndex) (uint64, bool, error) } type TransactionStreamerInterface interface { @@ -320,10 +320,13 @@ func (v *StatelessBlockValidator) GlobalStatePositionsAtCount(count arbutil.Mess if count == 1 { return GlobalStatePosition{}, GlobalStatePosition{1, 0}, nil } - batch, err := v.inboxTracker.FindInboxBatchContainingMessage(count - 1) + batch, found, err := v.inboxTracker.FindInboxBatchContainingMessage(count - 1) if err != nil { return GlobalStatePosition{}, GlobalStatePosition{}, err } + if !found { + return GlobalStatePosition{}, GlobalStatePosition{}, errors.New("batch not found on L1 yet") + } return GlobalStatePositionsAtCount(v.inboxTracker, count, batch) } From 69758dfb076942914e2e4e18731c8e683dae4160 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Wed, 20 Mar 2024 13:54:02 -0600 Subject: [PATCH 24/34] NodeInterface: update comments --- execution/nodeInterface/NodeInterface.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/execution/nodeInterface/NodeInterface.go b/execution/nodeInterface/NodeInterface.go index d1fb102ad2..b16d56468d 100644 --- a/execution/nodeInterface/NodeInterface.go +++ b/execution/nodeInterface/NodeInterface.go @@ -56,7 +56,7 @@ func (n NodeInterface) NitroGenesisBlock(c ctx) (huge, error) { return arbmath.UintToBig(block), nil } -// returns 0 if blockNumbver is behind genesis +// bool will be false but no error if behind genesis func (n NodeInterface) blockNumToMessageIndex(blockNum uint64) (arbutil.MessageIndex, bool, error) { node, err := gethExecFromNodeInterfaceBackend(n.backend) if err != nil { @@ -118,10 +118,12 @@ func (n NodeInterface) GetL1Confirmations(c ctx, evm mech, blockHash bytes32) (u } blockNum := header.Number.Uint64() + // blocks behind genesis are treated as belonging to batch 0 msgNum, _, err := n.blockNumToMessageIndex(blockNum) if err != nil { return 0, err } + // batches not yet posted have 0 confirmations but no error batchNum, found, err := n.msgNumToInboxBatch(msgNum) if err != nil || !found { return 0, err From 7e82acf04ce19977667aab3fb7d49c39e00f7d1a Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Wed, 20 Mar 2024 15:35:48 -0600 Subject: [PATCH 25/34] remove seqinbox_test from race testing --- system_tests/seqinbox_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/system_tests/seqinbox_test.go b/system_tests/seqinbox_test.go index e00bda8e84..d4ba82313a 100644 --- a/system_tests/seqinbox_test.go +++ b/system_tests/seqinbox_test.go @@ -1,6 +1,10 @@ // Copyright 2021-2022, Offchain Labs, Inc. // For license information, see https://github.com/nitro/blob/master/LICENSE +// race detection makes things slow and miss timeouts +//go:build !race +// +build !race + package arbtest import ( From b49b660ec0002d2a00a5d1111c889b35eb90c8ae Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Wed, 20 Mar 2024 16:07:26 -0600 Subject: [PATCH 26/34] Revert "remove seqinbox_test from race testing" This reverts commit 7e82acf04ce19977667aab3fb7d49c39e00f7d1a. --- system_tests/seqinbox_test.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/system_tests/seqinbox_test.go b/system_tests/seqinbox_test.go index d4ba82313a..e00bda8e84 100644 --- a/system_tests/seqinbox_test.go +++ b/system_tests/seqinbox_test.go @@ -1,10 +1,6 @@ // Copyright 2021-2022, Offchain Labs, Inc. // For license information, see https://github.com/nitro/blob/master/LICENSE -// race detection makes things slow and miss timeouts -//go:build !race -// +build !race - package arbtest import ( From db76dff27b96d0dd03ca5c73d76ac1e7f5732106 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Wed, 20 Mar 2024 16:08:30 -0600 Subject: [PATCH 27/34] remove nodeInterface from race tests --- system_tests/nodeinterface_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/system_tests/nodeinterface_test.go b/system_tests/nodeinterface_test.go index 77f30649d0..e5cc41adc5 100644 --- a/system_tests/nodeinterface_test.go +++ b/system_tests/nodeinterface_test.go @@ -1,6 +1,10 @@ // Copyright 2021-2022, Offchain Labs, Inc. // For license information, see https://github.com/nitro/blob/master/LICENSE +// race detection makes things slow and miss timeouts +//go:build !race +// +build !race + package arbtest import ( From 157633b9a03f67b2df1c59a00faae572da696af0 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Mon, 25 Mar 2024 12:21:41 -0600 Subject: [PATCH 28/34] fix small PR comments --- execution/gethexec/node.go | 1 - execution/nodeInterface/NodeInterface.go | 7 +++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/execution/gethexec/node.go b/execution/gethexec/node.go index 5606b5ca35..2053f86aa3 100644 --- a/execution/gethexec/node.go +++ b/execution/gethexec/node.go @@ -122,7 +122,6 @@ func ConfigDefaultTest() *Config { config.Sequencer = TestSequencerConfig config.ParentChainReader = headerreader.TestConfig config.ForwardingTarget = "null" - config.ParentChainReader = headerreader.TestConfig _ = config.Validate() diff --git a/execution/nodeInterface/NodeInterface.go b/execution/nodeInterface/NodeInterface.go index b16d56468d..7e524731d0 100644 --- a/execution/nodeInterface/NodeInterface.go +++ b/execution/nodeInterface/NodeInterface.go @@ -98,7 +98,7 @@ func (n NodeInterface) FindBatchContainingBlock(c ctx, evm mech, blockNum uint64 } res, found, err := n.msgNumToInboxBatch(msgIndex) if err == nil && !found { - return 0, errors.New("block not yet found on any block") + return 0, errors.New("block not yet found on any batch") } return res, err } @@ -125,9 +125,12 @@ func (n NodeInterface) GetL1Confirmations(c ctx, evm mech, blockHash bytes32) (u } // batches not yet posted have 0 confirmations but no error batchNum, found, err := n.msgNumToInboxBatch(msgNum) - if err != nil || !found { + if err != nil { return 0, err } + if !found { + return 0, nil + } parentChainBlockNum, err := node.ExecEngine.GetBatchFetcher().GetBatchParentChainBlock(batchNum) if err != nil { return 0, err From 9777dd036e7a9b717eb63ae3245bcdcc43b0e7cf Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Mon, 25 Mar 2024 12:54:33 -0600 Subject: [PATCH 29/34] nodeinterface_test: use bindAPI --- system_tests/nodeinterface_test.go | 61 ++++-------------------------- 1 file changed, 7 insertions(+), 54 deletions(-) diff --git a/system_tests/nodeinterface_test.go b/system_tests/nodeinterface_test.go index e5cc41adc5..b692af6e30 100644 --- a/system_tests/nodeinterface_test.go +++ b/system_tests/nodeinterface_test.go @@ -12,66 +12,15 @@ import ( "math/big" "testing" - "github.com/ethereum/go-ethereum" - "github.com/ethereum/go-ethereum/accounts/abi" "github.com/ethereum/go-ethereum/accounts/abi/bind" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" - "github.com/ethereum/go-ethereum/ethclient" "github.com/ethereum/go-ethereum/params" "github.com/offchainlabs/nitro/arbnode" "github.com/offchainlabs/nitro/arbos/util" "github.com/offchainlabs/nitro/solgen/go/node_interfacegen" ) -func callFindBatchContainig(t *testing.T, ctx context.Context, client *ethclient.Client, nodeAbi *abi.ABI, blockNum uint64) uint64 { - findBatch := nodeAbi.Methods["findBatchContainingBlock"] - callData := append([]byte{}, findBatch.ID...) - packed, err := findBatch.Inputs.Pack(blockNum) - Require(t, err) - callData = append(callData, packed...) - msg := ethereum.CallMsg{ - To: &types.NodeInterfaceAddress, - Data: callData, - } - returnData, err := client.CallContract(ctx, msg, nil) - Require(t, err) - outputs, err := findBatch.Outputs.Unpack(returnData) - Require(t, err) - if len(outputs) != 1 { - Fatal(t, "expected 1 output from findBatchContainingBlock, got", len(outputs)) - } - gotBatchNum, ok := outputs[0].(uint64) - if !ok { - Fatal(t, "bad output from findBatchContainingBlock") - } - return gotBatchNum -} - -func callGetL1Confirmations(t *testing.T, ctx context.Context, client *ethclient.Client, nodeAbi *abi.ABI, blockHash common.Hash) uint64 { - getConfirmations := nodeAbi.Methods["getL1Confirmations"] - callData := append([]byte{}, getConfirmations.ID...) - packed, err := getConfirmations.Inputs.Pack(blockHash) - Require(t, err) - callData = append(callData, packed...) - msg := ethereum.CallMsg{ - To: &types.NodeInterfaceAddress, - Data: callData, - } - returnData, err := client.CallContract(ctx, msg, nil) - Require(t, err) - outputs, err := getConfirmations.Outputs.Unpack(returnData) - Require(t, err) - if len(outputs) != 1 { - Fatal(t, "expected 1 output from findBatchContainingBlock, got", len(outputs)) - } - confirmations, ok := outputs[0].(uint64) - if !ok { - Fatal(t, "bad output from findBatchContainingBlock") - } - return confirmations -} - func TestFindBatch(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -94,6 +43,8 @@ func TestFindBatch(t *testing.T) { bridgeAddr, seqInbox, seqInboxAddr := setupSequencerInboxStub(ctx, t, l1Info, l1Backend, chainConfig) + callOpts := bind.CallOpts{Context: ctx} + rollupAddresses.Bridge = bridgeAddr rollupAddresses.SequencerInbox = seqInboxAddr l2Info := NewArbTestInfo(t, chainConfig.ChainID) @@ -102,7 +53,7 @@ func TestFindBatch(t *testing.T) { Require(t, err) l2Client := ClientForStack(t, consensus.Stack) - nodeAbi, err := node_interfacegen.NodeInterfaceMetaData.GetAbi() + nodeInterface, err := node_interfacegen.NewNodeInterface(types.NodeInterfaceAddress, l2Client) Require(t, err) sequencerTxOpts := l1Info.GetDefaultTransactOpts("sequencer", ctx) @@ -112,7 +63,8 @@ func TestFindBatch(t *testing.T) { makeBatch(t, consensus, l2Info, l1Backend, &sequencerTxOpts, seqInbox, seqInboxAddr, -1) for blockNum := uint64(0); blockNum < uint64(makeBatch_MsgsPerBatch)*3; blockNum++ { - gotBatchNum := callFindBatchContainig(t, ctx, l2Client, nodeAbi, blockNum) + gotBatchNum, err := nodeInterface.FindBatchContainingBlock(&callOpts, blockNum) + Require(t, err) expBatchNum := uint64(0) if blockNum > 0 { expBatchNum = 1 + (blockNum-1)/uint64(makeBatch_MsgsPerBatch) @@ -128,7 +80,8 @@ func TestFindBatch(t *testing.T) { minCurrentL1Block, err := l1Backend.BlockNumber(ctx) Require(t, err) - gotConfirmations := callGetL1Confirmations(t, ctx, l2Client, nodeAbi, blockHash) + gotConfirmations, err := nodeInterface.GetL1Confirmations(&callOpts, blockHash) + Require(t, err) maxCurrentL1Block, err := l1Backend.BlockNumber(ctx) Require(t, err) From 543be5d534228d6a15a27d8e1c613dad12eb2721 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Mon, 25 Mar 2024 15:21:26 -0600 Subject: [PATCH 30/34] sync_monitor update --- arbnode/node.go | 8 ++++++-- arbnode/sync_monitor.go | 14 +++++++++----- execution/gethexec/sync_monitor.go | 19 +++++++++++++------ execution/interface.go | 3 ++- 4 files changed, 30 insertions(+), 14 deletions(-) diff --git a/arbnode/node.go b/arbnode/node.go index 7086566408..29c1071b7a 100644 --- a/arbnode/node.go +++ b/arbnode/node.go @@ -965,8 +965,12 @@ func (n *Node) GetBatchParentChainBlock(seqNum uint64) (uint64, error) { return n.InboxTracker.GetBatchParentChainBlock(seqNum) } -func (n *Node) SyncProgressMap() map[string]interface{} { - return n.SyncMonitor.SyncProgressMap() +func (n *Node) FullSyncProgressMap() map[string]interface{} { + return n.SyncMonitor.FullSyncProgressMap() +} + +func (n *Node) Synced() bool { + return n.SyncMonitor.Synced() } func (n *Node) SyncTargetMessageCount() arbutil.MessageIndex { diff --git a/arbnode/sync_monitor.go b/arbnode/sync_monitor.go index 3641a40790..d3b9a7e1c6 100644 --- a/arbnode/sync_monitor.go +++ b/arbnode/sync_monitor.go @@ -110,13 +110,9 @@ func (s *SyncMonitor) maxMessageCount() (arbutil.MessageIndex, error) { return msgCount, nil } -func (s *SyncMonitor) SyncProgressMap() map[string]interface{} { +func (s *SyncMonitor) FullSyncProgressMap() map[string]interface{} { res := make(map[string]interface{}) - if s.Synced() { - return res - } - if !s.initialized { res["err"] = "uninitialized" return res @@ -173,6 +169,14 @@ func (s *SyncMonitor) SyncProgressMap() map[string]interface{} { return res } +func (s *SyncMonitor) SyncProgressMap() map[string]interface{} { + if s.Synced() { + return make(map[string]interface{}) + } + + return s.FullSyncProgressMap() +} + func (s *SyncMonitor) Start(ctx_in context.Context) { s.StopWaiter.Start(ctx_in, s) s.CallIteratively(s.updateSyncTarget) diff --git a/execution/gethexec/sync_monitor.go b/execution/gethexec/sync_monitor.go index 22b8f4f91c..84f45083e9 100644 --- a/execution/gethexec/sync_monitor.go +++ b/execution/gethexec/sync_monitor.go @@ -36,8 +36,8 @@ func NewSyncMonitor(config *SyncMonitorConfig, exec *ExecutionEngine) *SyncMonit } } -func (s *SyncMonitor) SyncProgressMap() map[string]interface{} { - res := s.consensus.SyncProgressMap() +func (s *SyncMonitor) FullSyncProgressMap() map[string]interface{} { + res := s.consensus.FullSyncProgressMap() consensusSyncTarget := s.consensus.SyncTargetMessageCount() built, err := s.exec.HeadMessageNumber() @@ -45,16 +45,23 @@ func (s *SyncMonitor) SyncProgressMap() map[string]interface{} { res["headMsgNumberError"] = err } - if built+1 >= consensusSyncTarget && len(res) == 0 { - return res - } - res["builtBlock"] = built res["consensusSyncTarget"] = consensusSyncTarget return res } +func (s *SyncMonitor) SyncProgressMap() map[string]interface{} { + if s.consensus.Synced() { + built, err := s.exec.HeadMessageNumber() + consensusSyncTarget := s.consensus.SyncTargetMessageCount() + if err != nil && built+1 >= consensusSyncTarget { + return make(map[string]interface{}) + } + } + return s.FullSyncProgressMap() +} + func (s *SyncMonitor) SafeBlockNumber(ctx context.Context) (uint64, error) { if s.consensus == nil { return 0, errors.New("not set up for safeblock") diff --git a/execution/interface.go b/execution/interface.go index b7ffce7857..b0817aeac4 100644 --- a/execution/interface.go +++ b/execution/interface.go @@ -78,7 +78,8 @@ type BatchFetcher interface { } type ConsensusInfo interface { - SyncProgressMap() map[string]interface{} + Synced() bool + FullSyncProgressMap() map[string]interface{} SyncTargetMessageCount() arbutil.MessageIndex // TODO: switch from pulling to pushing safe/finalized From d806f4d38ca421554a51dd6f002ddc06d65a26a4 Mon Sep 17 00:00:00 2001 From: Tsahi Zidenberg Date: Mon, 25 Mar 2024 15:45:04 -0600 Subject: [PATCH 31/34] inbox_tracker: add comment --- arbnode/inbox_tracker.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/arbnode/inbox_tracker.go b/arbnode/inbox_tracker.go index 4d892ac4cc..b758e95e62 100644 --- a/arbnode/inbox_tracker.go +++ b/arbnode/inbox_tracker.go @@ -228,6 +228,8 @@ func (t *InboxTracker) GetBatchCount() (uint64, error) { return count, nil } +// err will return unexpected/internal errors +// bool will be false if batch not found (meaning, block not yet posted on a batch) func (t *InboxTracker) FindInboxBatchContainingMessage(pos arbutil.MessageIndex) (uint64, bool, error) { batchCount, err := t.GetBatchCount() if err != nil { From 97c2bfa1e40518541345cfed4e089eb4e6c2c886 Mon Sep 17 00:00:00 2001 From: Ganesh Vanahalli Date: Tue, 26 Mar 2024 10:04:28 -0500 Subject: [PATCH 32/34] Add metrics for L2 gas pricing [NIT-2361] --- execution/gethexec/executionengine.go | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/execution/gethexec/executionengine.go b/execution/gethexec/executionengine.go index a662de3621..4ff10c70b7 100644 --- a/execution/gethexec/executionengine.go +++ b/execution/gethexec/executionengine.go @@ -13,6 +13,7 @@ import ( "github.com/ethereum/go-ethereum/core/state" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/metrics" "github.com/ethereum/go-ethereum/params" "github.com/offchainlabs/nitro/arbos" "github.com/offchainlabs/nitro/arbos/arbosState" @@ -20,10 +21,18 @@ import ( "github.com/offchainlabs/nitro/arbos/l1pricing" "github.com/offchainlabs/nitro/arbutil" "github.com/offchainlabs/nitro/execution" + "github.com/offchainlabs/nitro/util/arbmath" "github.com/offchainlabs/nitro/util/sharedmetrics" "github.com/offchainlabs/nitro/util/stopwaiter" ) +var ( + baseFeeGauge = metrics.NewRegisteredGauge("arb/block/basefee", nil) + blockGasUsedGauge = metrics.NewRegisteredHistogram("arb/block/gasused", nil, metrics.NewBoundedHistogramSample()) + txCountGauge = metrics.NewRegisteredHistogram("arb/block/transactions/count", nil, metrics.NewBoundedHistogramSample()) + txGasUsedGauge = metrics.NewRegisteredHistogram("arb/transaction/gasused", nil, metrics.NewBoundedHistogramSample()) +) + type ExecutionEngine struct { stopwaiter.StopWaiter @@ -487,6 +496,15 @@ func (s *ExecutionEngine) appendBlock(block *types.Block, statedb *state.StateDB if status == core.SideStatTy { return errors.New("geth rejected block as non-canonical") } + baseFeeGauge.Update(block.BaseFee().Int64()) + txCountGauge.Update(int64(len(block.Transactions()) - 1)) + var blockGasused uint64 + for i := 1; i < len(receipts); i++ { + val := arbmath.SaturatingUSub(receipts[i].GasUsed, receipts[i].GasUsedForL1) + txGasUsedGauge.Update(int64(val)) + blockGasused += val + } + blockGasUsedGauge.Update(int64(blockGasused)) return nil } From 8abc85558bc6952a275ce6f6f4308bc3dad1d0c1 Mon Sep 17 00:00:00 2001 From: Ganesh Vanahalli Date: Tue, 26 Mar 2024 11:20:10 -0500 Subject: [PATCH 33/34] address PR comments --- execution/gethexec/executionengine.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/execution/gethexec/executionengine.go b/execution/gethexec/executionengine.go index 4ff10c70b7..518f95f1fa 100644 --- a/execution/gethexec/executionengine.go +++ b/execution/gethexec/executionengine.go @@ -27,10 +27,10 @@ import ( ) var ( - baseFeeGauge = metrics.NewRegisteredGauge("arb/block/basefee", nil) - blockGasUsedGauge = metrics.NewRegisteredHistogram("arb/block/gasused", nil, metrics.NewBoundedHistogramSample()) - txCountGauge = metrics.NewRegisteredHistogram("arb/block/transactions/count", nil, metrics.NewBoundedHistogramSample()) - txGasUsedGauge = metrics.NewRegisteredHistogram("arb/transaction/gasused", nil, metrics.NewBoundedHistogramSample()) + baseFeeGauge = metrics.NewRegisteredGauge("arb/block/basefee", nil) + blockGasUsedHistogram = metrics.NewRegisteredHistogram("arb/block/gasused", nil, metrics.NewBoundedHistogramSample()) + txCountHistogram = metrics.NewRegisteredHistogram("arb/block/transactions/count", nil, metrics.NewBoundedHistogramSample()) + txGasUsedHistogram = metrics.NewRegisteredHistogram("arb/block/transactions/gasused", nil, metrics.NewBoundedHistogramSample()) ) type ExecutionEngine struct { @@ -497,14 +497,14 @@ func (s *ExecutionEngine) appendBlock(block *types.Block, statedb *state.StateDB return errors.New("geth rejected block as non-canonical") } baseFeeGauge.Update(block.BaseFee().Int64()) - txCountGauge.Update(int64(len(block.Transactions()) - 1)) + txCountHistogram.Update(int64(len(block.Transactions()) - 1)) var blockGasused uint64 for i := 1; i < len(receipts); i++ { val := arbmath.SaturatingUSub(receipts[i].GasUsed, receipts[i].GasUsedForL1) - txGasUsedGauge.Update(int64(val)) + txGasUsedHistogram.Update(int64(val)) blockGasused += val } - blockGasUsedGauge.Update(int64(blockGasused)) + blockGasUsedHistogram.Update(int64(blockGasused)) return nil } From 5879b05a582de7a03d95fea2ce6915c8b6ab23f9 Mon Sep 17 00:00:00 2001 From: Lee Bousfield Date: Tue, 26 Mar 2024 15:46:40 -0500 Subject: [PATCH 34/34] Fix new rust warning about static mut references --- arbitrator/wasm-libraries/go-stub/src/value.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/arbitrator/wasm-libraries/go-stub/src/value.rs b/arbitrator/wasm-libraries/go-stub/src/value.rs index 3a015bbf70..22c1ed6a86 100644 --- a/arbitrator/wasm-libraries/go-stub/src/value.rs +++ b/arbitrator/wasm-libraries/go-stub/src/value.rs @@ -164,9 +164,9 @@ pub unsafe fn get_field(source: u32, field: &[u8]) -> GoValue { } } else if source == GO_ID { if field == b"_pendingEvent" { - if let Some(event) = &PENDING_EVENT { + if let Some(event) = PENDING_EVENT.clone() { let id = DynamicObjectPool::singleton() - .insert(DynamicObject::PendingEvent(event.clone())); + .insert(DynamicObject::PendingEvent(event)); return GoValue::Object(id); } else { return GoValue::Null;