From aa398fefd88747cb870e33199b98a58dc955bba3 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Tue, 31 Oct 2023 15:33:43 +0000 Subject: [PATCH 1/6] add features byte to sequencer message --- arbnode/delayed.go | 1 + arbnode/delayed_seq_reorg_test.go | 1 + arbnode/inbox_test.go | 1 + arbnode/transaction_streamer.go | 2 + arbos/arbostypes/incomingmessage.go | 117 ++++++++++++------ arbos/block_processor.go | 3 +- arbos/incomingmessage_test.go | 3 +- arbos/parse_l2.go | 11 +- arbos/tx_processor.go | 3 +- arbstate/das_reader.go | 8 +- arbstate/inbox.go | 28 +++-- broadcaster/broadcaster_serialization_test.go | 1 + execution/gethexec/executionengine.go | 4 +- execution/gethexec/sequencer.go | 3 +- gethhook/geth_test.go | 4 +- system_tests/contract_tx_test.go | 1 + system_tests/reorg_resequencing_test.go | 1 + system_tests/retryable_test.go | 2 +- system_tests/seq_coordinator_test.go | 1 + system_tests/tippingtx_test.go | 4 +- 20 files changed, 138 insertions(+), 61 deletions(-) diff --git a/arbnode/delayed.go b/arbnode/delayed.go index f2c3d62004..db80267ffa 100644 --- a/arbnode/delayed.go +++ b/arbnode/delayed.go @@ -211,6 +211,7 @@ func (b *DelayedBridge) logsToDeliveredMessages(ctx context.Context, logs []type Timestamp: parsedLog.Timestamp, RequestId: &requestId, L1BaseFee: parsedLog.BaseFeeL1, + Features: arbostypes.FeatureFlag_Invalid, }, L2msg: data, }, diff --git a/arbnode/delayed_seq_reorg_test.go b/arbnode/delayed_seq_reorg_test.go index a28eebb5dc..f1330b76f9 100644 --- a/arbnode/delayed_seq_reorg_test.go +++ b/arbnode/delayed_seq_reorg_test.go @@ -45,6 +45,7 @@ func TestSequencerReorgFromDelayed(t *testing.T) { Timestamp: 0, RequestId: &delayedRequestId, L1BaseFee: common.Big0, + Features: 0, // TODO(magic) }, }, } diff --git a/arbnode/inbox_test.go b/arbnode/inbox_test.go index 3060ae2ae6..a8628a4b53 100644 --- a/arbnode/inbox_test.go +++ b/arbnode/inbox_test.go @@ -160,6 +160,7 @@ func TestTransactionStreamer(t *testing.T) { Kind: arbostypes.L1MessageType_L2Message, Poster: source, RequestId: &requestId, + Features: 0, // TODO(magic) }, L2msg: l2Message, }, diff --git a/arbnode/transaction_streamer.go b/arbnode/transaction_streamer.go index bcc389dc01..3d47b1875e 100644 --- a/arbnode/transaction_streamer.go +++ b/arbnode/transaction_streamer.go @@ -318,6 +318,7 @@ func (s *TransactionStreamer) reorg(batch ethdb.Batch, count arbutil.MessageInde if delayedFound.Message.Header.RequestId.Big().Uint64() != delayedSeqNum { continue delayedInBlockLoop } + delayedFound.Message.Header.Features = oldMessage.Message.Header.Features if expectedAcc == delayedFound.AfterInboxAcc() && delayedFound.Message.Equals(oldMessage.Message) { messageFound = true } @@ -538,6 +539,7 @@ func (s *TransactionStreamer) AddFakeInitMessage() error { Kind: arbostypes.L1MessageType_Initialize, RequestId: &common.Hash{}, L1BaseFee: common.Big0, + Features: 0, // TODO(magic) }, L2msg: msg, }, diff --git a/arbos/arbostypes/incomingmessage.go b/arbos/arbostypes/incomingmessage.go index 9ecd207e89..d54b8401d5 100644 --- a/arbos/arbostypes/incomingmessage.go +++ b/arbos/arbostypes/incomingmessage.go @@ -9,9 +9,11 @@ import ( "errors" "fmt" "io" + "math" "math/big" "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/params" @@ -29,12 +31,48 @@ const ( L1MessageType_Initialize = 11 L1MessageType_EthDeposit = 12 L1MessageType_BatchPostingReport = 13 + L1MessageType_L1MessageWithFeatures = 14 L1MessageType_Invalid = 0xFF ) const MaxL2MessageSize = 256 * 1024 -const ArbosVersion_ArbitrumSubtypedTx = 12 +const ArbosVersion_ArbitrumTippingTx = 12 + +func RequiredArobosVersionForTxSubtype(txSubtype uint8) uint64 { + switch txSubtype { + case types.ArbitrumTippingTxSubtype: + return ArbosVersion_ArbitrumTippingTx + default: + // shouldn't be ever possible as tx with unsupported subtype is dropped earlier + return math.MaxUint64 + } +} + +type Features uint8 + +const ( + FeatureFlag_Invalid Features = (1 << 0) // when set indicates that feature flags value is not set to valid value + FeatureFlag_ArbitrumTippingTx Features = (1 << 1) + FeatureFlag_Reserved Features = (1 << 7) // could be used to implement future feature format versioning where version = feature_flags & 0x7f +) + +func (f Features) TxSubtypeSupported(subtype uint8) bool { + switch subtype { + case types.ArbitrumTippingTxSubtype: + return (f & FeatureFlag_ArbitrumTippingTx) > 0 + default: + return false + } +} + +func ArbosVersionBasedFeatureFlags(arbosVersion uint64) Features { + var features Features + if arbosVersion > ArbosVersion_ArbitrumTippingTx { + features |= FeatureFlag_ArbitrumTippingTx + } + return features +} type L1IncomingMessageHeader struct { Kind uint8 `json:"kind"` @@ -43,6 +81,7 @@ type L1IncomingMessageHeader struct { Timestamp uint64 `json:"timestamp"` RequestId *common.Hash `json:"requestId" rlp:"nilList"` L1BaseFee *big.Int `json:"baseFeeL1"` + Features Features `json:"features,omitempty" rlp:"optional"` } func (h L1IncomingMessageHeader) SeqNum() (uint64, error) { @@ -83,50 +122,56 @@ var InvalidL1Message = &L1IncomingMessage{ L2msg: []byte{}, } -func (msg *L1IncomingMessage) Serialize() ([]byte, error) { +func (m *L1IncomingMessage) Serialize() ([]byte, error) { wr := &bytes.Buffer{} - if err := wr.WriteByte(msg.Header.Kind); err != nil { + if err := wr.WriteByte(m.Header.Kind); err != nil { return nil, err } - if err := util.AddressTo256ToWriter(msg.Header.Poster, wr); err != nil { + if err := util.AddressTo256ToWriter(m.Header.Poster, wr); err != nil { return nil, err } - if err := util.Uint64ToWriter(msg.Header.BlockNumber, wr); err != nil { + if err := util.Uint64ToWriter(m.Header.BlockNumber, wr); err != nil { return nil, err } - if err := util.Uint64ToWriter(msg.Header.Timestamp, wr); err != nil { + if err := util.Uint64ToWriter(m.Header.Timestamp, wr); err != nil { return nil, err } - if msg.Header.RequestId == nil { + if m.Header.RequestId == nil { return nil, errors.New("cannot serialize L1IncomingMessage without RequestId") } - requestId := *msg.Header.RequestId + requestId := *m.Header.RequestId if err := util.HashToWriter(requestId, wr); err != nil { return nil, err } var l1BaseFeeHash common.Hash - if msg.Header.L1BaseFee == nil { + if m.Header.L1BaseFee == nil { return nil, errors.New("cannot serialize L1IncomingMessage without L1BaseFee") } - l1BaseFeeHash = common.BigToHash(msg.Header.L1BaseFee) + l1BaseFeeHash = common.BigToHash(m.Header.L1BaseFee) if err := util.HashToWriter(l1BaseFeeHash, wr); err != nil { return nil, err } - if _, err := wr.Write(msg.L2msg); err != nil { + if m.Header.Features != 0 { + if err := wr.WriteByte(uint8(m.Header.Features)); err != nil { + return nil, err + } + } + + if _, err := wr.Write(m.L2msg); err != nil { return nil, err } return wr.Bytes(), nil } -func (msg *L1IncomingMessage) Equals(other *L1IncomingMessage) bool { - return msg.Header.Equals(other.Header) && bytes.Equal(msg.L2msg, other.L2msg) +func (m *L1IncomingMessage) Equals(other *L1IncomingMessage) bool { + return m.Header.Equals(other.Header) && bytes.Equal(m.L2msg, other.L2msg) } func hashesEqual(ha, hb *common.Hash) bool { @@ -143,7 +188,8 @@ func (h *L1IncomingMessageHeader) Equals(other *L1IncomingMessageHeader) bool { h.BlockNumber == other.BlockNumber && h.Timestamp == other.Timestamp && hashesEqual(h.RequestId, other.RequestId) && - arbmath.BigEquals(h.L1BaseFee, other.L1BaseFee) + arbmath.BigEquals(h.L1BaseFee, other.L1BaseFee) && + h.Features == other.Features } func ComputeBatchGasCost(data []byte) uint64 { @@ -163,11 +209,11 @@ func ComputeBatchGasCost(data []byte) uint64 { return gas } -func (msg *L1IncomingMessage) FillInBatchGasCost(batchFetcher FallibleBatchFetcher) error { - if batchFetcher == nil || msg.Header.Kind != L1MessageType_BatchPostingReport || msg.BatchGasCost != nil { +func (m *L1IncomingMessage) FillInBatchGasCost(batchFetcher FallibleBatchFetcher) error { + if batchFetcher == nil || m.Header.Kind != L1MessageType_BatchPostingReport || m.BatchGasCost != nil { return nil } - _, _, batchHash, batchNum, _, _, err := ParseBatchPostingReportMessageFields(bytes.NewReader(msg.L2msg)) + _, _, batchHash, batchNum, _, _, err := ParseBatchPostingReportMessageFields(bytes.NewReader(m.L2msg)) if err != nil { return fmt.Errorf("failed to parse batch posting report: %w", err) } @@ -180,7 +226,7 @@ func (msg *L1IncomingMessage) FillInBatchGasCost(batchFetcher FallibleBatchFetch return fmt.Errorf("batch fetcher returned incorrect data hash %v (wanted %v for batch %v)", gotHash, batchHash, batchNum) } gas := ComputeBatchGasCost(batchData) - msg.BatchGasCost = &gas + m.BatchGasCost = &gas return nil } @@ -224,12 +270,13 @@ func ParseIncomingL1Message(rd io.Reader, batchFetcher FallibleBatchFetcher) (*L msg := &L1IncomingMessage{ &L1IncomingMessageHeader{ - kind, - sender, - blockNumber, - timestamp, - &requestId, - baseFeeL1.Big(), + Kind: kind, + Poster: sender, + BlockNumber: blockNumber, + Timestamp: timestamp, + RequestId: &requestId, + L1BaseFee: baseFeeL1.Big(), + Features: 0, // TODO(magic) }, data, nil, @@ -261,21 +308,21 @@ var TestInitMessage = &ParsedInitMessage{ } // ParseInitMessage returns the chain id on success -func (msg *L1IncomingMessage) ParseInitMessage() (*ParsedInitMessage, error) { - if msg.Header.Kind != L1MessageType_Initialize { - return nil, fmt.Errorf("invalid init message kind %v", msg.Header.Kind) +func (m *L1IncomingMessage) ParseInitMessage() (*ParsedInitMessage, error) { + if m.Header.Kind != L1MessageType_Initialize { + return nil, fmt.Errorf("invalid init message kind %v", m.Header.Kind) } basefee := new(big.Int).Set(DefaultInitialL1BaseFee) var chainConfig params.ChainConfig var chainId *big.Int - if len(msg.L2msg) == 32 { - chainId = new(big.Int).SetBytes(msg.L2msg[:32]) + if len(m.L2msg) == 32 { + chainId = new(big.Int).SetBytes(m.L2msg[:32]) return &ParsedInitMessage{chainId, basefee, nil, nil}, nil } - if len(msg.L2msg) > 32 { - chainId = new(big.Int).SetBytes(msg.L2msg[:32]) - version := msg.L2msg[32] - reader := bytes.NewReader(msg.L2msg[33:]) + if len(m.L2msg) > 32 { + chainId = new(big.Int).SetBytes(m.L2msg[:32]) + version := m.L2msg[32] + reader := bytes.NewReader(m.L2msg[33:]) switch version { case 1: var err error @@ -291,12 +338,12 @@ func (msg *L1IncomingMessage) ParseInitMessage() (*ParsedInitMessage, error) { } err = json.Unmarshal(serializedChainConfig, &chainConfig) if err != nil { - return nil, fmt.Errorf("failed to parse init message, err: %w, message data: %v", err, string(msg.L2msg)) + return nil, fmt.Errorf("failed to parse init message, err: %w, message data: %v", err, string(m.L2msg)) } return &ParsedInitMessage{chainId, basefee, &chainConfig, serializedChainConfig}, nil } } - return nil, fmt.Errorf("invalid init message data %v", string(msg.L2msg)) + return nil, fmt.Errorf("invalid init message data %v", string(m.L2msg)) } func ParseBatchPostingReportMessageFields(rd io.Reader) (*big.Int, common.Address, common.Hash, uint64, *big.Int, uint64, error) { diff --git a/arbos/block_processor.go b/arbos/block_processor.go index b6baa3ea93..87e88455df 100644 --- a/arbos/block_processor.go +++ b/arbos/block_processor.go @@ -138,8 +138,9 @@ func ProduceBlock( if err != nil { return nil, nil, err } + message.Header.Features = arbostypes.ArbosVersionBasedFeatureFlags(arbState.ArbOSVersion()) var batchFetchErr error - txes, err := ParseL2Transactions(message, chainConfig.ChainID, arbState.ArbOSVersion(), func(batchNum uint64, batchHash common.Hash) []byte { + txes, err := ParseL2Transactions(message, chainConfig.ChainID, func(batchNum uint64, batchHash common.Hash) []byte { data, err := batchFetcher(batchNum) if err != nil { batchFetchErr = err diff --git a/arbos/incomingmessage_test.go b/arbos/incomingmessage_test.go index e96a43c6a3..b44a489356 100644 --- a/arbos/incomingmessage_test.go +++ b/arbos/incomingmessage_test.go @@ -22,6 +22,7 @@ func TestSerializeAndParseL1Message(t *testing.T) { Timestamp: 8794561564, RequestId: &requestId, L1BaseFee: big.NewInt(10000000000000), + Features: 0, // TODO(magic) } msg := arbostypes.L1IncomingMessage{ Header: &header, @@ -36,7 +37,7 @@ func TestSerializeAndParseL1Message(t *testing.T) { if err != nil { t.Error(err) } - txes, err := ParseL2Transactions(newMsg, chainId, 0, nil) + txes, err := ParseL2Transactions(newMsg, chainId, nil) if err != nil { t.Error(err) } diff --git a/arbos/parse_l2.go b/arbos/parse_l2.go index 4c89a7908c..538d898b8e 100644 --- a/arbos/parse_l2.go +++ b/arbos/parse_l2.go @@ -20,14 +20,14 @@ import ( type InfallibleBatchFetcher func(batchNum uint64, batchHash common.Hash) []byte -func ParseL2Transactions(msg *arbostypes.L1IncomingMessage, chainId *big.Int, arbOSVersion uint64, batchFetcher InfallibleBatchFetcher) (types.Transactions, error) { +func ParseL2Transactions(msg *arbostypes.L1IncomingMessage, chainId *big.Int, batchFetcher InfallibleBatchFetcher) (types.Transactions, error) { if len(msg.L2msg) > arbostypes.MaxL2MessageSize { // ignore the message if l2msg is too large return nil, errors.New("message too large") } switch msg.Header.Kind { case arbostypes.L1MessageType_L2Message: - return parseL2Message(bytes.NewReader(msg.L2msg), msg.Header.Poster, msg.Header.Timestamp, msg.Header.RequestId, chainId, 0, arbOSVersion) + return parseL2Message(bytes.NewReader(msg.L2msg), msg.Header.Poster, msg.Header.Timestamp, msg.Header.RequestId, chainId, msg.Header.Features, 0) case arbostypes.L1MessageType_Initialize: return nil, errors.New("ParseL2Transactions encounted initialize message (should've been handled explicitly at genesis)") case arbostypes.L1MessageType_EndOfBlock: @@ -109,7 +109,7 @@ func parseTimeOrPanic(format string, value string) time.Time { var HeartbeatsDisabledAt = uint64(parseTimeOrPanic(time.RFC1123, "Mon, 08 Aug 2022 16:00:00 GMT").Unix()) -func parseL2Message(rd io.Reader, poster common.Address, timestamp uint64, requestId *common.Hash, chainId *big.Int, depth int, arbOSVersion uint64) (types.Transactions, error) { +func parseL2Message(rd io.Reader, poster common.Address, timestamp uint64, requestId *common.Hash, chainId *big.Int, features arbostypes.Features, depth int) (types.Transactions, error) { var l2KindBuf [1]byte if _, err := rd.Read(l2KindBuf[:]); err != nil { return nil, err @@ -149,7 +149,7 @@ func parseL2Message(rd io.Reader, poster common.Address, timestamp uint64, reque subRequestId := crypto.Keccak256Hash(requestId[:], math.U256Bytes(index)) nextRequestId = &subRequestId } - nestedSegments, err := parseL2Message(bytes.NewReader(nextMsg), poster, timestamp, nextRequestId, chainId, depth+1, arbOSVersion) + nestedSegments, err := parseL2Message(bytes.NewReader(nextMsg), poster, timestamp, nextRequestId, chainId, features, depth+1) if err != nil { return nil, err } @@ -166,7 +166,8 @@ func parseL2Message(rd io.Reader, poster common.Address, timestamp uint64, reque if err := newTx.UnmarshalBinary(readBytes); err != nil { return nil, err } - if newTx.Type() == types.ArbitrumSubtypedTxType && arbOSVersion < arbostypes.ArbosVersion_ArbitrumSubtypedTx { + if newTx.Type() == types.ArbitrumSubtypedTxType && features.TxSubtypeSupported(types.GetArbitrumTxSubtype(newTx)) { + return nil, types.ErrTxTypeNotSupported } if newTx.Type() >= types.ArbitrumDepositTxType { diff --git a/arbos/tx_processor.go b/arbos/tx_processor.go index 5f2bcbe0e7..72bf5e9ee6 100644 --- a/arbos/tx_processor.go +++ b/arbos/tx_processor.go @@ -675,7 +675,8 @@ func (p *TxProcessor) DropTip() bool { version := p.state.ArbOSVersion() transaction := p.msg.Tx tippingTx := false - if version >= arbostypes.ArbosVersion_ArbitrumSubtypedTx && transaction != nil && transaction.Type() == types.ArbitrumSubtypedTxType { + // TODO(magic) + if version >= arbostypes.ArbosVersion_ArbitrumTippingTx && transaction != nil && transaction.Type() == types.ArbitrumSubtypedTxType { subtype := types.GetArbitrumTxSubtype(transaction) tippingTx = subtype == types.ArbitrumTippingTxSubtype } diff --git a/arbstate/das_reader.go b/arbstate/das_reader.go index a6d351b49e..6b82434c9a 100644 --- a/arbstate/das_reader.go +++ b/arbstate/das_reader.go @@ -34,8 +34,8 @@ const DASMessageHeaderFlag byte = 0x80 // Ignored when DASMessageHeaderFlag is not set. const TreeDASMessageHeaderFlag byte = 0x08 -// L1AuthenticatedMessageHeaderFlag indicates that this message was authenticated by L1. Currently unused. -const L1AuthenticatedMessageHeaderFlag byte = 0x40 +// FeaturesHeaderFlag indicates that TODO +const FeaturesHeaderFlag byte = 0x40 // ZeroheavyMessageHeaderFlag indicates that this message is zeroheavy-encoded. const ZeroheavyMessageHeaderFlag byte = 0x20 @@ -55,6 +55,10 @@ func IsZeroheavyEncodedHeaderByte(header byte) bool { return (ZeroheavyMessageHeaderFlag & header) > 0 } +func FeaturesFlagSet(header byte) bool { + return (FeaturesHeaderFlag & header) > 0 +} + func IsBrotliMessageHeaderByte(b uint8) bool { return b == BrotliMessageHeaderByte } diff --git a/arbstate/inbox.go b/arbstate/inbox.go index 3995bcf308..e32f8b926d 100644 --- a/arbstate/inbox.go +++ b/arbstate/inbox.go @@ -42,6 +42,7 @@ type sequencerMessage struct { minL1Block uint64 maxL1Block uint64 afterDelayedMessages uint64 + features uint8 segments [][]byte } @@ -60,6 +61,7 @@ func parseSequencerMessage(ctx context.Context, batchNum uint64, data []byte, da minL1Block: binary.BigEndian.Uint64(data[16:24]), maxL1Block: binary.BigEndian.Uint64(data[24:32]), afterDelayedMessages: binary.BigEndian.Uint64(data[32:40]), + features: 0, segments: [][]byte{}, } payload := data[40:] @@ -87,9 +89,22 @@ func parseSequencerMessage(ctx context.Context, batchNum uint64, data []byte, da } payload = pl } - - if len(payload) > 0 && IsBrotliMessageHeaderByte(payload[0]) { - decompressed, err := arbcompress.Decompress(payload[1:], MaxDecompressedLen) + var header *byte + if len(payload) > 0 { + header = &payload[0] + payload = payload[1:] + } + if header != nil && FeaturesFlagSet(*header) { + if len(payload) > 0 { + parsedMsg.features = payload[0] + payload = payload[1:] + } else { + log.Warn("failed to read sequencer message version") + return parsedMsg, nil + } + } + if header != nil && IsBrotliMessageHeaderByte(*header) { + decompressed, err := arbcompress.Decompress(payload, MaxDecompressedLen) if err == nil { reader := bytes.NewReader(decompressed) stream := rlp.NewStream(reader, uint64(MaxDecompressedLen)) @@ -112,13 +127,11 @@ func parseSequencerMessage(ctx context.Context, batchNum uint64, data []byte, da log.Warn("sequencer msg decompression failed", "err", err) } } else { - length := len(payload) - if length == 0 { + if header == nil { log.Warn("empty sequencer message") } else { - log.Warn("unknown sequencer message format", "length", length, "firstByte", payload[0]) + log.Warn("unknown sequencer message format", "length", len(payload)+1, "firstByte", *header) } - } return parsedMsg, nil @@ -427,6 +440,7 @@ func (r *inboxMultiplexer) getNextMsg() (*arbostypes.MessageWithMetadata, error) Timestamp: timestamp, RequestId: nil, L1BaseFee: big.NewInt(0), + Features: arbostypes.Features(seqMsg.features), }, L2msg: segment, }, diff --git a/broadcaster/broadcaster_serialization_test.go b/broadcaster/broadcaster_serialization_test.go index 64adb49126..488d60850c 100644 --- a/broadcaster/broadcaster_serialization_test.go +++ b/broadcaster/broadcaster_serialization_test.go @@ -29,6 +29,7 @@ func ExampleBroadcastMessage_broadcastfeedmessage() { Timestamp: 0, RequestId: &requestId, L1BaseFee: big.NewInt(0), + Features: 0, // TODO(magic) }, L2msg: []byte{0xde, 0xad, 0xbe, 0xef}, }, diff --git a/execution/gethexec/executionengine.go b/execution/gethexec/executionengine.go index c3e4155aa5..c428800834 100644 --- a/execution/gethexec/executionengine.go +++ b/execution/gethexec/executionengine.go @@ -223,10 +223,8 @@ func (s *ExecutionEngine) resequenceReorgedMessages(messages []*arbostypes.Messa log.Warn("skipping non-standard sequencer message found from reorg", "header", header) continue } - currentBlockHeader := s.bc.CurrentBlock() - arbOSVersion := types.DeserializeHeaderExtraInformation(currentBlockHeader).ArbOSFormatVersion // We don't need a batch fetcher as this is an L2 message - txes, err := arbos.ParseL2Transactions(msg.Message, s.bc.Config().ChainID, arbOSVersion, nil) + txes, err := arbos.ParseL2Transactions(msg.Message, s.bc.Config().ChainID, nil) if err != nil { log.Warn("failed to parse sequencer message found from reorg", "err", err) continue diff --git a/execution/gethexec/sequencer.go b/execution/gethexec/sequencer.go index 804841f690..a9085f23ab 100644 --- a/execution/gethexec/sequencer.go +++ b/execution/gethexec/sequencer.go @@ -429,7 +429,7 @@ func (s *Sequencer) PublishTransaction(parentCtx context.Context, tx *types.Tran } func (s *Sequencer) preTxFilter(_ *params.ChainConfig, header *types.Header, statedb *state.StateDB, arbState *arbosState.ArbosState, tx *types.Transaction, options *arbitrum_types.ConditionalOptions, sender common.Address, l1Info *arbos.L1Info) error { - if tx.Type() == types.ArbitrumSubtypedTxType && arbState.ArbOSVersion() < arbostypes.ArbosVersion_ArbitrumSubtypedTx { + if tx.Type() == types.ArbitrumSubtypedTxType && arbState.ArbOSVersion() < arbostypes.RequiredArobosVersionForTxSubtype(types.GetArbitrumTxSubtype(tx)) { return types.ErrTxTypeNotSupported } if s.nonceCache.Caching() { @@ -844,6 +844,7 @@ func (s *Sequencer) createBlock(ctx context.Context) (returnValue bool) { Timestamp: uint64(timestamp), RequestId: nil, L1BaseFee: nil, + Features: arbostypes.FeatureFlag_Invalid, } start := time.Now() diff --git a/gethhook/geth_test.go b/gethhook/geth_test.go index c8fe2051f9..62836c37ec 100644 --- a/gethhook/geth_test.go +++ b/gethhook/geth_test.go @@ -71,6 +71,7 @@ func TestEthDepositMessage(t *testing.T) { Timestamp: 8794561564, RequestId: &firstRequestId, L1BaseFee: big.NewInt(10000000000000), + Features: 0, // TODO(magic) } msgBuf := bytes.Buffer{} if err := util.AddressToWriter(addr, &msgBuf); err != nil { @@ -118,13 +119,12 @@ func TestEthDepositMessage(t *testing.T) { func RunMessagesThroughAPI(t *testing.T, msgs [][]byte, statedb *state.StateDB) { chainId := big.NewInt(6456554) - arbOSVersion := arbosState.ArbOSVersion(statedb) for _, data := range msgs { msg, err := arbostypes.ParseIncomingL1Message(bytes.NewReader(data), nil) if err != nil { t.Error(err) } - txes, err := arbos.ParseL2Transactions(msg, chainId, arbOSVersion, nil) + txes, err := arbos.ParseL2Transactions(msg, chainId, nil) if err != nil { t.Error(err) } diff --git a/system_tests/contract_tx_test.go b/system_tests/contract_tx_test.go index d6c2eb5f38..ec20fdc622 100644 --- a/system_tests/contract_tx_test.go +++ b/system_tests/contract_tx_test.go @@ -78,6 +78,7 @@ func TestContractTxDeploy(t *testing.T) { Timestamp: 0, RequestId: &contractTx.RequestId, L1BaseFee: &big.Int{}, + Features: 0, // TODO(magic) }, L2msg: l2Msg, BatchGasCost: new(uint64), diff --git a/system_tests/reorg_resequencing_test.go b/system_tests/reorg_resequencing_test.go index bdd4c4af45..75f55a1146 100644 --- a/system_tests/reorg_resequencing_test.go +++ b/system_tests/reorg_resequencing_test.go @@ -69,6 +69,7 @@ func TestReorgResequencing(t *testing.T) { Timestamp: 0, RequestId: &delayedIndexHash, L1BaseFee: common.Big0, + Features: 0, // TODO(magic) }, L2msg: append(l2info.GetAddress("User4").Bytes(), math.U256Bytes(big.NewInt(params.Ether))...), } diff --git a/system_tests/retryable_test.go b/system_tests/retryable_test.go index 0f5da38a88..0fc6d24ed0 100644 --- a/system_tests/retryable_test.go +++ b/system_tests/retryable_test.go @@ -75,7 +75,7 @@ func retryableSetup(t *testing.T) ( if !msgTypes[message.Message.Header.Kind] { continue } - txs, err := arbos.ParseL2Transactions(message.Message, params.ArbitrumDevTestChainConfig().ChainID, 10, nil) + txs, err := arbos.ParseL2Transactions(message.Message, params.ArbitrumDevTestChainConfig().ChainID, nil) Require(t, err) for _, tx := range txs { if txTypes[tx.Type()] { diff --git a/system_tests/seq_coordinator_test.go b/system_tests/seq_coordinator_test.go index a213c366cf..d2c20f824e 100644 --- a/system_tests/seq_coordinator_test.go +++ b/system_tests/seq_coordinator_test.go @@ -80,6 +80,7 @@ func TestRedisSeqCoordinatorPriorities(t *testing.T) { Timestamp: 0, RequestId: &common.Hash{}, L1BaseFee: common.Big0, + Features: 0, // TODO(magic) }, L2msg: nil, }, diff --git a/system_tests/tippingtx_test.go b/system_tests/tippingtx_test.go index 01390511b4..ed619d5407 100644 --- a/system_tests/tippingtx_test.go +++ b/system_tests/tippingtx_test.go @@ -110,7 +110,7 @@ func TestTippingTxJsonRPC(t *testing.T) { defer cancel() chainConfig := params.ArbitrumDevTestChainConfig() // make sure ArbOSVersion supports ArbitrumSubtypedTx - chainConfig.ArbitrumChainParams.InitialArbOSVersion = arbmath.MaxInt(arbostypes.ArbosVersion_ArbitrumSubtypedTx, chainConfig.ArbitrumChainParams.InitialArbOSVersion) + chainConfig.ArbitrumChainParams.InitialArbOSVersion = arbmath.MaxInt(arbostypes.ArbosVersion_ArbitrumTippingTx, chainConfig.ArbitrumChainParams.InitialArbOSVersion) l2info, l2node, l2client, _, _, _, l1stack := createTestNodeOnL1WithConfig(t, ctx, true, nil, nil, chainConfig, nil) defer requireClose(t, l1stack) defer l2node.StopAndWait() @@ -153,7 +153,7 @@ func TestTippingTxTipPaid(t *testing.T) { defer cancel() chainConfig := params.ArbitrumDevTestChainConfig() // make sure ArbOSVersion supports ArbitrumSubtypedTx - chainConfig.ArbitrumChainParams.InitialArbOSVersion = arbmath.MaxInt(arbostypes.ArbosVersion_ArbitrumSubtypedTx, chainConfig.ArbitrumChainParams.InitialArbOSVersion) + chainConfig.ArbitrumChainParams.InitialArbOSVersion = arbmath.MaxInt(arbostypes.ArbosVersion_ArbitrumTippingTx, chainConfig.ArbitrumChainParams.InitialArbOSVersion) l2info, l2node, l2client, _, _, _, l1stack := createTestNodeOnL1WithConfig(t, ctx, true, nil, nil, chainConfig, nil) defer requireClose(t, l1stack) defer l2node.StopAndWait() From 0f2ab8cf6017b2630e8fcdb7236ad547fe179735 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Tue, 31 Oct 2023 17:19:15 +0000 Subject: [PATCH 2/6] remove uneeded L1MessageType_L1MessageWithFeatures --- arbos/arbostypes/incomingmessage.go | 1 - 1 file changed, 1 deletion(-) diff --git a/arbos/arbostypes/incomingmessage.go b/arbos/arbostypes/incomingmessage.go index d54b8401d5..84062e4d1d 100644 --- a/arbos/arbostypes/incomingmessage.go +++ b/arbos/arbostypes/incomingmessage.go @@ -31,7 +31,6 @@ const ( L1MessageType_Initialize = 11 L1MessageType_EthDeposit = 12 L1MessageType_BatchPostingReport = 13 - L1MessageType_L1MessageWithFeatures = 14 L1MessageType_Invalid = 0xFF ) From 33c8c0ffe74da9af968bcf22b07d17030efd61c4 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Fri, 3 Nov 2023 16:26:23 +0000 Subject: [PATCH 3/6] system_tests: pass chainConfig from NodeBuilder also when creating L2-only node --- system_tests/common_test.go | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/system_tests/common_test.go b/system_tests/common_test.go index 6d41f8e101..976e05aba4 100644 --- a/system_tests/common_test.go +++ b/system_tests/common_test.go @@ -186,7 +186,7 @@ func (b *NodeBuilder) Build(t *testing.T) func() { } else { l2 := NewTestClient(b.ctx) b.L2Info, l2.ConsensusNode, l2.Client = - createTestNode(t, b.ctx, b.L2Info, b.nodeConfig, b.execConfig, b.takeOwnership) + createTestNode(t, b.ctx, b.L2Info, b.nodeConfig, b.execConfig, b.chainConfig, b.takeOwnership) b.L2 = l2 } b.L2.ExecNode = getExecNode(t, b.L2.ConsensusNode) @@ -791,7 +791,7 @@ func createTestNodeWithL1( // L2 -Only. Enough for tests that needs no interface to L1 // Requires precompiles.AllowDebugPrecompiles = true func createTestNode( - t *testing.T, ctx context.Context, l2Info *BlockchainTestInfo, nodeConfig *arbnode.Config, execConfig *gethexec.Config, takeOwnership bool, + t *testing.T, ctx context.Context, l2Info *BlockchainTestInfo, nodeConfig *arbnode.Config, execConfig *gethexec.Config, chainConfig *params.ChainConfig, takeOwnership bool, ) (*BlockchainTestInfo, *arbnode.Node, *ethclient.Client) { if nodeConfig == nil { nodeConfig = arbnode.ConfigDefaultL2Test() @@ -799,12 +799,15 @@ func createTestNode( if execConfig == nil { execConfig = gethexec.ConfigDefaultTest() } + if chainConfig == nil { + chainConfig = params.ArbitrumDevTestChainConfig() + } feedErrChan := make(chan error, 10) AddDefaultValNode(t, ctx, nodeConfig, true) - l2info, stack, chainDb, arbDb, blockchain := createL2BlockChain(t, l2Info, "", params.ArbitrumDevTestChainConfig(), &execConfig.Caching) + l2info, stack, chainDb, arbDb, blockchain := createL2BlockChain(t, l2Info, "", chainConfig, &execConfig.Caching) Require(t, execConfig.Validate()) execConfigFetcher := func() *gethexec.Config { return execConfig } From 336fe671da934b36b09aa9a8802a9f5d5b54b871 Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Mon, 27 Nov 2023 15:02:31 +0000 Subject: [PATCH 4/6] remove invalid feature flag --- arbnode/delayed.go | 2 +- arbos/arbostypes/incomingmessage.go | 5 ++--- arbos/block_processor.go | 1 - execution/gethexec/executionengine.go | 8 ++++++-- execution/gethexec/sequencer.go | 2 +- 5 files changed, 10 insertions(+), 8 deletions(-) diff --git a/arbnode/delayed.go b/arbnode/delayed.go index db80267ffa..7099bf8f69 100644 --- a/arbnode/delayed.go +++ b/arbnode/delayed.go @@ -211,7 +211,7 @@ func (b *DelayedBridge) logsToDeliveredMessages(ctx context.Context, logs []type Timestamp: parsedLog.Timestamp, RequestId: &requestId, L1BaseFee: parsedLog.BaseFeeL1, - Features: arbostypes.FeatureFlag_Invalid, + Features: 0, // TODO (magic) comment, for now it seems that we don't need any features set here }, L2msg: data, }, diff --git a/arbos/arbostypes/incomingmessage.go b/arbos/arbostypes/incomingmessage.go index 84062e4d1d..5b76d1d57f 100644 --- a/arbos/arbostypes/incomingmessage.go +++ b/arbos/arbostypes/incomingmessage.go @@ -51,9 +51,8 @@ func RequiredArobosVersionForTxSubtype(txSubtype uint8) uint64 { type Features uint8 const ( - FeatureFlag_Invalid Features = (1 << 0) // when set indicates that feature flags value is not set to valid value - FeatureFlag_ArbitrumTippingTx Features = (1 << 1) - FeatureFlag_Reserved Features = (1 << 7) // could be used to implement future feature format versioning where version = feature_flags & 0x7f + FeatureFlag_ArbitrumTippingTx Features = (1 << 0) + FeatureFlag_Reserved Features = (1 << 7) // could be used to implement future feature format versioning e.g. where version = feature_flags & 0x7f ) func (f Features) TxSubtypeSupported(subtype uint8) bool { diff --git a/arbos/block_processor.go b/arbos/block_processor.go index 87e88455df..fab81ff9ca 100644 --- a/arbos/block_processor.go +++ b/arbos/block_processor.go @@ -138,7 +138,6 @@ func ProduceBlock( if err != nil { return nil, nil, err } - message.Header.Features = arbostypes.ArbosVersionBasedFeatureFlags(arbState.ArbOSVersion()) var batchFetchErr error txes, err := ParseL2Transactions(message, chainConfig.ChainID, func(batchNum uint64, batchHash common.Hash) []byte { data, err := batchFetcher(batchNum) diff --git a/execution/gethexec/executionengine.go b/execution/gethexec/executionengine.go index c428800834..1580023f32 100644 --- a/execution/gethexec/executionengine.go +++ b/execution/gethexec/executionengine.go @@ -285,9 +285,13 @@ func (s *ExecutionEngine) sequenceTransactionsWithBlockMutex(header *arbostypes. if err != nil { return nil, err } + arbState, err := arbosState.OpenSystemArbosState(statedb, nil, true) + if err != nil { + return nil, err + } delayedMessagesRead := lastBlockHeader.Nonce.Uint64() - + header.Features = arbostypes.ArbosVersionBasedFeatureFlags(arbState.ArbOSVersion()) startTime := time.Now() block, receipts, err := arbos.ProduceBlockAdvanced( header, @@ -295,7 +299,7 @@ func (s *ExecutionEngine) sequenceTransactionsWithBlockMutex(header *arbostypes. delayedMessagesRead, lastBlockHeader, statedb, - nil, + arbState, s.bc, s.bc.Config(), hooks, diff --git a/execution/gethexec/sequencer.go b/execution/gethexec/sequencer.go index a9085f23ab..30f98155d4 100644 --- a/execution/gethexec/sequencer.go +++ b/execution/gethexec/sequencer.go @@ -844,7 +844,7 @@ func (s *Sequencer) createBlock(ctx context.Context) (returnValue bool) { Timestamp: uint64(timestamp), RequestId: nil, L1BaseFee: nil, - Features: arbostypes.FeatureFlag_Invalid, + Features: 0, // TODO(magic) } start := time.Now() From 12aebb2610e4a5b95f52cf041f0e6641a2022ead Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Mon, 27 Nov 2023 15:07:47 +0000 Subject: [PATCH 5/6] update geth --- go-ethereum | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go-ethereum b/go-ethereum index ce5b73757e..873384d04e 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit ce5b73757e12a4df8ebab1369e23e3a35c0fffe8 +Subproject commit 873384d04e1693f207c6c0dee34510098bd21632 From 8b9ef263c1c9420c56f96e7e4fd96fc0e22a9c2c Mon Sep 17 00:00:00 2001 From: Maciej Kulawik Date: Mon, 27 Nov 2023 15:24:53 +0000 Subject: [PATCH 6/6] fix features check in parse_l2 --- arbos/parse_l2.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/arbos/parse_l2.go b/arbos/parse_l2.go index 85f004c0d8..5e8b6b6328 100644 --- a/arbos/parse_l2.go +++ b/arbos/parse_l2.go @@ -166,7 +166,7 @@ func parseL2Message(rd io.Reader, poster common.Address, timestamp uint64, reque if err := newTx.UnmarshalBinary(readBytes); err != nil { return nil, err } - if newTx.Type() == types.ArbitrumSubtypedTxType && features.TxSubtypeSupported(types.GetArbitrumTxSubtype(newTx)) { + if newTx.Type() == types.ArbitrumSubtypedTxType && !features.TxSubtypeSupported(types.GetArbitrumTxSubtype(newTx)) { return nil, types.ErrTxTypeNotSupported }