diff --git a/arbnode/node.go b/arbnode/node.go index 705a48da08..86adb783ad 100644 --- a/arbnode/node.go +++ b/arbnode/node.go @@ -1020,7 +1020,7 @@ func (n *Node) GetFinalizedMsgCount(ctx context.Context) (arbutil.MessageIndex, return n.InboxReader.GetFinalizedMsgCount(ctx) } -func (n *Node) WriteMessageFromSequencer(pos arbutil.MessageIndex, msgWithMeta arbostypes.MessageWithMetadata, msgResult execution.MessageResult, blockMetadata arbostypes.BlockMetadata) error { +func (n *Node) WriteMessageFromSequencer(pos arbutil.MessageIndex, msgWithMeta arbostypes.MessageWithMetadata, msgResult execution.MessageResult, blockMetadata common.BlockMetadata) error { return n.TxStreamer.WriteMessageFromSequencer(pos, msgWithMeta, msgResult, blockMetadata) } @@ -1035,6 +1035,6 @@ func (n *Node) ValidatedMessageCount() (arbutil.MessageIndex, error) { return n.BlockValidator.GetValidated(), nil } -func (n *Node) BlockMetadataAtCount(count arbutil.MessageIndex) (arbostypes.BlockMetadata, error) { +func (n *Node) BlockMetadataAtCount(count arbutil.MessageIndex) (common.BlockMetadata, error) { return n.TxStreamer.BlockMetadataAtCount(count) } diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index 1636e06bd3..40466030dc 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -960,7 +960,7 @@ func (s *TransactionStreamer) WriteMessageFromSequencer( pos arbutil.MessageIndex, msgWithMeta arbostypes.MessageWithMetadata, msgResult execution.MessageResult, - blockMetadata arbostypes.BlockMetadata, + blockMetadata common.BlockMetadata, ) error { if err := s.ExpectChosenSequencer(); err != nil { return err @@ -1091,7 +1091,7 @@ func (s *TransactionStreamer) writeMessages(pos arbutil.MessageIndex, messages [ return nil } -func (s *TransactionStreamer) BlockMetadataAtCount(count arbutil.MessageIndex) (arbostypes.BlockMetadata, error) { +func (s *TransactionStreamer) BlockMetadataAtCount(count arbutil.MessageIndex) (common.BlockMetadata, error) { if count == 0 { return []byte{}, nil } diff --git a/arbos/arbostypes/messagewithmeta.go b/arbos/arbostypes/messagewithmeta.go index 6701f352de..59b39c9246 100644 --- a/arbos/arbostypes/messagewithmeta.go +++ b/arbos/arbostypes/messagewithmeta.go @@ -18,12 +18,10 @@ type MessageWithMetadata struct { DelayedMessagesRead uint64 `json:"delayedMessagesRead"` } -type BlockMetadata []byte - type MessageWithMetadataAndBlockInfo struct { MessageWithMeta MessageWithMetadata BlockHash *common.Hash - BlockMetadata BlockMetadata + BlockMetadata common.BlockMetadata } var EmptyTestMessageWithMetadata = MessageWithMetadata{ @@ -35,15 +33,6 @@ var TestMessageWithMetadataAndRequestId = MessageWithMetadata{ Message: &TestIncomingMessageWithRequestId, } -// IsTxTimeboosted given a tx's index in the block returns whether the tx was timeboosted or not -func (b BlockMetadata) IsTxTimeboosted(txIndex int) bool { - maxTxCount := (len(b) - 1) * 8 - if txIndex >= maxTxCount { - return false - } - return b[1+(txIndex/8)]&(1<<(txIndex%8)) != 0 -} - func (m *MessageWithMetadata) Hash(sequenceNumber arbutil.MessageIndex, chainId uint64) (common.Hash, error) { serializedExtraData := make([]byte, 24) binary.BigEndian.PutUint64(serializedExtraData[:8], uint64(sequenceNumber)) diff --git a/broadcaster/broadcaster.go b/broadcaster/broadcaster.go index da856f98b4..fd34718df7 100644 --- a/broadcaster/broadcaster.go +++ b/broadcaster/broadcaster.go @@ -43,7 +43,7 @@ func (b *Broadcaster) NewBroadcastFeedMessage( message arbostypes.MessageWithMetadata, sequenceNumber arbutil.MessageIndex, blockHash *common.Hash, - blockMetadata arbostypes.BlockMetadata, + blockMetadata common.BlockMetadata, ) (*m.BroadcastFeedMessage, error) { var messageSignature []byte if b.dataSigner != nil { @@ -70,7 +70,7 @@ func (b *Broadcaster) BroadcastSingle( msg arbostypes.MessageWithMetadata, seq arbutil.MessageIndex, blockHash *common.Hash, - blockMetadata arbostypes.BlockMetadata, + blockMetadata common.BlockMetadata, ) (err error) { defer func() { if r := recover(); r != nil { diff --git a/broadcaster/message/message.go b/broadcaster/message/message.go index b5aae20f2b..87cc83db4e 100644 --- a/broadcaster/message/message.go +++ b/broadcaster/message/message.go @@ -37,7 +37,7 @@ type BroadcastFeedMessage struct { Message arbostypes.MessageWithMetadata `json:"message"` BlockHash *common.Hash `json:"blockHash,omitempty"` Signature []byte `json:"signature"` - BlockMetadata arbostypes.BlockMetadata `json:"blockMetadata"` + BlockMetadata common.BlockMetadata `json:"blockMetadata"` CumulativeSumMsgSize uint64 `json:"-"` } diff --git a/broadcaster/message/message_blockmetadata_test.go b/broadcaster/message/message_blockmetadata_test.go index ca51b5bbc0..625dd5b944 100644 --- a/broadcaster/message/message_blockmetadata_test.go +++ b/broadcaster/message/message_blockmetadata_test.go @@ -3,14 +3,14 @@ package message import ( "testing" - "github.com/offchainlabs/nitro/arbos/arbostypes" + "github.com/ethereum/go-ethereum/common" ) func TestTimeboostedInDifferentScenarios(t *testing.T) { t.Parallel() for _, tc := range []struct { name string - blockMetadata arbostypes.BlockMetadata + blockMetadata common.BlockMetadata txs []bool // Array representing whether the tx is timeboosted or not. First tx is always false as its an arbitrum internal tx }{ { diff --git a/execution/gethexec/blockmetadata.go b/execution/gethexec/blockmetadata.go index 7605861ffe..37c46bbaa7 100644 --- a/execution/gethexec/blockmetadata.go +++ b/execution/gethexec/blockmetadata.go @@ -5,11 +5,11 @@ import ( "errors" "fmt" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/common/lru" "github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/rpc" - "github.com/offchainlabs/nitro/arbos/arbostypes" "github.com/offchainlabs/nitro/arbutil" "github.com/offchainlabs/nitro/util/stopwaiter" ) @@ -17,7 +17,7 @@ import ( var ErrBlockMetadataApiBlocksLimitExceeded = errors.New("number of blocks requested for blockMetadata exceeded") type BlockMetadataFetcher interface { - BlockMetadataAtCount(count arbutil.MessageIndex) (arbostypes.BlockMetadata, error) + BlockMetadataAtCount(count arbutil.MessageIndex) (common.BlockMetadata, error) BlockNumberToMessageIndex(blockNum uint64) (arbutil.MessageIndex, error) MessageIndexToBlockNumber(messageNum arbutil.MessageIndex) uint64 SetReorgEventsNotifier(reorgEventsNotifier chan struct{}) @@ -29,14 +29,14 @@ type BulkBlockMetadataFetcher struct { fetcher BlockMetadataFetcher reorgDetector chan struct{} blocksLimit uint64 - cache *lru.SizeConstrainedCache[arbutil.MessageIndex, arbostypes.BlockMetadata] + cache *lru.SizeConstrainedCache[arbutil.MessageIndex, common.BlockMetadata] } func NewBulkBlockMetadataFetcher(bc *core.BlockChain, fetcher BlockMetadataFetcher, cacheSize, blocksLimit uint64) *BulkBlockMetadataFetcher { - var cache *lru.SizeConstrainedCache[arbutil.MessageIndex, arbostypes.BlockMetadata] + var cache *lru.SizeConstrainedCache[arbutil.MessageIndex, common.BlockMetadata] var reorgDetector chan struct{} if cacheSize != 0 { - cache = lru.NewSizeConstrainedCache[arbutil.MessageIndex, arbostypes.BlockMetadata](cacheSize) + cache = lru.NewSizeConstrainedCache[arbutil.MessageIndex, common.BlockMetadata](cacheSize) reorgDetector = make(chan struct{}) fetcher.SetReorgEventsNotifier(reorgDetector) } @@ -68,7 +68,7 @@ func (b *BulkBlockMetadataFetcher) Fetch(fromBlock, toBlock rpc.BlockNumber) ([] } var result []NumberAndBlockMetadata for i := start; i <= end; i++ { - var data arbostypes.BlockMetadata + var data common.BlockMetadata var found bool if b.cache != nil { data, found = b.cache.Get(i) diff --git a/execution/gethexec/executionengine.go b/execution/gethexec/executionengine.go index 193e8e7414..f65e65a52c 100644 --- a/execution/gethexec/executionengine.go +++ b/execution/gethexec/executionengine.go @@ -223,7 +223,7 @@ func (s *ExecutionEngine) SetConsensus(consensus execution.FullConsensusClient) s.consensus = consensus } -func (s *ExecutionEngine) BlockMetadataAtCount(count arbutil.MessageIndex) (arbostypes.BlockMetadata, error) { +func (s *ExecutionEngine) BlockMetadataAtCount(count arbutil.MessageIndex) (common.BlockMetadata, error) { if s.consensus != nil { return s.consensus.BlockMetadataAtCount(count) } @@ -575,8 +575,8 @@ func (s *ExecutionEngine) sequenceTransactionsWithBlockMutex(header *arbostypes. // starting from the second byte, (N)th bit would represent if (N)th tx is timeboosted or not, 1 means yes and 0 means no // blockMetadata[index / 8 + 1] & (1 << (index % 8)) != 0; where index = (N - 1), implies whether (N)th tx in a block is timeboosted // note that number of txs in a block will always lag behind (len(blockMetadata) - 1) * 8 but it wont lag more than a value of 7 -func (s *ExecutionEngine) blockMetadataFromBlock(block *types.Block, timeboostedTxs map[common.Hash]struct{}) arbostypes.BlockMetadata { - bits := make(arbostypes.BlockMetadata, 1+arbmath.DivCeil(uint64(len(block.Transactions())), 8)) +func (s *ExecutionEngine) blockMetadataFromBlock(block *types.Block, timeboostedTxs map[common.Hash]struct{}) common.BlockMetadata { + bits := make(common.BlockMetadata, 1+arbmath.DivCeil(uint64(len(block.Transactions())), 8)) if len(timeboostedTxs) == 0 { return bits } diff --git a/execution/gethexec/sync_monitor.go b/execution/gethexec/sync_monitor.go index 86949c7767..6a739f4ae0 100644 --- a/execution/gethexec/sync_monitor.go +++ b/execution/gethexec/sync_monitor.go @@ -3,6 +3,8 @@ package gethexec import ( "context" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/log" "github.com/offchainlabs/nitro/execution" "github.com/pkg/errors" flag "github.com/spf13/pflag" @@ -121,3 +123,15 @@ func (s *SyncMonitor) Synced() bool { func (s *SyncMonitor) SetConsensusInfo(consensus execution.ConsensusInfo) { s.consensus = consensus } + +func (s *SyncMonitor) BlockMetadataByNumber(blockNum uint64) (common.BlockMetadata, error) { + count, err := s.exec.BlockNumberToMessageIndex(blockNum) + if err != nil { + return nil, err + } + if s.consensus != nil { + return s.consensus.BlockMetadataAtCount(count + 1) + } + log.Debug("FullConsensusClient is not accessible to execution, BlockMetadataByNumber will return nil") + return nil, nil +} diff --git a/execution/interface.go b/execution/interface.go index 01f71d4422..967fc0c92e 100644 --- a/execution/interface.go +++ b/execution/interface.go @@ -83,7 +83,7 @@ type ConsensusInfo interface { Synced() bool FullSyncProgressMap() map[string]interface{} SyncTargetMessageCount() arbutil.MessageIndex - BlockMetadataAtCount(count arbutil.MessageIndex) (arbostypes.BlockMetadata, error) + BlockMetadataAtCount(count arbutil.MessageIndex) (common.BlockMetadata, error) // TODO: switch from pulling to pushing safe/finalized GetSafeMsgCount(ctx context.Context) (arbutil.MessageIndex, error) @@ -92,7 +92,7 @@ type ConsensusInfo interface { } type ConsensusSequencer interface { - WriteMessageFromSequencer(pos arbutil.MessageIndex, msgWithMeta arbostypes.MessageWithMetadata, msgResult MessageResult, blockMetadata arbostypes.BlockMetadata) error + WriteMessageFromSequencer(pos arbutil.MessageIndex, msgWithMeta arbostypes.MessageWithMetadata, msgResult MessageResult, blockMetadata common.BlockMetadata) error ExpectChosenSequencer() error } diff --git a/go-ethereum b/go-ethereum index 68cb86d1ec..01a4427cce 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 68cb86d1eca03353375c24befaa7814f145d425a +Subproject commit 01a4427cce14f1e508a6e2b607ea29a60ebef2ef diff --git a/system_tests/timeboost_test.go b/system_tests/timeboost_test.go index 1e7ca47db4..80abbb37b4 100644 --- a/system_tests/timeboost_test.go +++ b/system_tests/timeboost_test.go @@ -5,6 +5,7 @@ import ( "context" "crypto/ecdsa" "encoding/binary" + "encoding/json" "fmt" "math/big" "net" @@ -39,13 +40,24 @@ import ( "github.com/offchainlabs/nitro/timeboost" "github.com/offchainlabs/nitro/timeboost/bindings" "github.com/offchainlabs/nitro/util/arbmath" + "github.com/offchainlabs/nitro/util/colors" "github.com/offchainlabs/nitro/util/containers" "github.com/offchainlabs/nitro/util/redisutil" "github.com/offchainlabs/nitro/util/stopwaiter" "github.com/stretchr/testify/require" ) -func TestTimeboostBulkBlockMetadataAPI(t *testing.T) { +func blockMetadataInputFeedKey(pos uint64) []byte { + var key []byte + prefix := []byte("t") + key = append(key, prefix...) + data := make([]byte, 8) + binary.BigEndian.PutUint64(data, pos) + key = append(key, data...) + return key +} + +func TestTimeboostedFieldInReceiptsObject(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -54,16 +66,115 @@ func TestTimeboostBulkBlockMetadataAPI(t *testing.T) { cleanup := builder.Build(t) defer cleanup() + // Generate blocks until current block is totalBlocks arbDb := builder.L2.ConsensusNode.ArbDB - blockMetadataInputFeedKey := func(pos uint64) []byte { - var key []byte - prefix := []byte("t") - key = append(key, prefix...) - data := make([]byte, 8) - binary.BigEndian.PutUint64(data, pos) - key = append(key, data...) - return key + blockNum := big.NewInt(2) + builder.L2Info.GenerateAccount("User") + user := builder.L2Info.GetDefaultTransactOpts("User", ctx) + var latestL2 uint64 + var err error + for i := 0; ; i++ { + builder.L2.TransferBalanceTo(t, "Owner", util.RemapL1Address(user.From), big.NewInt(1e18), builder.L2Info) + latestL2, err = builder.L2.Client.BlockNumber(ctx) + Require(t, err) + if latestL2 >= blockNum.Uint64() { + break + } + } + + for i := uint64(1); i < latestL2; i++ { + // Clean BlockMetadata from arbDB so that we can modify it at will + Require(t, arbDb.Delete(blockMetadataInputFeedKey(i))) + } + + block, err := builder.L2.Client.BlockByNumber(ctx, blockNum) + Require(t, err) + if len(block.Transactions()) != 2 { + t.Fatalf("expecting two txs in the second block, but found: %d txs", len(block.Transactions())) + } + + // Set first tx (internal tx anyway) to not timeboosted and Second one to timeboosted- BlockMetadata (in bits)-> 00000000 00000010 + arbDb.Put(blockMetadataInputFeedKey(blockNum.Uint64()), []byte{0, 2}) + l2rpc := builder.L2.Stack.Attach() + // Extra timeboosted field in pointer form to check for its existence + type timeboostedFromReceipt struct { + Timeboosted *bool `json:"timeboosted"` + } + var receiptResult []timeboostedFromReceipt + err = l2rpc.CallContext(ctx, &receiptResult, "eth_getBlockReceipts", rpc.BlockNumber(blockNum.Int64())) + Require(t, err) + if receiptResult[0].Timeboosted == nil || receiptResult[1].Timeboosted == nil { + t.Fatal("timeboosted field should exist in the receipt object of both- first and second txs") + } + if *receiptResult[0].Timeboosted != false { + t.Fatal("first tx was not timeboosted, but the field indicates otherwise") + } + if *receiptResult[1].Timeboosted != true { + t.Fatal("second tx was timeboosted, but the field indicates otherwise") + } + + // Check that timeboosted is accurate for eth_getTransactionReceipt as well + var txReceipt timeboostedFromReceipt + err = l2rpc.CallContext(ctx, &txReceipt, "eth_getTransactionReceipt", block.Transactions()[0].Hash()) + Require(t, err) + if txReceipt.Timeboosted == nil { + t.Fatal("timeboosted field should exist in the receipt object of first tx") + } + if *txReceipt.Timeboosted != false { + t.Fatal("first tx was not timeboosted, but the field indicates otherwise") } + err = l2rpc.CallContext(ctx, &txReceipt, "eth_getTransactionReceipt", block.Transactions()[1].Hash()) + Require(t, err) + if txReceipt.Timeboosted == nil { + t.Fatal("timeboosted field should exist in the receipt object of second tx") + } + if *txReceipt.Timeboosted != true { + t.Fatal("second tx was timeboosted, but the field indicates otherwise") + } + + // Check that timeboosted field shouldn't exist for any txs of block=1, as this block doesn't have blockMetadata + block, err = builder.L2.Client.BlockByNumber(ctx, common.Big1) + Require(t, err) + if len(block.Transactions()) != 2 { + t.Fatalf("expecting two txs in the first block, but found: %d txs", len(block.Transactions())) + } + var receiptResult2 []timeboostedFromReceipt + err = l2rpc.CallContext(ctx, &receiptResult2, "eth_getBlockReceipts", rpc.BlockNumber(1)) + Require(t, err) + if receiptResult2[0].Timeboosted != nil || receiptResult2[1].Timeboosted != nil { + t.Fatal("timeboosted field shouldn't exist in the receipt object of all the txs") + } + var txReceipt2 timeboostedFromReceipt + err = l2rpc.CallContext(ctx, &txReceipt2, "eth_getTransactionReceipt", block.Transactions()[0].Hash()) + Require(t, err) + if txReceipt2.Timeboosted != nil { + t.Fatal("timeboosted field shouldn't exist in the receipt object of all the txs") + } + var txReceipt3 timeboostedFromReceipt + err = l2rpc.CallContext(ctx, &txReceipt3, "eth_getTransactionReceipt", block.Transactions()[1].Hash()) + Require(t, err) + if txReceipt3.Timeboosted != nil { + t.Fatal("timeboosted field shouldn't exist in the receipt object of all the txs") + } + + // Print the receipt object for reference + var receiptResultRaw json.RawMessage + err = l2rpc.CallContext(ctx, &receiptResultRaw, "eth_getBlockReceipts", rpc.BlockNumber(blockNum.Int64())) + Require(t, err) + colors.PrintGrey("receipt object- ", string(receiptResultRaw)) + +} + +func TestTimeboostBulkBlockMetadataAPI(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + builder := NewNodeBuilder(ctx).DefaultConfig(t, false) + builder.execConfig.BlockMetadataApiCacheSize = 0 // Caching is disabled + cleanup := builder.Build(t) + defer cleanup() + + arbDb := builder.L2.ConsensusNode.ArbDB // Generate blocks until current block is end start := 1