diff --git a/execution/gethexec/sequencer.go b/execution/gethexec/sequencer.go index c5256c04f3..78ec61e594 100644 --- a/execution/gethexec/sequencer.go +++ b/execution/gethexec/sequencer.go @@ -504,6 +504,78 @@ func (s *Sequencer) PublishTransaction(parentCtx context.Context, tx *types.Tran } } +func (s *Sequencer) PublishAuctionResolutionTransaction(ctx context.Context, tx *types.Transaction) error { + if !s.config().Timeboost.Enable { + return errors.New("timeboost not enabled") + } + + _, forwarder := s.GetPauseAndForwarder() + if forwarder != nil { + return fmt.Errorf("sequencer is currently not the chosen one, cannot accept auction resolution tx") + } + + arrivalTime := time.Now() + auctioneerAddr := s.auctioneerAddr + if auctioneerAddr == (common.Address{}) { + return errors.New("invalid auctioneer address") + } + if tx.To() == nil { + return errors.New("transaction has no recipient") + } + if *tx.To() != s.expressLaneService.auctionContractAddr { + return errors.New("transaction recipient is not the auction contract") + } + signer := types.LatestSigner(s.execEngine.bc.Config()) + sender, err := types.Sender(signer, tx) + if err != nil { + return err + } + if sender != auctioneerAddr { + return fmt.Errorf("sender %#x is not the auctioneer address %#x", sender, auctioneerAddr) + } + if !s.expressLaneService.roundTimingInfo.IsWithinAuctionCloseWindow(arrivalTime) { + return fmt.Errorf("transaction arrival time not within auction closure window: %v", arrivalTime) + } + txBytes, err := tx.MarshalBinary() + if err != nil { + return err + } + log.Info("Prioritizing auction resolution transaction from auctioneer", "txHash", tx.Hash().Hex()) + s.timeboostAuctionResolutionTxQueue <- txQueueItem{ + tx: tx, + txSize: len(txBytes), + options: nil, + resultChan: make(chan error, 1), + returnedResult: &atomic.Bool{}, + ctx: context.TODO(), + firstAppearance: time.Now(), + isTimeboosted: true, + } + return nil +} + +func (s *Sequencer) PublishExpressLaneTransaction(ctx context.Context, msg *timeboost.ExpressLaneSubmission) error { + if !s.config().Timeboost.Enable { + return errors.New("timeboost not enabled") + } + + _, forwarder := s.GetPauseAndForwarder() + if forwarder != nil { + err := forwarder.PublishExpressLaneTransaction(ctx, msg) + if !errors.Is(err, ErrNoSequencer) { + return err + } + } + + if s.expressLaneService == nil { + return errors.New("express lane service not enabled") + } + if err := s.expressLaneService.validateExpressLaneTx(msg); err != nil { + return err + } + return s.expressLaneService.sequenceExpressLaneSubmission(ctx, msg) +} + func (s *Sequencer) PublishTimeboostedTransaction(queueCtx context.Context, tx *types.Transaction, options *arbitrum_types.ConditionalOptions, resultChan chan error) error { return s.publishTransactionToQueue(queueCtx, tx, options, resultChan, true) // Is it safe to ignore queueCtx's CancelFunc here? } @@ -570,63 +642,6 @@ func (s *Sequencer) publishTransactionToQueue(queueCtx context.Context, tx *type return nil } -func (s *Sequencer) PublishExpressLaneTransaction(ctx context.Context, msg *timeboost.ExpressLaneSubmission) error { - if !s.config().Timeboost.Enable { - return errors.New("timeboost not enabled") - } - if s.expressLaneService == nil { - return errors.New("express lane service not enabled") - } - if err := s.expressLaneService.validateExpressLaneTx(msg); err != nil { - return err - } - return s.expressLaneService.sequenceExpressLaneSubmission(ctx, msg) -} - -func (s *Sequencer) PublishAuctionResolutionTransaction(ctx context.Context, tx *types.Transaction) error { - if !s.config().Timeboost.Enable { - return errors.New("timeboost not enabled") - } - arrivalTime := time.Now() - auctioneerAddr := s.auctioneerAddr - if auctioneerAddr == (common.Address{}) { - return errors.New("invalid auctioneer address") - } - if tx.To() == nil { - return errors.New("transaction has no recipient") - } - if *tx.To() != s.expressLaneService.auctionContractAddr { - return errors.New("transaction recipient is not the auction contract") - } - signer := types.LatestSigner(s.execEngine.bc.Config()) - sender, err := types.Sender(signer, tx) - if err != nil { - return err - } - if sender != auctioneerAddr { - return fmt.Errorf("sender %#x is not the auctioneer address %#x", sender, auctioneerAddr) - } - if !s.expressLaneService.roundTimingInfo.IsWithinAuctionCloseWindow(arrivalTime) { - return fmt.Errorf("transaction arrival time not within auction closure window: %v", arrivalTime) - } - txBytes, err := tx.MarshalBinary() - if err != nil { - return err - } - log.Info("Prioritizing auction resolution transaction from auctioneer", "txHash", tx.Hash().Hex()) - s.timeboostAuctionResolutionTxQueue <- txQueueItem{ - tx: tx, - txSize: len(txBytes), - options: nil, - resultChan: make(chan error, 1), - returnedResult: &atomic.Bool{}, - ctx: context.TODO(), - firstAppearance: time.Now(), - isTimeboosted: true, - } - return nil -} - func (s *Sequencer) preTxFilter(_ *params.ChainConfig, header *types.Header, statedb *state.StateDB, _ *arbosState.ArbosState, tx *types.Transaction, options *arbitrum_types.ConditionalOptions, sender common.Address, l1Info *arbos.L1Info) error { if s.nonceCache.Caching() { stateNonce := s.nonceCache.Get(header, statedb, sender) diff --git a/system_tests/timeboost_test.go b/system_tests/timeboost_test.go index edab8a10f0..ad658275fe 100644 --- a/system_tests/timeboost_test.go +++ b/system_tests/timeboost_test.go @@ -53,6 +53,51 @@ import ( "github.com/offchainlabs/nitro/util/testhelpers" ) +func TestForwardingExpressLaneTxs(t *testing.T) { + t.Parallel() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + tmpDir, err := os.MkdirTemp("", "*") + require.NoError(t, err) + t.Cleanup(func() { + require.NoError(t, os.RemoveAll(tmpDir)) + }) + jwtSecretPath := filepath.Join(tmpDir, "sequencer.jwt") + + _, seqClient, seqInfo, auctionContractAddr, aliceBidderClient, bobBidderClient, roundDuration, cleanupSeq, forwarder, cleanupForwarder := setupExpressLaneAuction(t, tmpDir, ctx, jwtSecretPath, withForwardingSeq) + defer cleanupSeq() + defer cleanupForwarder() + + auctionContract, err := express_lane_auctiongen.NewExpressLaneAuction(auctionContractAddr, seqClient) + Require(t, err) + rawRoundTimingInfo, err := auctionContract.RoundTimingInfo(&bind.CallOpts{}) + Require(t, err) + roundTimingInfo, err := timeboost.NewRoundTimingInfo(rawRoundTimingInfo) + Require(t, err) + + placeBidsAndDecideWinner(t, ctx, seqClient, seqInfo, auctionContract, "Bob", "Alice", bobBidderClient, aliceBidderClient, roundDuration) + time.Sleep(roundTimingInfo.TimeTilNextRound()) + + chainId, err := seqClient.ChainID(ctx) + Require(t, err) + + // Prepare a client that can submit txs to the sequencer via the express lane. + bobPriv := seqInfo.Accounts["Bob"].PrivateKey + forwardingSeqDial, err := rpc.Dial(forwarder.ConsensusNode.Stack.HTTPEndpoint()) + Require(t, err) + expressLaneClient := newExpressLaneClient( + bobPriv, + chainId, + *roundTimingInfo, + auctionContractAddr, + forwardingSeqDial, + ) + expressLaneClient.Start(ctx) + + verifyControllerAdvantage(t, ctx, seqClient, expressLaneClient, seqInfo, "Bob", "Alice") +} + func TestExpressLaneTransactionHandlingComplex(t *testing.T) { t.Parallel() ctx, cancel := context.WithCancel(context.Background()) @@ -65,9 +110,8 @@ func TestExpressLaneTransactionHandlingComplex(t *testing.T) { }) jwtSecretPath := filepath.Join(tmpDir, "sequencer.jwt") - seq, seqClient, seqInfo, auctionContractAddr, aliceBidderClient, bobBidderClient, roundDuration, cleanupSeq, _, cleanupFeedListener := setupExpressLaneAuction(t, tmpDir, ctx, jwtSecretPath) + seq, seqClient, seqInfo, auctionContractAddr, aliceBidderClient, bobBidderClient, roundDuration, cleanupSeq, _, _ := setupExpressLaneAuction(t, tmpDir, ctx, jwtSecretPath, 0) defer cleanupSeq() - defer cleanupFeedListener() auctionContract, err := express_lane_auctiongen.NewExpressLaneAuction(auctionContractAddr, seqClient) Require(t, err) @@ -163,9 +207,8 @@ func TestExpressLaneTransactionHandling(t *testing.T) { }) jwtSecretPath := filepath.Join(tmpDir, "sequencer.jwt") - seq, seqClient, seqInfo, auctionContractAddr, aliceBidderClient, bobBidderClient, roundDuration, cleanupSeq, _, cleanupFeedListener := setupExpressLaneAuction(t, tmpDir, ctx, jwtSecretPath) + seq, seqClient, seqInfo, auctionContractAddr, aliceBidderClient, bobBidderClient, roundDuration, cleanupSeq, _, _ := setupExpressLaneAuction(t, tmpDir, ctx, jwtSecretPath, 0) defer cleanupSeq() - defer cleanupFeedListener() auctionContract, err := express_lane_auctiongen.NewExpressLaneAuction(auctionContractAddr, seqClient) Require(t, err) @@ -656,9 +699,8 @@ func TestExpressLaneControlTransfer(t *testing.T) { }) jwtSecretPath := filepath.Join(tmpDir, "sequencer.jwt") - seq, seqClient, seqInfo, auctionContractAddr, aliceBidderClient, bobBidderClient, roundDuration, cleanupSeq, _, cleanupFeedListener := setupExpressLaneAuction(t, tmpDir, ctx, jwtSecretPath) + seq, seqClient, seqInfo, auctionContractAddr, aliceBidderClient, bobBidderClient, roundDuration, cleanupSeq, _, _ := setupExpressLaneAuction(t, tmpDir, ctx, jwtSecretPath, 0) defer cleanupSeq() - defer cleanupFeedListener() auctionContract, err := express_lane_auctiongen.NewExpressLaneAuction(auctionContractAddr, seqClient) Require(t, err) @@ -757,9 +799,8 @@ func TestSequencerFeed_ExpressLaneAuction_ExpressLaneTxsHaveAdvantage(t *testing }) jwtSecretPath := filepath.Join(tmpDir, "sequencer.jwt") - seq, seqClient, seqInfo, auctionContractAddr, aliceBidderClient, bobBidderClient, roundDuration, cleanupSeq, _, cleanupFeedListener := setupExpressLaneAuction(t, tmpDir, ctx, jwtSecretPath) + seq, seqClient, seqInfo, auctionContractAddr, aliceBidderClient, bobBidderClient, roundDuration, cleanupSeq, _, _ := setupExpressLaneAuction(t, tmpDir, ctx, jwtSecretPath, 0) defer cleanupSeq() - defer cleanupFeedListener() auctionContract, err := express_lane_auctiongen.NewExpressLaneAuction(auctionContractAddr, seqClient) Require(t, err) @@ -803,7 +844,7 @@ func TestSequencerFeed_ExpressLaneAuction_InnerPayloadNoncesAreRespected_Timeboo require.NoError(t, os.RemoveAll(tmpDir)) }) jwtSecretPath := filepath.Join(tmpDir, "sequencer.jwt") - seq, seqClient, seqInfo, auctionContractAddr, aliceBidderClient, bobBidderClient, roundDuration, cleanupSeq, feedListener, cleanupFeedListener := setupExpressLaneAuction(t, tmpDir, ctx, jwtSecretPath) + seq, seqClient, seqInfo, auctionContractAddr, aliceBidderClient, bobBidderClient, roundDuration, cleanupSeq, feedListener, cleanupFeedListener := setupExpressLaneAuction(t, tmpDir, ctx, jwtSecretPath, withFeedListener) defer cleanupSeq() defer cleanupFeedListener() @@ -1067,11 +1108,19 @@ func verifyControllerAdvantage(t *testing.T, ctx context.Context, seqClient *eth } } +type extraNodeType int + +const ( + withForwardingSeq extraNodeType = iota + 1 + withFeedListener +) + func setupExpressLaneAuction( t *testing.T, dbDirPath string, ctx context.Context, jwtSecretPath string, + extraNodeTy extraNodeType, ) (*arbnode.Node, *ethclient.Client, *BlockchainTestInfo, common.Address, *timeboost.BidderClient, *timeboost.BidderClient, time.Duration, func(), *TestClient, func()) { builderSeq := NewNodeBuilder(ctx).DefaultConfig(t, true) @@ -1094,15 +1143,28 @@ func setupExpressLaneAuction( cleanupSeq := builderSeq.Build(t) seqInfo, seqNode, seqClient := builderSeq.L2Info, builderSeq.L2.ConsensusNode, builderSeq.L2.Client - tcpAddr, ok := seqNode.BroadcastServer.ListenerAddr().(*net.TCPAddr) - if !ok { - t.Fatalf("failed to cast listener address to *net.TCPAddr") + var extraNode *TestClient + var cleanupExtraNode func() + switch extraNodeTy { + case withForwardingSeq: + forwarderNodeCfg := arbnode.ConfigDefaultL1Test() + forwarderNodeCfg.BatchPoster.Enable = false + builderSeq.l2StackConfig.HTTPPort = getRandomPort(t) + builderSeq.l2StackConfig.AuthPort = getRandomPort(t) + builderSeq.l2StackConfig.JWTSecret = jwtSecretPath + extraNode, cleanupExtraNode = builderSeq.Build2ndNode(t, &SecondNodeParams{nodeConfig: forwarderNodeCfg}) + Require(t, extraNode.ExecNode.ForwardTo(seqNode.Stack.HTTPEndpoint())) + case withFeedListener: + tcpAddr, ok := seqNode.BroadcastServer.ListenerAddr().(*net.TCPAddr) + if !ok { + t.Fatalf("failed to cast listener address to *net.TCPAddr") + } + port := tcpAddr.Port + nodeConfig := arbnode.ConfigDefaultL1NonSequencerTest() + nodeConfig.Feed.Input = *newBroadcastClientConfigTest(port) + nodeConfig.Feed.Input.Timeout = broadcastclient.DefaultConfig.Timeout + extraNode, cleanupExtraNode = builderSeq.Build2ndNode(t, &SecondNodeParams{nodeConfig: nodeConfig, stackConfig: testhelpers.CreateStackConfigForTest(t.TempDir())}) } - port := tcpAddr.Port - nodeConfig := arbnode.ConfigDefaultL1NonSequencerTest() - nodeConfig.Feed.Input = *newBroadcastClientConfigTest(port) - nodeConfig.Feed.Input.Timeout = broadcastclient.DefaultConfig.Timeout - feedListener, cleanupFeedListener := builderSeq.Build2ndNode(t, &SecondNodeParams{nodeConfig: nodeConfig, stackConfig: testhelpers.CreateStackConfigForTest(t.TempDir())}) // Send an L2 tx in the background every two seconds to keep the chain moving. go func() { @@ -1377,7 +1439,7 @@ func setupExpressLaneAuction( time.Sleep(roundTimingInfo.TimeTilNextRound()) t.Logf("Reached the bidding round at %v", time.Now()) time.Sleep(time.Second * 5) - return seqNode, seqClient, seqInfo, proxyAddr, alice, bob, roundDuration, cleanupSeq, feedListener, cleanupFeedListener + return seqNode, seqClient, seqInfo, proxyAddr, alice, bob, roundDuration, cleanupSeq, extraNode, cleanupExtraNode } func awaitAuctionResolved(