diff --git a/Makefile b/Makefile index 96f15c20ce..b828b6c19c 100644 --- a/Makefile +++ b/Makefile @@ -346,10 +346,9 @@ contracts/test/prover/proofs/%.json: $(arbitrator_cases)/%.wasm $(arbitrator_pro test -f target/lib-wasm/libbrotlidec-static.a || ./scripts/build-brotli.sh -w -d @touch $@ -.make/wasm-lib: $(DEP_PREDICATE) $(ORDER_ONLY_PREDICATE) .make +.make/wasm-lib: $(DEP_PREDICATE) arbitrator/wasm-libraries/soft-float/SoftFloat/build/Wasm-Clang/softfloat.a $(ORDER_ONLY_PREDICATE) .make test -f arbitrator/wasm-libraries/soft-float/bindings32.o || ./scripts/build-brotli.sh -f -d -t . test -f arbitrator/wasm-libraries/soft-float/bindings64.o || ./scripts/build-brotli.sh -f -d -t . - test -f arbitrator/wasm-libraries/soft-float/SoftFloat/build/Wasm-Clang/softfloat.a || ./scripts/build-brotli.sh -f -d -t . @touch $@ .make: diff --git a/arbnode/batch_poster.go b/arbnode/batch_poster.go index 5aa07f5157..0676903966 100644 --- a/arbnode/batch_poster.go +++ b/arbnode/batch_poster.go @@ -9,6 +9,7 @@ import ( "encoding/hex" "errors" "fmt" + "math" "math/big" "sync/atomic" "time" @@ -73,6 +74,18 @@ type BatchPoster struct { batchReverted atomic.Bool // indicates whether data poster batch was reverted } +type l1BlockBound int + +// This enum starts at 1 to avoid the empty initialization of 0 being valid +const ( + // Default is Safe if the L1 reader has finality data enabled, otherwise Latest + l1BlockBoundDefault l1BlockBound = iota + 1 + l1BlockBoundSafe + l1BlockBoundFinalized + l1BlockBoundLatest + l1BlockBoundIgnore +) + type BatchPosterConfig struct { Enable bool `koanf:"enable"` DisableDasFallbackStoreDataOnChain bool `koanf:"disable-das-fallback-store-data-on-chain" reload:"hot"` @@ -89,8 +102,11 @@ type BatchPosterConfig struct { RedisLock SimpleRedisLockConfig `koanf:"redis-lock" reload:"hot"` ExtraBatchGas uint64 `koanf:"extra-batch-gas" reload:"hot"` L1Wallet genericconf.WalletConfig `koanf:"parent-chain-wallet"` + L1BlockBound string `koanf:"l1-block-bound" reload:"hot"` + L1BlockBoundBypass time.Duration `koanf:"l1-block-bound-bypass" reload:"hot"` - gasRefunder common.Address + gasRefunder common.Address + l1BlockBound l1BlockBound } func (c *BatchPosterConfig) Validate() error { @@ -101,6 +117,19 @@ func (c *BatchPosterConfig) Validate() error { if c.MaxBatchSize <= 40 { return errors.New("MaxBatchSize too small") } + if c.L1BlockBound == "" { + c.l1BlockBound = l1BlockBoundDefault + } else if c.L1BlockBound == "safe" { + c.l1BlockBound = l1BlockBoundSafe + } else if c.L1BlockBound == "finalized" { + c.l1BlockBound = l1BlockBoundFinalized + } else if c.L1BlockBound == "latest" { + c.l1BlockBound = l1BlockBoundLatest + } else if c.L1BlockBound == "ignore" { + c.l1BlockBound = l1BlockBoundIgnore + } else { + return fmt.Errorf("invalid L1 block bound tag \"%v\" (see --help for options)", c.L1BlockBound) + } return nil } @@ -119,6 +148,8 @@ func BatchPosterConfigAddOptions(prefix string, f *pflag.FlagSet) { f.String(prefix+".gas-refunder-address", DefaultBatchPosterConfig.GasRefunderAddress, "The gas refunder contract address (optional)") f.Uint64(prefix+".extra-batch-gas", DefaultBatchPosterConfig.ExtraBatchGas, "use this much more gas than estimation says is necessary to post batches") f.String(prefix+".redis-url", DefaultBatchPosterConfig.RedisUrl, "if non-empty, the Redis URL to store queued transactions in") + f.String(prefix+".l1-block-bound", DefaultBatchPosterConfig.L1BlockBound, "only post messages to batches when they're within the max future block/timestamp as of this L1 block tag (\"safe\", \"finalized\", \"latest\", or \"ignore\" to ignore this check)") + f.Duration(prefix+".l1-block-bound-bypass", DefaultBatchPosterConfig.L1BlockBoundBypass, "post batches even if not within the layer 1 future bounds if we're within this margin of the max delay") RedisLockConfigAddOptions(prefix+".redis-lock", f) dataposter.DataPosterConfigAddOptions(prefix+".data-poster", f) genericconf.WalletConfigAddOptions(prefix+".parent-chain-wallet", f, DefaultBatchPosterConfig.L1Wallet.Pathname) @@ -138,6 +169,8 @@ var DefaultBatchPosterConfig = BatchPosterConfig{ ExtraBatchGas: 50_000, DataPoster: dataposter.DefaultDataPosterConfig, L1Wallet: DefaultBatchPosterL1WalletConfig, + L1BlockBound: "", + L1BlockBoundBypass: time.Hour, } var DefaultBatchPosterL1WalletConfig = genericconf.WalletConfig{ @@ -161,6 +194,8 @@ var TestBatchPosterConfig = BatchPosterConfig{ ExtraBatchGas: 10_000, DataPoster: dataposter.TestDataPosterConfig, L1Wallet: DefaultBatchPosterL1WalletConfig, + L1BlockBound: "", + L1BlockBoundBypass: time.Hour, } func NewBatchPoster(dataPosterDB ethdb.Database, l1Reader *headerreader.HeaderReader, inbox *InboxTracker, streamer *TransactionStreamer, syncMonitor *SyncMonitor, config BatchPosterConfigFetcher, deployInfo *chaininfo.RollupAddresses, transactOpts *bind.TransactOpts, daWriter das.DataAvailabilityServiceWriter) (*BatchPoster, error) { @@ -636,6 +671,8 @@ func (b *BatchPoster) estimateGas(ctx context.Context, sequencerMessage []byte, return gas + config.ExtraBatchGas, nil } +const ethPosBlockTime = 12 * time.Second + func (b *BatchPoster) maybePostSequencerBatch(ctx context.Context) (bool, error) { if b.batchReverted.Load() { return false, fmt.Errorf("batch was reverted, not posting any more batches") @@ -677,12 +714,94 @@ func (b *BatchPoster) maybePostSequencerBatch(ctx context.Context) (bool, error) config := b.config() forcePostBatch := time.Since(firstMsgTime) >= config.MaxBatchPostDelay + var l1BoundMaxBlockNumber uint64 = math.MaxUint64 + var l1BoundMaxTimestamp uint64 = math.MaxUint64 + var l1BoundMinBlockNumber uint64 + var l1BoundMinTimestamp uint64 + hasL1Bound := config.l1BlockBound != l1BlockBoundIgnore + if hasL1Bound { + var l1Bound *types.Header + var err error + if config.l1BlockBound == l1BlockBoundLatest { + l1Bound, err = b.l1Reader.LastHeader(ctx) + } else if config.l1BlockBound == l1BlockBoundSafe || config.l1BlockBound == l1BlockBoundDefault { + l1Bound, err = b.l1Reader.LatestSafeBlockHeader(ctx) + if errors.Is(err, headerreader.ErrBlockNumberNotSupported) && config.l1BlockBound == l1BlockBoundDefault { + // If getting the latest safe block is unsupported, and the L1BlockBound configuration is the default, + // fall back to using the latest block instead of the safe block. + l1Bound, err = b.l1Reader.LastHeader(ctx) + } + } else { + if config.l1BlockBound != l1BlockBoundFinalized { + log.Error( + "unknown L1 block bound config value; falling back on using finalized", + "l1BlockBoundString", config.L1BlockBound, + "l1BlockBoundEnum", config.l1BlockBound, + ) + } + l1Bound, err = b.l1Reader.LatestFinalizedBlockHeader(ctx) + } + if err != nil { + return false, fmt.Errorf("error getting L1 bound block: %w", err) + } + + maxTimeVariation, err := b.seqInbox.MaxTimeVariation(&bind.CallOpts{ + Context: ctx, + BlockNumber: l1Bound.Number, + }) + if err != nil { + // This might happen if the latest finalized block is old enough that our L1 node no longer has its state + log.Warn("error getting max time variation on L1 bound block; falling back on latest block", "err", err) + maxTimeVariation, err = b.seqInbox.MaxTimeVariation(&bind.CallOpts{Context: ctx}) + if err != nil { + return false, fmt.Errorf("error getting max time variation: %w", err) + } + } + + l1BoundBlockNumber := arbutil.ParentHeaderToL1BlockNumber(l1Bound) + l1BoundMaxBlockNumber = arbmath.SaturatingUAdd(l1BoundBlockNumber, arbmath.BigToUintSaturating(maxTimeVariation.FutureBlocks)) + l1BoundMaxTimestamp = arbmath.SaturatingUAdd(l1Bound.Time, arbmath.BigToUintSaturating(maxTimeVariation.FutureSeconds)) + + if config.L1BlockBoundBypass > 0 { + latestHeader, err := b.l1Reader.LastHeader(ctx) + if err != nil { + return false, err + } + blockNumberWithPadding := arbmath.SaturatingUAdd(arbmath.BigToUintSaturating(latestHeader.Number), uint64(config.L1BlockBoundBypass/ethPosBlockTime)) + timestampWithPadding := arbmath.SaturatingUAdd(latestHeader.Time, uint64(config.L1BlockBoundBypass/time.Second)) + + l1BoundMinBlockNumber = arbmath.SaturatingUSub(blockNumberWithPadding, arbmath.BigToUintSaturating(maxTimeVariation.DelayBlocks)) + l1BoundMinTimestamp = arbmath.SaturatingUSub(timestampWithPadding, arbmath.BigToUintSaturating(maxTimeVariation.DelaySeconds)) + } + } + for b.building.msgCount < msgCount { msg, err := b.streamer.GetMessage(b.building.msgCount) if err != nil { log.Error("error getting message from streamer", "error", err) break } + if msg.Message.Header.BlockNumber < l1BoundMinBlockNumber || msg.Message.Header.Timestamp < l1BoundMinTimestamp { + log.Error( + "disabling L1 bound as batch posting message is close to the maximum delay", + "blockNumber", msg.Message.Header.BlockNumber, + "l1BoundMinBlockNumber", l1BoundMinBlockNumber, + "timestamp", msg.Message.Header.Timestamp, + "l1BoundMinTimestamp", l1BoundMinTimestamp, + ) + l1BoundMaxBlockNumber = math.MaxUint64 + l1BoundMaxTimestamp = math.MaxUint64 + } + if msg.Message.Header.BlockNumber > l1BoundMaxBlockNumber || msg.Message.Header.Timestamp > l1BoundMaxTimestamp { + log.Info( + "not posting more messages because block number or timestamp exceed L1 bounds", + "blockNumber", msg.Message.Header.BlockNumber, + "l1BoundMaxBlockNumber", l1BoundMaxBlockNumber, + "timestamp", msg.Message.Header.Timestamp, + "l1BoundMaxTimestamp", l1BoundMaxTimestamp, + ) + break + } success, err := b.building.segments.AddMessage(msg) if err != nil { // Clear our cache diff --git a/arbnode/dataposter/data_poster.go b/arbnode/dataposter/data_poster.go index 1dec6ad0c4..b06b4b2cb3 100644 --- a/arbnode/dataposter/data_poster.go +++ b/arbnode/dataposter/data_poster.go @@ -33,75 +33,6 @@ import ( redisstorage "github.com/offchainlabs/nitro/arbnode/dataposter/redis" ) -type queuedTransaction[Meta any] struct { - FullTx *types.Transaction - Data types.DynamicFeeTx - Meta Meta - Sent bool - Created time.Time // may be earlier than the tx was given to the tx poster - NextReplacement time.Time -} - -// Note: one of the implementation of this interface (Redis storage) does not -// support duplicate values. -type QueueStorage[Item any] interface { - GetContents(ctx context.Context, startingIndex uint64, maxResults uint64) ([]*Item, error) - GetLast(ctx context.Context) (*Item, error) - Prune(ctx context.Context, keepStartingAt uint64) error - Put(ctx context.Context, index uint64, prevItem *Item, newItem *Item) error - Length(ctx context.Context) (int, error) - IsPersistent() bool -} - -type DataPosterConfig struct { - RedisSigner signature.SimpleHmacConfig `koanf:"redis-signer"` - ReplacementTimes string `koanf:"replacement-times"` - WaitForL1Finality bool `koanf:"wait-for-l1-finality" reload:"hot"` - MaxMempoolTransactions uint64 `koanf:"max-mempool-transactions" reload:"hot"` - MaxQueuedTransactions int `koanf:"max-queued-transactions" reload:"hot"` - TargetPriceGwei float64 `koanf:"target-price-gwei" reload:"hot"` - UrgencyGwei float64 `koanf:"urgency-gwei" reload:"hot"` - MinFeeCapGwei float64 `koanf:"min-fee-cap-gwei" reload:"hot"` - MinTipCapGwei float64 `koanf:"min-tip-cap-gwei" reload:"hot"` - EnableLevelDB bool `koanf:"enable-leveldb" reload:"hot"` -} - -type DataPosterConfigFetcher func() *DataPosterConfig - -func DataPosterConfigAddOptions(prefix string, f *pflag.FlagSet) { - f.String(prefix+".replacement-times", DefaultDataPosterConfig.ReplacementTimes, "comma-separated list of durations since first posting to attempt a replace-by-fee") - f.Bool(prefix+".wait-for-l1-finality", DefaultDataPosterConfig.WaitForL1Finality, "only treat a transaction as confirmed after L1 finality has been achieved (recommended)") - f.Uint64(prefix+".max-mempool-transactions", DefaultDataPosterConfig.MaxMempoolTransactions, "the maximum number of transactions to have queued in the mempool at once (0 = unlimited)") - f.Int(prefix+".max-queued-transactions", DefaultDataPosterConfig.MaxQueuedTransactions, "the maximum number of unconfirmed transactions to track at once (0 = unlimited)") - f.Float64(prefix+".target-price-gwei", DefaultDataPosterConfig.TargetPriceGwei, "the target price to use for maximum fee cap calculation") - f.Float64(prefix+".urgency-gwei", DefaultDataPosterConfig.UrgencyGwei, "the urgency to use for maximum fee cap calculation") - f.Float64(prefix+".min-fee-cap-gwei", DefaultDataPosterConfig.MinFeeCapGwei, "the minimum fee cap to post transactions at") - f.Float64(prefix+".min-tip-cap-gwei", DefaultDataPosterConfig.MinTipCapGwei, "the minimum tip cap to post transactions at") - f.Bool(prefix+".enable-leveldb", DefaultDataPosterConfig.EnableLevelDB, "uses leveldb when enabled") - signature.SimpleHmacConfigAddOptions(prefix+".redis-signer", f) -} - -var DefaultDataPosterConfig = DataPosterConfig{ - ReplacementTimes: "5m,10m,20m,30m,1h,2h,4h,6h,8h,12h,16h,18h,20h,22h", - WaitForL1Finality: true, - TargetPriceGwei: 60., - UrgencyGwei: 2., - MaxMempoolTransactions: 64, - MinTipCapGwei: 0.05, - EnableLevelDB: false, -} - -var TestDataPosterConfig = DataPosterConfig{ - ReplacementTimes: "1s,2s,5s,10s,20s,30s,1m,5m", - RedisSigner: signature.TestSimpleHmacConfig, - WaitForL1Finality: false, - TargetPriceGwei: 60., - UrgencyGwei: 2., - MaxMempoolTransactions: 64, - MinTipCapGwei: 0.05, - EnableLevelDB: false, -} - // DataPoster must be RLP serializable and deserializable type DataPoster[Meta any] struct { stopwaiter.StopWaiter @@ -109,11 +40,11 @@ type DataPoster[Meta any] struct { client arbutil.L1Interface auth *bind.TransactOpts redisLock AttemptLocker - config DataPosterConfigFetcher + config ConfigFetcher replacementTimes []time.Duration metadataRetriever func(ctx context.Context, blockNum *big.Int) (Meta, error) - // these fields are protected by the mutex + // These fields are protected by the mutex. mutex sync.Mutex lastBlock *big.Int balance *big.Int @@ -126,7 +57,7 @@ type AttemptLocker interface { AttemptLock(context.Context) bool } -func NewDataPoster[Meta any](db ethdb.Database, headerReader *headerreader.HeaderReader, auth *bind.TransactOpts, redisClient redis.UniversalClient, redisLock AttemptLocker, config DataPosterConfigFetcher, metadataRetriever func(ctx context.Context, blockNum *big.Int) (Meta, error)) (*DataPoster[Meta], error) { +func NewDataPoster[Meta any](db ethdb.Database, headerReader *headerreader.HeaderReader, auth *bind.TransactOpts, redisClient redis.UniversalClient, redisLock AttemptLocker, config ConfigFetcher, metadataRetriever func(ctx context.Context, blockNum *big.Int) (Meta, error)) (*DataPoster[Meta], error) { var replacementTimes []time.Duration var lastReplacementTime time.Duration for _, s := range strings.Split(config().ReplacementTimes, ",") { @@ -184,7 +115,7 @@ func (p *DataPoster[Meta]) GetNextNonceAndMeta(ctx context.Context) (uint64, Met if err != nil { return 0, emptyMeta, err } - lastQueueItem, err := p.queue.GetLast(ctx) + lastQueueItem, err := p.queue.FetchLast(ctx) if err != nil { return 0, emptyMeta, err } @@ -231,7 +162,7 @@ func (p *DataPoster[Meta]) GetNextNonceAndMeta(ctx context.Context) (uint64, Met const minRbfIncrease = arbmath.OneInBips * 11 / 10 -func (p *DataPoster[Meta]) getFeeAndTipCaps(ctx context.Context, gasLimit uint64, lastFeeCap *big.Int, lastTipCap *big.Int, dataCreatedAt time.Time, backlogOfBatches uint64) (*big.Int, *big.Int, error) { +func (p *DataPoster[Meta]) feeAndTipCaps(ctx context.Context, gasLimit uint64, lastFeeCap *big.Int, lastTipCap *big.Int, dataCreatedAt time.Time, backlogOfBatches uint64) (*big.Int, *big.Int, error) { config := p.config() latestHeader, err := p.headerReader.LastHeader(ctx) if err != nil { @@ -312,7 +243,7 @@ func (p *DataPoster[Meta]) PostTransaction(ctx context.Context, dataCreatedAt ti if err != nil { return fmt.Errorf("failed to update data poster balance: %w", err) } - feeCap, tipCap, err := p.getFeeAndTipCaps(ctx, gasLimit, nil, nil, dataCreatedAt, 0) + feeCap, tipCap, err := p.feeAndTipCaps(ctx, gasLimit, nil, nil, dataCreatedAt, 0) if err != nil { return err } @@ -374,7 +305,7 @@ func (p *DataPoster[Meta]) sendTx(ctx context.Context, prevTx *queuedTransaction // the mutex must be held by the caller func (p *DataPoster[Meta]) replaceTx(ctx context.Context, prevTx *queuedTransaction[Meta], backlogOfBatches uint64) error { - newFeeCap, newTipCap, err := p.getFeeAndTipCaps(ctx, prevTx.Data.Gas, prevTx.Data.GasFeeCap, prevTx.Data.GasTipCap, prevTx.Created, backlogOfBatches) + newFeeCap, newTipCap, err := p.feeAndTipCaps(ctx, prevTx.Data.Gas, prevTx.Data.GasFeeCap, prevTx.Data.GasTipCap, prevTx.Created, backlogOfBatches) if err != nil { return err } @@ -455,6 +386,7 @@ func (p *DataPoster[Meta]) updateNonce(ctx context.Context) error { return nil } +// Updates dataposter balance to balance at pending block. func (p *DataPoster[Meta]) updateBalance(ctx context.Context) error { // Use the pending (representated as -1) balance because we're looking at batches we'd post, // so we want to see how much gas we could afford with our pending state. @@ -520,7 +452,7 @@ func (p *DataPoster[Meta]) Start(ctxIn context.Context) { // We use unconfirmedNonce here to replace-by-fee transactions that aren't in a block, // excluding those that are in an unconfirmed block. If a reorg occurs, we'll continue // replacing them by fee. - queueContents, err := p.queue.GetContents(ctx, unconfirmedNonce, maxTxsToRbf) + queueContents, err := p.queue.FetchContents(ctx, unconfirmedNonce, maxTxsToRbf) if err != nil { log.Warn("failed to get tx queue contents", "err", err) return minWait @@ -554,3 +486,85 @@ func (p *DataPoster[Meta]) Start(ctxIn context.Context) { return wait }) } + +type queuedTransaction[Meta any] struct { + FullTx *types.Transaction + Data types.DynamicFeeTx + Meta Meta + Sent bool + Created time.Time // may be earlier than the tx was given to the tx poster + NextReplacement time.Time +} + +// Implements queue-alike storage that can +// - Insert item at specified index +// - Update item with the condition that existing value equals assumed value +// - Delete all the items up to specified index (prune) +// - Calculate length +// Note: one of the implementation of this interface (Redis storage) does not +// support duplicate values. +type QueueStorage[Item any] interface { + // Returns at most maxResults items starting from specified index. + FetchContents(ctx context.Context, startingIndex uint64, maxResults uint64) ([]*Item, error) + // Returns item with the biggest index. + FetchLast(ctx context.Context) (*Item, error) + // Prunes items up to (excluding) specified index. + Prune(ctx context.Context, until uint64) error + // Inserts new item at specified index if previous value matches specified value. + Put(ctx context.Context, index uint64, prevItem *Item, newItem *Item) error + // Returns the size of a queue. + Length(ctx context.Context) (int, error) + // Indicates whether queue stored at disk. + IsPersistent() bool +} + +type DataPosterConfig struct { + RedisSigner signature.SimpleHmacConfig `koanf:"redis-signer"` + ReplacementTimes string `koanf:"replacement-times"` + WaitForL1Finality bool `koanf:"wait-for-l1-finality" reload:"hot"` + MaxMempoolTransactions uint64 `koanf:"max-mempool-transactions" reload:"hot"` + MaxQueuedTransactions int `koanf:"max-queued-transactions" reload:"hot"` + TargetPriceGwei float64 `koanf:"target-price-gwei" reload:"hot"` + UrgencyGwei float64 `koanf:"urgency-gwei" reload:"hot"` + MinFeeCapGwei float64 `koanf:"min-fee-cap-gwei" reload:"hot"` + MinTipCapGwei float64 `koanf:"min-tip-cap-gwei" reload:"hot"` + EnableLevelDB bool `koanf:"enable-leveldb" reload:"hot"` +} + +// ConfigFetcher function type is used instead of directly passing config so +// that flags can be reloaded dynamically. +type ConfigFetcher func() *DataPosterConfig + +func DataPosterConfigAddOptions(prefix string, f *pflag.FlagSet) { + f.String(prefix+".replacement-times", DefaultDataPosterConfig.ReplacementTimes, "comma-separated list of durations since first posting to attempt a replace-by-fee") + f.Bool(prefix+".wait-for-l1-finality", DefaultDataPosterConfig.WaitForL1Finality, "only treat a transaction as confirmed after L1 finality has been achieved (recommended)") + f.Uint64(prefix+".max-mempool-transactions", DefaultDataPosterConfig.MaxMempoolTransactions, "the maximum number of transactions to have queued in the mempool at once (0 = unlimited)") + f.Int(prefix+".max-queued-transactions", DefaultDataPosterConfig.MaxQueuedTransactions, "the maximum number of unconfirmed transactions to track at once (0 = unlimited)") + f.Float64(prefix+".target-price-gwei", DefaultDataPosterConfig.TargetPriceGwei, "the target price to use for maximum fee cap calculation") + f.Float64(prefix+".urgency-gwei", DefaultDataPosterConfig.UrgencyGwei, "the urgency to use for maximum fee cap calculation") + f.Float64(prefix+".min-fee-cap-gwei", DefaultDataPosterConfig.MinFeeCapGwei, "the minimum fee cap to post transactions at") + f.Float64(prefix+".min-tip-cap-gwei", DefaultDataPosterConfig.MinTipCapGwei, "the minimum tip cap to post transactions at") + f.Bool(prefix+".enable-leveldb", DefaultDataPosterConfig.EnableLevelDB, "uses leveldb when enabled") + signature.SimpleHmacConfigAddOptions(prefix+".redis-signer", f) +} + +var DefaultDataPosterConfig = DataPosterConfig{ + ReplacementTimes: "5m,10m,20m,30m,1h,2h,4h,6h,8h,12h,16h,18h,20h,22h", + WaitForL1Finality: true, + TargetPriceGwei: 60., + UrgencyGwei: 2., + MaxMempoolTransactions: 64, + MinTipCapGwei: 0.05, + EnableLevelDB: false, +} + +var TestDataPosterConfig = DataPosterConfig{ + ReplacementTimes: "1s,2s,5s,10s,20s,30s,1m,5m", + RedisSigner: signature.TestSimpleHmacConfig, + WaitForL1Finality: false, + TargetPriceGwei: 60., + UrgencyGwei: 2., + MaxMempoolTransactions: 64, + MinTipCapGwei: 0.05, + EnableLevelDB: false, +} diff --git a/arbnode/dataposter/leveldb/leveldb.go b/arbnode/dataposter/leveldb/leveldb.go index c271b71267..aaf4935f15 100644 --- a/arbnode/dataposter/leveldb/leveldb.go +++ b/arbnode/dataposter/leveldb/leveldb.go @@ -45,7 +45,7 @@ func idxToKey(idx uint64) []byte { return []byte(fmt.Sprintf("%020d", idx)) } -func (s *Storage[Item]) GetContents(_ context.Context, startingIndex uint64, maxResults uint64) ([]*Item, error) { +func (s *Storage[Item]) FetchContents(_ context.Context, startingIndex uint64, maxResults uint64) ([]*Item, error) { var res []*Item it := s.db.NewIterator([]byte(""), idxToKey(startingIndex)) defer it.Release() @@ -66,7 +66,7 @@ func (s *Storage[Item]) lastItemIdx(context.Context) ([]byte, error) { return s.db.Get(lastItemIdxKey) } -func (s *Storage[Item]) GetLast(ctx context.Context) (*Item, error) { +func (s *Storage[Item]) FetchLast(ctx context.Context) (*Item, error) { size, err := s.Length(ctx) if err != nil { return nil, err @@ -85,12 +85,12 @@ func (s *Storage[Item]) GetLast(ctx context.Context) (*Item, error) { return s.decodeItem(val) } -func (s *Storage[Item]) Prune(ctx context.Context, keepStartingAt uint64) error { +func (s *Storage[Item]) Prune(ctx context.Context, until uint64) error { cnt, err := s.Length(ctx) if err != nil { return err } - end := idxToKey(keepStartingAt) + end := idxToKey(until) it := s.db.NewIterator([]byte{}, idxToKey(0)) defer it.Release() b := s.db.NewBatch() diff --git a/arbnode/dataposter/redis/redisstorage.go b/arbnode/dataposter/redis/redisstorage.go index f7aed00e59..738eea521d 100644 --- a/arbnode/dataposter/redis/redisstorage.go +++ b/arbnode/dataposter/redis/redisstorage.go @@ -52,7 +52,7 @@ func (s *Storage[Item]) peelVerifySignature(data []byte) ([]byte, error) { return data[32:], nil } -func (s *Storage[Item]) GetContents(ctx context.Context, startingIndex uint64, maxResults uint64) ([]*Item, error) { +func (s *Storage[Item]) FetchContents(ctx context.Context, startingIndex uint64, maxResults uint64) ([]*Item, error) { query := redis.ZRangeArgs{ Key: s.key, ByScore: true, @@ -79,7 +79,7 @@ func (s *Storage[Item]) GetContents(ctx context.Context, startingIndex uint64, m return items, nil } -func (s *Storage[Item]) GetLast(ctx context.Context) (*Item, error) { +func (s *Storage[Item]) FetchLast(ctx context.Context) (*Item, error) { query := redis.ZRangeArgs{ Key: s.key, Start: 0, @@ -109,9 +109,9 @@ func (s *Storage[Item]) GetLast(ctx context.Context) (*Item, error) { return ret, nil } -func (s *Storage[Item]) Prune(ctx context.Context, keepStartingAt uint64) error { - if keepStartingAt > 0 { - return s.client.ZRemRangeByScore(ctx, s.key, "-inf", fmt.Sprintf("%v", keepStartingAt-1)).Err() +func (s *Storage[Item]) Prune(ctx context.Context, until uint64) error { + if until > 0 { + return s.client.ZRemRangeByScore(ctx, s.key, "-inf", fmt.Sprintf("%v", until-1)).Err() } return nil } diff --git a/arbnode/dataposter/slice/slicestorage.go b/arbnode/dataposter/slice/slicestorage.go index b0a253086f..e438f9fe46 100644 --- a/arbnode/dataposter/slice/slicestorage.go +++ b/arbnode/dataposter/slice/slicestorage.go @@ -19,7 +19,7 @@ func NewStorage[Item any]() *Storage[Item] { return &Storage[Item]{} } -func (s *Storage[Item]) GetContents(_ context.Context, startingIndex uint64, maxResults uint64) ([]*Item, error) { +func (s *Storage[Item]) FetchContents(_ context.Context, startingIndex uint64, maxResults uint64) ([]*Item, error) { ret := s.queue if startingIndex >= s.firstNonce+uint64(len(s.queue)) || maxResults == 0 { return nil, nil @@ -33,19 +33,19 @@ func (s *Storage[Item]) GetContents(_ context.Context, startingIndex uint64, max return ret, nil } -func (s *Storage[Item]) GetLast(context.Context) (*Item, error) { +func (s *Storage[Item]) FetchLast(context.Context) (*Item, error) { if len(s.queue) == 0 { return nil, nil } return s.queue[len(s.queue)-1], nil } -func (s *Storage[Item]) Prune(_ context.Context, keepStartingAt uint64) error { - if keepStartingAt >= s.firstNonce+uint64(len(s.queue)) { +func (s *Storage[Item]) Prune(_ context.Context, until uint64) error { + if until >= s.firstNonce+uint64(len(s.queue)) { s.queue = nil - } else if keepStartingAt >= s.firstNonce { - s.queue = s.queue[keepStartingAt-s.firstNonce:] - s.firstNonce = keepStartingAt + } else if until >= s.firstNonce { + s.queue = s.queue[until-s.firstNonce:] + s.firstNonce = until } return nil } diff --git a/arbnode/dataposter/storage_test.go b/arbnode/dataposter/storage_test.go index 0ef83ed5ba..057080f603 100644 --- a/arbnode/dataposter/storage_test.go +++ b/arbnode/dataposter/storage_test.go @@ -84,7 +84,7 @@ func strPtrs(values []string) []*string { return res } -func TestGetContents(t *testing.T) { +func TestFetchContents(t *testing.T) { ctx := context.Background() for name, s := range initStorages(ctx, t) { for _, tc := range []struct { @@ -126,19 +126,19 @@ func TestGetContents(t *testing.T) { }, } { t.Run(name+"_"+tc.desc, func(t *testing.T) { - values, err := s.GetContents(ctx, tc.startIdx, tc.maxResults) + values, err := s.FetchContents(ctx, tc.startIdx, tc.maxResults) if err != nil { - t.Fatalf("GetContents(%d, %d) unexpected error: %v", tc.startIdx, tc.maxResults, err) + t.Fatalf("FetchContents(%d, %d) unexpected error: %v", tc.startIdx, tc.maxResults, err) } if diff := cmp.Diff(tc.want, values); diff != "" { - t.Errorf("GetContext(%d, %d) unexpected diff:\n%s", tc.startIdx, tc.maxResults, diff) + t.Errorf("FetchContents(%d, %d) unexpected diff:\n%s", tc.startIdx, tc.maxResults, diff) } }) } } } -func TestGetLast(t *testing.T) { +func TestLast(t *testing.T) { cnt := 100 for name, s := range storages(t) { t.Run(name, func(t *testing.T) { @@ -148,12 +148,12 @@ func TestGetLast(t *testing.T) { if err := s.Put(ctx, uint64(i), nil, &val); err != nil { t.Fatalf("Error putting a key/value: %v", err) } - got, err := s.GetLast(ctx) + got, err := s.FetchLast(ctx) if err != nil { t.Fatalf("Error getting a last element: %v", err) } if *got != val { - t.Errorf("GetLast() = %q want %q", *got, val) + t.Errorf("FetchLast() = %q want %q", *got, val) } } @@ -167,12 +167,12 @@ func TestGetLast(t *testing.T) { if err := s.Put(ctx, uint64(i), &prev, &newVal); err != nil { t.Fatalf("Error putting a key/value: %v, prev: %v, new: %v", err, prev, newVal) } - got, err := s.GetLast(ctx) + got, err := s.FetchLast(ctx) if err != nil { t.Fatalf("Error getting a last element: %v", err) } if *got != last { - t.Errorf("GetLast() = %q want %q", *got, last) + t.Errorf("FetchLast() = %q want %q", *got, last) } gotCnt, err := s.Length(ctx) if err != nil { @@ -225,9 +225,9 @@ func TestPrune(t *testing.T) { if err := s.Prune(ctx, tc.pruneFrom); err != nil { t.Fatalf("Prune(%d) unexpected error: %v", tc.pruneFrom, err) } - got, err := s.GetContents(ctx, 0, 20) + got, err := s.FetchContents(ctx, 0, 20) if err != nil { - t.Fatalf("GetContents() unexpected error: %v", err) + t.Fatalf("FetchContents() unexpected error: %v", err) } if diff := cmp.Diff(tc.want, got); diff != "" { t.Errorf("Prune(%d) unexpected diff:\n%s", tc.pruneFrom, diff) diff --git a/arbos/arbosState/arbosstate.go b/arbos/arbosState/arbosstate.go index 2bea8f7c54..4891197a10 100644 --- a/arbos/arbosState/arbosstate.go +++ b/arbos/arbosState/arbosstate.go @@ -303,7 +303,16 @@ func (state *ArbosState) UpgradeArbosVersion( ErrFatalNodeOutOfDate, ) } - // no state changes needed + // Update the PerBatchGasCost to a more accurate value compared to the old v6 default. + ensure(state.l1PricingState.SetPerBatchGasCost(l1pricing.InitialPerBatchGasCostV12)) + + // We had mistakenly initialized AmortizedCostCapBips to math.MaxUint64 in older versions, + // but the correct value to disable the amortization cap is 0. + oldAmortizationCap, err := state.l1PricingState.AmortizedCostCapBips() + ensure(err) + if oldAmortizationCap == math.MaxUint64 { + ensure(state.l1PricingState.SetAmortizedCostCapBips(0)) + } default: return fmt.Errorf( "the chain is upgrading to unsupported ArbOS version %v, %w", @@ -315,7 +324,9 @@ func (state *ArbosState) UpgradeArbosVersion( } if firstTime && upgradeTo >= 6 { - state.Restrict(state.l1PricingState.SetPerBatchGasCost(l1pricing.InitialPerBatchGasCostV6)) + if upgradeTo < 11 { + state.Restrict(state.l1PricingState.SetPerBatchGasCost(l1pricing.InitialPerBatchGasCostV6)) + } state.Restrict(state.l1PricingState.SetEquilibrationUnits(l1pricing.InitialEquilibrationUnitsV6)) state.Restrict(state.l2PricingState.SetSpeedLimitPerSecond(l2pricing.InitialSpeedLimitPerSecondV6)) state.Restrict(state.l2PricingState.SetMaxPerBlockGasLimit(l2pricing.InitialPerBlockGasLimitV6)) diff --git a/arbos/l1pricing/l1pricing.go b/arbos/l1pricing/l1pricing.go index 64698957e8..34d6021f00 100644 --- a/arbos/l1pricing/l1pricing.go +++ b/arbos/l1pricing/l1pricing.go @@ -72,9 +72,10 @@ const ( ) const ( - InitialInertia = 10 - InitialPerUnitReward = 10 - InitialPerBatchGasCostV6 = 100000 + InitialInertia = 10 + InitialPerUnitReward = 10 + InitialPerBatchGasCostV6 = 100_000 + InitialPerBatchGasCostV12 = 210_000 // overriden as part of the upgrade ) // one minute at 100000 bytes / sec diff --git a/arbos/tx_processor.go b/arbos/tx_processor.go index d0f999d0de..2bb02276ee 100644 --- a/arbos/tx_processor.go +++ b/arbos/tx_processor.go @@ -385,7 +385,10 @@ func (p *TxProcessor) GasChargingHook(gasRemaining *uint64) (common.Address, err poster = p.evm.Context.Coinbase } - if basefee.Sign() > 0 { + if p.msg.TxRunMode == core.MessageCommitMode { + p.msg.SkipL1Charging = false + } + if basefee.Sign() > 0 && !p.msg.SkipL1Charging { // Since tips go to the network, and not to the poster, we use the basefee. // Note, this only determines the amount of gas bought, not the price per gas. diff --git a/cmd/datool/datool.go b/cmd/datool/datool.go index 522e021ee1..6f975ec712 100644 --- a/cmd/datool/datool.go +++ b/cmd/datool/datool.go @@ -16,6 +16,7 @@ import ( "strings" "time" + koanfjson "github.com/knadh/koanf/parsers/json" flag "github.com/spf13/pflag" "github.com/ethereum/go-ethereum/common" @@ -34,7 +35,7 @@ import ( func main() { args := os.Args if len(args) < 2 { - panic("Usage: datool [client|keygen|generatehash] ...") + panic("Usage: datool [client|keygen|generatehash|dumpkeyset] ...") } var err error @@ -45,6 +46,8 @@ func main() { err = startKeyGen(args[2:]) case "generatehash": err = generateHash(args[2]) + case "dumpkeyset": + err = dumpKeyset(args[2:]) default: panic(fmt.Sprintf("Unknown tool '%s' specified, valid tools are 'client', 'keygen', 'generatehash'", args[1])) } @@ -313,3 +316,68 @@ func generateHash(message string) error { fmt.Printf("Hex Encoded Data Hash: %s\n", hexutil.Encode(dastree.HashBytes([]byte(message)))) return nil } + +func parseDumpKeyset(args []string) (*DumpKeysetConfig, error) { + f := flag.NewFlagSet("dump keyset", flag.ContinueOnError) + + das.AggregatorConfigAddOptions("keyset", f) + genericconf.ConfConfigAddOptions("conf", f) + + k, err := confighelpers.BeginCommonParse(f, args) + if err != nil { + return nil, err + } + + var config DumpKeysetConfig + if err := confighelpers.EndCommonParse(k, &config); err != nil { + return nil, err + } + + if config.ConfConfig.Dump { + c, err := k.Marshal(koanfjson.Parser()) + if err != nil { + return nil, fmt.Errorf("unable to marshal config file to JSON: %w", err) + } + + fmt.Println(string(c)) + os.Exit(0) + } + + if config.KeysetConfig.AssumedHonest == 0 { + return nil, errors.New("--keyset.assumed-honest must be set") + } + if config.KeysetConfig.Backends == "" { + return nil, errors.New("--keyset.backends must be set") + } + + return &config, nil +} + +// das keygen + +type DumpKeysetConfig struct { + KeysetConfig das.AggregatorConfig `koanf:"keyset"` + ConfConfig genericconf.ConfConfig `koanf:"conf"` +} + +func dumpKeyset(args []string) error { + config, err := parseDumpKeyset(args) + if err != nil { + return err + } + + services, err := das.ParseServices(config.KeysetConfig) + if err != nil { + return err + } + + keysetHash, keysetBytes, err := das.KeysetHashFromServices(services, uint64(config.KeysetConfig.AssumedHonest)) + if err != nil { + return err + } + + fmt.Printf("Keyset: %s\n", hexutil.Encode(keysetBytes)) + fmt.Printf("KeysetHash: %s\n", hexutil.Encode(keysetHash[:])) + + return err +} diff --git a/das/aggregator.go b/das/aggregator.go index 7c1504d6f1..33ce5ad489 100644 --- a/das/aggregator.go +++ b/das/aggregator.go @@ -9,13 +9,11 @@ import ( "errors" "fmt" "math/bits" - "os" "time" flag "github.com/spf13/pflag" "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/metrics" @@ -32,13 +30,11 @@ type AggregatorConfig struct { Enable bool `koanf:"enable"` AssumedHonest int `koanf:"assumed-honest"` Backends string `koanf:"backends"` - DumpKeyset bool `koanf:"dump-keyset"` } var DefaultAggregatorConfig = AggregatorConfig{ AssumedHonest: 0, Backends: "", - DumpKeyset: false, } var BatchToDasFailed = errors.New("unable to batch to DAS") @@ -47,7 +43,6 @@ func AggregatorConfigAddOptions(prefix string, f *flag.FlagSet) { f.Bool(prefix+".enable", DefaultAggregatorConfig.Enable, "enable storage/retrieval of sequencer batch data from a list of RPC endpoints; this should only be used by the batch poster and not in combination with other DAS storage types") f.Int(prefix+".assumed-honest", DefaultAggregatorConfig.AssumedHonest, "Number of assumed honest backends (H). If there are N backends, K=N+1-H valid responses are required to consider an Store request to be successful.") f.String(prefix+".backends", DefaultAggregatorConfig.Backends, "JSON RPC backend configuration") - f.Bool(prefix+".dump-keyset", DefaultAggregatorConfig.DumpKeyset, "Dump the keyset encoded in hexadecimal for the backends string") } type Aggregator struct { @@ -122,36 +117,11 @@ func NewAggregatorWithSeqInboxCaller( services []ServiceDetails, seqInboxCaller *bridgegen.SequencerInboxCaller, ) (*Aggregator, error) { - var aggSignersMask uint64 - pubKeys := []blsSignatures.PublicKey{} - for _, d := range services { - if bits.OnesCount64(d.signersMask) != 1 { - return nil, fmt.Errorf("tried to configure backend DAS %v with invalid signersMask %X", d.service, d.signersMask) - } - aggSignersMask |= d.signersMask - pubKeys = append(pubKeys, d.pubKey) - } - if bits.OnesCount64(aggSignersMask) != len(services) { - return nil, errors.New("at least two signers share a mask") - } - keyset := &arbstate.DataAvailabilityKeyset{ - AssumedHonest: uint64(config.AggregatorConfig.AssumedHonest), - PubKeys: pubKeys, - } - ksBuf := bytes.NewBuffer([]byte{}) - if err := keyset.Serialize(ksBuf); err != nil { - return nil, err - } - keysetHash, err := keyset.Hash() + keysetHash, keysetBytes, err := KeysetHashFromServices(services, uint64(config.AggregatorConfig.AssumedHonest)) if err != nil { return nil, err } - if config.AggregatorConfig.DumpKeyset { - fmt.Printf("Keyset: %s\n", hexutil.Encode(ksBuf.Bytes())) - fmt.Printf("KeysetHash: %s\n", hexutil.Encode(keysetHash[:])) - os.Exit(0) - } var bpVerifier *contracts.BatchPosterVerifier if seqInboxCaller != nil { @@ -165,7 +135,7 @@ func NewAggregatorWithSeqInboxCaller( requiredServicesForStore: len(services) + 1 - config.AggregatorConfig.AssumedHonest, maxAllowedServiceStoreFailures: config.AggregatorConfig.AssumedHonest - 1, keysetHash: keysetHash, - keysetBytes: ksBuf.Bytes(), + keysetBytes: keysetBytes, bpVerifier: bpVerifier, }, nil } diff --git a/das/rpc_aggregator.go b/das/rpc_aggregator.go index ff5f4aedb8..cc455250d3 100644 --- a/das/rpc_aggregator.go +++ b/das/rpc_aggregator.go @@ -4,10 +4,16 @@ package das import ( + "bytes" "context" "encoding/json" + "errors" + "fmt" + "math/bits" "net/url" + "github.com/offchainlabs/nitro/arbstate" + "github.com/offchainlabs/nitro/blsSignatures" "github.com/offchainlabs/nitro/solgen/go/bridgegen" "github.com/offchainlabs/nitro/util/metricsutil" @@ -22,7 +28,7 @@ type BackendConfig struct { } func NewRPCAggregator(ctx context.Context, config DataAvailabilityConfig) (*Aggregator, error) { - services, err := setUpServices(config) + services, err := ParseServices(config.AggregatorConfig) if err != nil { return nil, err } @@ -30,7 +36,7 @@ func NewRPCAggregator(ctx context.Context, config DataAvailabilityConfig) (*Aggr } func NewRPCAggregatorWithL1Info(config DataAvailabilityConfig, l1client arbutil.L1Interface, seqInboxAddress common.Address) (*Aggregator, error) { - services, err := setUpServices(config) + services, err := ParseServices(config.AggregatorConfig) if err != nil { return nil, err } @@ -38,16 +44,16 @@ func NewRPCAggregatorWithL1Info(config DataAvailabilityConfig, l1client arbutil. } func NewRPCAggregatorWithSeqInboxCaller(config DataAvailabilityConfig, seqInboxCaller *bridgegen.SequencerInboxCaller) (*Aggregator, error) { - services, err := setUpServices(config) + services, err := ParseServices(config.AggregatorConfig) if err != nil { return nil, err } return NewAggregatorWithSeqInboxCaller(config, services, seqInboxCaller) } -func setUpServices(config DataAvailabilityConfig) ([]ServiceDetails, error) { +func ParseServices(config AggregatorConfig) ([]ServiceDetails, error) { var cs []BackendConfig - err := json.Unmarshal([]byte(config.AggregatorConfig.Backends), &cs) + err := json.Unmarshal([]byte(config.Backends), &cs) if err != nil { return nil, err } @@ -81,3 +87,33 @@ func setUpServices(config DataAvailabilityConfig) ([]ServiceDetails, error) { return services, nil } + +func KeysetHashFromServices(services []ServiceDetails, assumedHonest uint64) ([32]byte, []byte, error) { + var aggSignersMask uint64 + pubKeys := []blsSignatures.PublicKey{} + for _, d := range services { + if bits.OnesCount64(d.signersMask) != 1 { + return [32]byte{}, nil, fmt.Errorf("tried to configure backend DAS %v with invalid signersMask %X", d.service, d.signersMask) + } + aggSignersMask |= d.signersMask + pubKeys = append(pubKeys, d.pubKey) + } + if bits.OnesCount64(aggSignersMask) != len(services) { + return [32]byte{}, nil, errors.New("at least two signers share a mask") + } + + keyset := &arbstate.DataAvailabilityKeyset{ + AssumedHonest: uint64(assumedHonest), + PubKeys: pubKeys, + } + ksBuf := bytes.NewBuffer([]byte{}) + if err := keyset.Serialize(ksBuf); err != nil { + return [32]byte{}, nil, err + } + keysetHash, err := keyset.Hash() + if err != nil { + return [32]byte{}, nil, err + } + + return keysetHash, ksBuf.Bytes(), nil +} diff --git a/go-ethereum b/go-ethereum index 3725b60e04..491fee52fa 160000 --- a/go-ethereum +++ b/go-ethereum @@ -1 +1 @@ -Subproject commit 3725b60e0494df145672ab67dd3ec18a85a2b5d1 +Subproject commit 491fee52faf1e7dfcf5431f91a68e95ba779ea66 diff --git a/go.mod b/go.mod index 37ab04ff30..509dec9a5b 100644 --- a/go.mod +++ b/go.mod @@ -113,7 +113,6 @@ require ( github.com/hannahhoward/go-pubsub v0.0.0-20200423002714-8d62886cc36e // indirect github.com/hashicorp/errwrap v1.1.0 // indirect github.com/hashicorp/go-multierror v1.1.1 // indirect - github.com/holiman/big v0.0.0-20221017200358-a027dc42d04e // indirect github.com/ipfs/bbloom v0.0.4 // indirect github.com/ipfs/go-bitfield v1.1.0 // indirect github.com/ipfs/go-block-format v0.1.1 // indirect diff --git a/go.sum b/go.sum index ca552ef60a..304f7cc4aa 100644 --- a/go.sum +++ b/go.sum @@ -600,8 +600,6 @@ github.com/hashicorp/vault/api v1.0.4/go.mod h1:gDcqh3WGcR1cpF5AJz/B1UFheUEneMoI github.com/hashicorp/vault/sdk v0.1.13/go.mod h1:B+hVj7TpuQY1Y/GPbCpffmgd+tSEwvhkWnjtSYCaS2M= github.com/hashicorp/yamux v0.0.0-20180604194846-3520598351bb/go.mod h1:+NfK9FKeTrX5uv1uIXGdwYDTeHna2qgaIlx54MXqjAM= github.com/hashicorp/yamux v0.0.0-20181012175058-2f1d1f20f75d/go.mod h1:+NfK9FKeTrX5uv1uIXGdwYDTeHna2qgaIlx54MXqjAM= -github.com/holiman/big v0.0.0-20221017200358-a027dc42d04e h1:pIYdhNkDh+YENVNi3gto8n9hAmRxKxoar0iE6BLucjw= -github.com/holiman/big v0.0.0-20221017200358-a027dc42d04e/go.mod h1:j9cQbcqHQujT0oKJ38PylVfqohClLr3CvDC+Qcg+lhU= github.com/holiman/bloomfilter/v2 v2.0.3 h1:73e0e/V0tCydx14a0SCYS/EWCxgwLZ18CZcZKVu0fao= github.com/holiman/bloomfilter/v2 v2.0.3/go.mod h1:zpoh+gs7qcpqrHr3dB55AMiJwo0iURXE7ZOP9L9hSkA= github.com/holiman/uint256 v1.2.0 h1:gpSYcPLWGv4sG43I2mVLiDZCNDh/EpGjSk8tmtxitHM= diff --git a/precompiles/ArbOwner_test.go b/precompiles/ArbOwner_test.go index b5527e0017..ab128a8cb2 100644 --- a/precompiles/ArbOwner_test.go +++ b/precompiles/ArbOwner_test.go @@ -9,20 +9,17 @@ import ( "math/big" "testing" - "github.com/offchainlabs/nitro/arbos/l1pricing" - - "github.com/ethereum/go-ethereum/common/math" + "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/params" "github.com/offchainlabs/nitro/arbos/arbosState" "github.com/offchainlabs/nitro/arbos/burn" - "github.com/offchainlabs/nitro/util/testhelpers" - - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/core/types" - "github.com/ethereum/go-ethereum/crypto" + "github.com/offchainlabs/nitro/arbos/l1pricing" "github.com/offchainlabs/nitro/arbos/util" + "github.com/offchainlabs/nitro/util/testhelpers" ) func TestArbOwner(t *testing.T) { @@ -99,7 +96,7 @@ func TestArbOwner(t *testing.T) { costCap, err := gasInfo.GetAmortizedCostCapBips(callCtx, evm) Require(t, err) - if costCap != math.MaxUint64 { + if costCap != 0 { Fail(t, costCap) } newCostCap := uint64(77734) diff --git a/system_tests/estimation_test.go b/system_tests/estimation_test.go index 2a416ad179..26b5a78145 100644 --- a/system_tests/estimation_test.go +++ b/system_tests/estimation_test.go @@ -218,3 +218,36 @@ func TestComponentEstimate(t *testing.T) { 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/staker_test.go b/system_tests/staker_test.go index 8a8ef29bf3..2aae1fc794 100644 --- a/system_tests/staker_test.go +++ b/system_tests/staker_test.go @@ -35,7 +35,7 @@ import ( "github.com/offchainlabs/nitro/validator/valnode" ) -func makeBackgroundTxs(ctx context.Context, l2info *BlockchainTestInfo, l2clientA arbutil.L1Interface, l2clientB arbutil.L1Interface, faultyStaker bool) error { +func makeBackgroundTxs(ctx context.Context, l2info *BlockchainTestInfo, l2clientA arbutil.L1Interface) error { for i := uint64(0); ctx.Err() == nil; i++ { l2info.Accounts["BackgroundUser"].Nonce = i tx := l2info.PrepareTx("BackgroundUser", "BackgroundUser", l2info.TransferGas, common.Big0, nil) @@ -47,19 +47,6 @@ func makeBackgroundTxs(ctx context.Context, l2info *BlockchainTestInfo, l2client if err != nil { return err } - if faultyStaker { - // Create a different transaction for the second node - l2info.Accounts["BackgroundUser"].Nonce = i - tx = l2info.PrepareTx("BackgroundUser", "BackgroundUser", l2info.TransferGas, common.Big1, nil) - err = l2clientB.SendTransaction(ctx, tx) - if err != nil { - return err - } - _, err = EnsureTxSucceeded(ctx, l2clientB, tx) - if err != nil { - return err - } - } } return nil } @@ -82,7 +69,11 @@ func stakerTestImpl(t *testing.T, faultyStaker bool, honestStakerInactive bool) if faultyStaker { l2info.GenerateGenesisAccount("FaultyAddr", common.Big1) } - l2clientB, l2nodeB := Create2ndNodeWithConfig(t, ctx, l2nodeA, l1stack, l1info, &l2info.ArbInitData, arbnode.ConfigDefaultL1Test(), nil) + config := arbnode.ConfigDefaultL1Test() + config.Sequencer.Enable = false + config.DelayedSequencer.Enable = false + config.BatchPoster.Enable = false + _, l2nodeB := Create2ndNodeWithConfig(t, ctx, l2nodeA, l1stack, l1info, &l2info.ArbInitData, config, nil) defer l2nodeB.StopAndWait() nodeAGenesis := l2nodeA.Execution.Backend.APIBackend().CurrentHeader().Hash() @@ -132,6 +123,9 @@ func stakerTestImpl(t *testing.T, faultyStaker bool, honestStakerInactive bool) _, err = EnsureTxSucceeded(ctx, l1client, tx) Require(t, err) + validatorUtils, err := rollupgen.NewValidatorUtils(l2nodeA.DeployInfo.ValidatorUtils, l1client) + Require(t, err) + valConfig := staker.L1ValidatorConfig{} valWalletA, err := staker.NewContractValidatorWallet(nil, l2nodeA.DeployInfo.ValidatorWalletCreator, l2nodeA.DeployInfo.Rollup, l2nodeA.L1Reader, &l1authA, 0, func(common.Address) {}) @@ -243,12 +237,6 @@ func stakerTestImpl(t *testing.T, faultyStaker bool, honestStakerInactive bool) Require(t, err) _, err = EnsureTxSucceeded(ctx, l2clientA, tx) Require(t, err) - if faultyStaker { - err = l2clientB.SendTransaction(ctx, tx) - Require(t, err) - _, err = EnsureTxSucceeded(ctx, l2clientB, tx) - Require(t, err) - } // Continually make L2 transactions in a background thread backgroundTxsCtx, cancelBackgroundTxs := context.WithCancel(ctx) @@ -259,7 +247,7 @@ func stakerTestImpl(t *testing.T, faultyStaker bool, honestStakerInactive bool) })() go (func() { defer close(backgroundTxsShutdownChan) - err := makeBackgroundTxs(backgroundTxsCtx, l2info, l2clientA, l2clientB, faultyStaker) + err := makeBackgroundTxs(backgroundTxsCtx, l2info, l2clientA) if !errors.Is(err, context.Canceled) { log.Warn("error making background txs", "err", err) } @@ -302,8 +290,11 @@ func stakerTestImpl(t *testing.T, faultyStaker bool, honestStakerInactive bool) if !challengeMangerTimedOut { // Upgrade the ChallengeManager contract to an implementation which says challenges are always timed out - mockImpl, _, _, err := mocksgen.DeployTimedOutChallengeManager(&deployAuth, l1client) + mockImpl, tx, _, err := mocksgen.DeployTimedOutChallengeManager(&deployAuth, l1client) Require(t, err) + _, err = EnsureTxSucceeded(ctx, l1client, tx) + Require(t, err) + managerAddr := valWalletA.ChallengeManagerAddress() // 0xb53127684a568b3173ae13b9f8a6016e243e63b6e8ee1178d6a717850b5d6103 proxyAdminSlot := common.BigToHash(arbmath.BigSub(crypto.Keccak256Hash([]byte("eip1967.proxy.admin")).Big(), common.Big1)) @@ -316,7 +307,7 @@ func stakerTestImpl(t *testing.T, faultyStaker bool, honestStakerInactive bool) proxyAdmin, err := mocksgen.NewProxyAdminForBinding(proxyAdminAddr, l1client) Require(t, err) - tx, err := proxyAdmin.Upgrade(&deployAuth, managerAddr, mockImpl) + tx, err = proxyAdmin.Upgrade(&deployAuth, managerAddr, mockImpl) Require(t, err) _, err = EnsureTxSucceeded(ctx, l1client, tx) Require(t, err) @@ -342,9 +333,9 @@ func stakerTestImpl(t *testing.T, faultyStaker bool, honestStakerInactive bool) Require(t, err, "EnsureTxSucceeded failed for staker", stakerName, "tx") } if faultyStaker { - challengeAddr, err := rollup.CurrentChallenge(&bind.CallOpts{}, valWalletAddrA) + conflictInfo, err := validatorUtils.FindStakerConflict(&bind.CallOpts{}, l2nodeA.DeployInfo.Rollup, l1authA.From, l1authB.From, big.NewInt(1024)) Require(t, err) - if challengeAddr != 0 { + if staker.ConflictType(conflictInfo.Ty) == staker.CONFLICT_TYPE_FOUND { cancelBackgroundTxs() } } @@ -357,6 +348,7 @@ func stakerTestImpl(t *testing.T, faultyStaker bool, honestStakerInactive bool) if isHonestZombie { Fatal(t, "staker A became a zombie") } + fmt.Printf("watchtower staker acting:\n") watchTx, err := stakerC.Act(ctx) if err != nil && !strings.Contains(err.Error(), "catch up") { Require(t, err, "watchtower staker failed to act") diff --git a/util/headerreader/header_reader.go b/util/headerreader/header_reader.go index 1d52e8f78d..28fef8ee07 100644 --- a/util/headerreader/header_reader.go +++ b/util/headerreader/header_reader.go @@ -40,15 +40,15 @@ type HeaderReader struct { lastPendingCallBlockNr uint64 requiresPendingCallUpdates int - safe cachedBlockNumber - finalized cachedBlockNumber + safe cachedHeader + finalized cachedHeader } -type cachedBlockNumber struct { +type cachedHeader struct { mutex sync.Mutex rpcBlockNum *big.Int headWhenCached *types.Header - blockNumber uint64 + header *types.Header } type Config struct { @@ -112,8 +112,8 @@ func New(ctx context.Context, client arbutil.L1Interface, config ConfigFetcher) arbSys: arbSys, outChannels: make(map[chan<- *types.Header]struct{}), outChannelsBehind: make(map[chan<- *types.Header]struct{}), - safe: cachedBlockNumber{rpcBlockNum: big.NewInt(rpc.SafeBlockNumber.Int64())}, - finalized: cachedBlockNumber{rpcBlockNum: big.NewInt(rpc.FinalizedBlockNumber.Int64())}, + safe: cachedHeader{rpcBlockNum: big.NewInt(rpc.SafeBlockNumber.Int64())}, + finalized: cachedHeader{rpcBlockNum: big.NewInt(rpc.FinalizedBlockNumber.Int64())}, }, nil } @@ -393,41 +393,58 @@ func headerIndicatesFinalitySupport(header *types.Header) bool { return false } -func (s *HeaderReader) getCached(ctx context.Context, c *cachedBlockNumber) (uint64, error) { +func (s *HeaderReader) getCached(ctx context.Context, c *cachedHeader) (*types.Header, error) { c.mutex.Lock() defer c.mutex.Unlock() currentHead, err := s.LastHeader(ctx) if err != nil { - return 0, err + return nil, err } if currentHead == c.headWhenCached { - return c.blockNumber, nil + return c.header, nil } if !s.config().UseFinalityData || !headerIndicatesFinalitySupport(currentHead) { - return 0, ErrBlockNumberNotSupported + return nil, ErrBlockNumberNotSupported } header, err := s.client.HeaderByNumber(ctx, c.rpcBlockNum) if err != nil { - return 0, err + return nil, err + } + c.header = header + c.headWhenCached = currentHead + return c.header, nil +} + +func (s *HeaderReader) LatestSafeBlockHeader(ctx context.Context) (*types.Header, error) { + header, err := s.getCached(ctx, &s.safe) + if errors.Is(err, ErrBlockNumberNotSupported) { + return nil, fmt.Errorf("%w: safe block not found", ErrBlockNumberNotSupported) } - c.blockNumber = header.Number.Uint64() - return c.blockNumber, nil + return header, err } func (s *HeaderReader) LatestSafeBlockNr(ctx context.Context) (uint64, error) { - blockNum, err := s.getCached(ctx, &s.safe) + header, err := s.LatestSafeBlockHeader(ctx) + if err != nil { + return 0, err + } + return header.Number.Uint64(), nil +} + +func (s *HeaderReader) LatestFinalizedBlockHeader(ctx context.Context) (*types.Header, error) { + header, err := s.getCached(ctx, &s.finalized) if errors.Is(err, ErrBlockNumberNotSupported) { - err = errors.New("safe block not found") + return nil, fmt.Errorf("%w: finalized block not found", ErrBlockNumberNotSupported) } - return blockNum, err + return header, err } func (s *HeaderReader) LatestFinalizedBlockNr(ctx context.Context) (uint64, error) { - blockNum, err := s.getCached(ctx, &s.finalized) - if errors.Is(err, ErrBlockNumberNotSupported) { - err = errors.New("finalized block not found") + header, err := s.LatestFinalizedBlockHeader(ctx) + if err != nil { + return 0, err } - return blockNum, err + return header.Number.Uint64(), nil } func (s *HeaderReader) Client() arbutil.L1Interface {