diff --git a/block/internal/common/metrics.go b/block/internal/common/metrics.go index 2a4c821774..13a0419432 100644 --- a/block/internal/common/metrics.go +++ b/block/internal/common/metrics.go @@ -65,6 +65,10 @@ type Metrics struct { DAInclusionHeight metrics.Gauge PendingHeadersCount metrics.Gauge PendingDataCount metrics.Gauge + + // Forced inclusion metrics + ForcedInclusionTxsInGracePeriod metrics.Gauge // Number of forced inclusion txs currently in grace period + ForcedInclusionTxsMalicious metrics.Counter // Total number of forced inclusion txs marked as malicious } // PrometheusMetrics returns Metrics built using Prometheus client library @@ -182,6 +186,21 @@ func PrometheusMetrics(namespace string, labelsAndValues ...string) *Metrics { Help: "Number of data blocks pending DA submission", }, labels).With(labelsAndValues...) + // Forced inclusion metrics + m.ForcedInclusionTxsInGracePeriod = prometheus.NewGaugeFrom(stdprometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: MetricsSubsystem, + Name: "forced_inclusion_txs_in_grace_period", + Help: "Number of forced inclusion transactions currently in grace period (past epoch end but within grace boundary)", + }, labels).With(labelsAndValues...) + + m.ForcedInclusionTxsMalicious = prometheus.NewCounterFrom(stdprometheus.CounterOpts{ + Namespace: namespace, + Subsystem: MetricsSubsystem, + Name: "forced_inclusion_txs_malicious_total", + Help: "Total number of forced inclusion transactions marked as malicious (past grace boundary)", + }, labels).With(labelsAndValues...) + // DA Submitter metrics m.DASubmitterPendingBlobs = prometheus.NewGaugeFrom(stdprometheus.GaugeOpts{ Namespace: namespace, @@ -246,6 +265,10 @@ func NopMetrics() *Metrics { DASubmitterLastFailure: make(map[DASubmitterFailureReason]metrics.Gauge), DASubmitterPendingBlobs: discard.NewGauge(), DASubmitterResends: discard.NewCounter(), + + // Forced inclusion metrics + ForcedInclusionTxsInGracePeriod: discard.NewGauge(), + ForcedInclusionTxsMalicious: discard.NewCounter(), } // Initialize maps with no-op metrics diff --git a/block/internal/syncing/syncer.go b/block/internal/syncing/syncer.go index 2c45fd1637..08e177cd6f 100644 --- a/block/internal/syncing/syncer.go +++ b/block/internal/syncing/syncer.go @@ -8,6 +8,7 @@ import ( "encoding/hex" "errors" "fmt" + "math" "sync" "sync/atomic" "time" @@ -18,6 +19,7 @@ import ( coreda "github.com/evstack/ev-node/core/da" coreexecutor "github.com/evstack/ev-node/core/execution" + seqcommon "github.com/evstack/ev-node/sequencers/common" "github.com/evstack/ev-node/block/internal/cache" "github.com/evstack/ev-node/block/internal/common" @@ -28,6 +30,47 @@ import ( "github.com/evstack/ev-node/types" ) +// forcedInclusionGracePeriodConfig contains internal configuration for forced inclusion grace periods. +type forcedInclusionGracePeriodConfig struct { + // basePeriod is the base number of additional epochs allowed for including forced inclusion transactions + // before marking the sequencer as malicious. This provides tolerance for temporary chain congestion. + // A value of 0 means strict enforcement (no grace period). + // A value of 1 means transactions from epoch N can be included in epoch N+1 without being marked malicious. + // Recommended: 1 epoch. + basePeriod uint64 + + // dynamicMinMultiplier is the minimum multiplier for the base grace period. + // The actual grace period will be at least: basePeriod * dynamicMinMultiplier. + // Example: base=2, min=0.5 → minimum grace period is 1 epoch. + dynamicMinMultiplier float64 + + // dynamicMaxMultiplier is the maximum multiplier for the base grace period. + // The actual grace period will be at most: basePeriod * dynamicMaxMultiplier. + // Example: base=2, max=3.0 → maximum grace period is 6 epochs. + dynamicMaxMultiplier float64 + + // dynamicFullnessThreshold defines what percentage of block capacity is considered "full". + // When EMA of block fullness is above this threshold, grace period increases. + // When below, grace period decreases. Value should be between 0.0 and 1.0. + dynamicFullnessThreshold float64 + + // dynamicAdjustmentRate controls how quickly the grace period multiplier adapts. + // Higher values make it adapt faster to congestion changes. Value should be between 0.0 and 1.0. + // Recommended: 0.05 for gradual adjustment, 0.1 for faster response. + dynamicAdjustmentRate float64 +} + +// newForcedInclusionGracePeriodConfig returns the internal grace period configuration. +func newForcedInclusionGracePeriodConfig() forcedInclusionGracePeriodConfig { + return forcedInclusionGracePeriodConfig{ + basePeriod: 1, // 1 epoch grace period + dynamicMinMultiplier: 0.5, // Minimum 0.5x base grace period + dynamicMaxMultiplier: 3.0, // Maximum 3x base grace period + dynamicFullnessThreshold: 0.8, // 80% capacity considered full + dynamicAdjustmentRate: 0.05, // 5% adjustment per block + } +} + // Syncer handles block synchronization from DA and P2P sources. type Syncer struct { // Core components @@ -66,6 +109,9 @@ type Syncer struct { // Forced inclusion tracking pendingForcedInclusionTxs sync.Map // map[string]pendingForcedInclusionTx + gracePeriodMultiplier *atomic.Pointer[float64] + blockFullnessEMA *atomic.Pointer[float64] + gracePeriodConfig forcedInclusionGracePeriodConfig // Lifecycle ctx context.Context @@ -102,22 +148,34 @@ func NewSyncer( daRetrieverHeight := &atomic.Uint64{} daRetrieverHeight.Store(genesis.DAStartHeight) + // Initialize dynamic grace period state + initialMultiplier := 1.0 + gracePeriodMultiplier := &atomic.Pointer[float64]{} + gracePeriodMultiplier.Store(&initialMultiplier) + + initialFullness := 0.0 + blockFullnessEMA := &atomic.Pointer[float64]{} + blockFullnessEMA.Store(&initialFullness) + return &Syncer{ - store: store, - exec: exec, - cache: cache, - metrics: metrics, - config: config, - genesis: genesis, - options: options, - lastState: &atomic.Pointer[types.State]{}, - daClient: daClient, - daRetrieverHeight: daRetrieverHeight, - headerStore: headerStore, - dataStore: dataStore, - heightInCh: make(chan common.DAHeightEvent, 100), - errorCh: errorCh, - logger: logger.With().Str("component", "syncer").Logger(), + store: store, + exec: exec, + cache: cache, + metrics: metrics, + config: config, + genesis: genesis, + options: options, + lastState: &atomic.Pointer[types.State]{}, + daClient: daClient, + daRetrieverHeight: daRetrieverHeight, + headerStore: headerStore, + dataStore: dataStore, + heightInCh: make(chan common.DAHeightEvent, 100), + errorCh: errorCh, + logger: logger.With().Str("component", "syncer").Logger(), + gracePeriodMultiplier: gracePeriodMultiplier, + blockFullnessEMA: blockFullnessEMA, + gracePeriodConfig: newForcedInclusionGracePeriodConfig(), } } @@ -677,15 +735,92 @@ func hashTx(tx []byte) string { return hex.EncodeToString(hash[:]) } +// calculateBlockFullness returns a value between 0.0 and 1.0 indicating how full the block is. +// It estimates fullness based on total data size. +// This is a heuristic - actual limits may vary by execution layer. +func (s *Syncer) calculateBlockFullness(data *types.Data) float64 { + const maxDataSize = seqcommon.AbsoluteMaxBlobSize + + var fullness float64 + count := 0 + + // Check data size fullness + dataSize := uint64(0) + for _, tx := range data.Txs { + dataSize += uint64(len(tx)) + } + sizeFullness := float64(dataSize) / float64(maxDataSize) + fullness += min(sizeFullness, 1.0) + count++ + + // Return average fullness + return fullness / float64(count) +} + +// updateDynamicGracePeriod updates the grace period multiplier based on block fullness. +// When blocks are consistently full, the multiplier increases (more lenient). +// When blocks have capacity, the multiplier decreases (stricter). +func (s *Syncer) updateDynamicGracePeriod(blockFullness float64) { + // Update exponential moving average of block fullness + currentEMA := *s.blockFullnessEMA.Load() + alpha := s.gracePeriodConfig.dynamicAdjustmentRate + newEMA := alpha*blockFullness + (1-alpha)*currentEMA + s.blockFullnessEMA.Store(&newEMA) + + // Adjust grace period multiplier based on EMA + currentMultiplier := *s.gracePeriodMultiplier.Load() + threshold := s.gracePeriodConfig.dynamicFullnessThreshold + + var newMultiplier float64 + if newEMA > threshold { + // Blocks are full - increase grace period (more lenient) + adjustment := alpha * (newEMA - threshold) / (1.0 - threshold) + newMultiplier = currentMultiplier + adjustment + } else { + // Blocks have capacity - decrease grace period (stricter) + adjustment := alpha * (threshold - newEMA) / threshold + newMultiplier = currentMultiplier - adjustment + } + + // Clamp to min/max bounds + newMultiplier = max(newMultiplier, s.gracePeriodConfig.dynamicMinMultiplier) + newMultiplier = min(newMultiplier, s.gracePeriodConfig.dynamicMaxMultiplier) + + s.gracePeriodMultiplier.Store(&newMultiplier) + + // Log significant changes (more than 10% change) + if math.Abs(newMultiplier-currentMultiplier) > 0.1 { + s.logger.Debug(). + Float64("block_fullness", blockFullness). + Float64("fullness_ema", newEMA). + Float64("old_multiplier", currentMultiplier). + Float64("new_multiplier", newMultiplier). + Msg("dynamic grace period multiplier adjusted") + } +} + +// getEffectiveGracePeriod returns the current effective grace period considering dynamic adjustment. +func (s *Syncer) getEffectiveGracePeriod() uint64 { + multiplier := *s.gracePeriodMultiplier.Load() + effectivePeriod := math.Round(float64(s.gracePeriodConfig.basePeriod) * multiplier) + minPeriod := float64(s.gracePeriodConfig.basePeriod) * s.gracePeriodConfig.dynamicMinMultiplier + + return uint64(max(effectivePeriod, minPeriod)) +} + // verifyForcedInclusionTxs verifies that forced inclusion transactions from DA are properly handled. // Note: Due to block size constraints (MaxBytes), sequencers may defer forced inclusion transactions // to future blocks (smoothing). This is legitimate behavior within an epoch. -// However, ALL forced inclusion txs from an epoch MUST be included before the next epoch begins. +// However, ALL forced inclusion txs from an epoch MUST be included before the next epoch begins or grace boundary (whichever comes later). func (s *Syncer) verifyForcedInclusionTxs(currentState types.State, data *types.Data) error { if s.fiRetriever == nil { return nil } + // Update dynamic grace period based on block fullness + blockFullness := s.calculateBlockFullness(data) + s.updateDynamicGracePeriod(blockFullness) + // Retrieve forced inclusion transactions from DA for current epoch forcedIncludedTxsEvent, err := s.fiRetriever.RetrieveForcedIncludedTxs(s.ctx, currentState.DAHeight) if err != nil { @@ -741,16 +876,36 @@ func (s *Syncer) verifyForcedInclusionTxs(currentState types.State, data *types. } // Check if we've moved past any epoch boundaries with pending txs + // Grace period: Allow forced inclusion txs from epoch N to be included in epoch N+1, N+2, etc. + // Only flag as malicious if past grace boundary to prevent false positives during chain congestion. var maliciousTxs, remainingPending []pendingForcedInclusionTx + var txsInGracePeriod int for _, pending := range stillPending { - // If current DA height is past this epoch's end, these txs should have been included - if currentState.DAHeight > pending.EpochEnd { + // Calculate grace boundary: epoch end + (effective grace periods × epoch size) + effectiveGracePeriod := s.getEffectiveGracePeriod() + graceBoundary := pending.EpochEnd + (effectiveGracePeriod * s.genesis.DAEpochForcedInclusion) + + if currentState.DAHeight > graceBoundary { maliciousTxs = append(maliciousTxs, pending) + s.logger.Warn(). + Uint64("current_da_height", currentState.DAHeight). + Uint64("epoch_end", pending.EpochEnd). + Uint64("grace_boundary", graceBoundary). + Uint64("base_grace_periods", s.gracePeriodConfig.basePeriod). + Uint64("effective_grace_periods", effectiveGracePeriod). + Float64("grace_multiplier", *s.gracePeriodMultiplier.Load()). + Str("tx_hash", pending.TxHash[:16]). + Msg("forced inclusion transaction past grace boundary - marking as malicious") } else { remainingPending = append(remainingPending, pending) + if currentState.DAHeight > pending.EpochEnd { + txsInGracePeriod++ + } } } + s.metrics.ForcedInclusionTxsInGracePeriod.Set(float64(txsInGracePeriod)) + // Update pending map - clear old entries and store only remaining pending s.pendingForcedInclusionTxs.Range(func(key, value any) bool { s.pendingForcedInclusionTxs.Delete(key) @@ -760,14 +915,20 @@ func (s *Syncer) verifyForcedInclusionTxs(currentState types.State, data *types. s.pendingForcedInclusionTxs.Store(pending.TxHash, pending) } - // If there are transactions from past epochs that weren't included, sequencer is malicious + // If there are transactions past grace boundary that weren't included, sequencer is malicious if len(maliciousTxs) > 0 { + s.metrics.ForcedInclusionTxsMalicious.Add(float64(len(maliciousTxs))) + + effectiveGracePeriod := s.getEffectiveGracePeriod() s.logger.Error(). Uint64("height", data.Height()). Uint64("current_da_height", currentState.DAHeight). Int("malicious_count", len(maliciousTxs)). - Msg("SEQUENCER IS MALICIOUS: forced inclusion transactions from past epoch(s) not included") - return errors.Join(errMaliciousProposer, fmt.Errorf("sequencer is malicious: %d forced inclusion transactions from past epoch(s) not included", len(maliciousTxs))) + Uint64("base_grace_periods", s.gracePeriodConfig.basePeriod). + Uint64("effective_grace_periods", effectiveGracePeriod). + Float64("grace_multiplier", *s.gracePeriodMultiplier.Load()). + Msg("SEQUENCER IS MALICIOUS: forced inclusion transactions past grace boundary not included") + return errors.Join(errMaliciousProposer, fmt.Errorf("sequencer is malicious: %d forced inclusion transactions past grace boundary (base_grace_periods=%d, effective_grace_periods=%d) not included", len(maliciousTxs), s.gracePeriodConfig.basePeriod, effectiveGracePeriod)) } // Log current state diff --git a/block/internal/syncing/syncer_forced_inclusion_test.go b/block/internal/syncing/syncer_forced_inclusion_test.go index 741432eb28..77909fc2f0 100644 --- a/block/internal/syncing/syncer_forced_inclusion_test.go +++ b/block/internal/syncing/syncer_forced_inclusion_test.go @@ -3,6 +3,7 @@ package syncing import ( "bytes" "context" + "sync/atomic" "testing" "time" @@ -23,6 +24,323 @@ import ( "github.com/evstack/ev-node/types" ) +func TestCalculateBlockFullness_HalfFull(t *testing.T) { + s := &Syncer{} + + // Create 5000 transactions of 100 bytes each = 500KB + txs := make([]types.Tx, 5000) + for i := range txs { + txs[i] = make([]byte, 100) + } + + data := &types.Data{ + Txs: txs, + } + + fullness := s.calculateBlockFullness(data) + // Size fullness: 500000/2097152 ≈ 0.238 + require.InDelta(t, 0.238, fullness, 0.05) +} + +func TestCalculateBlockFullness_Full(t *testing.T) { + s := &Syncer{} + + // Create 10000 transactions of 210 bytes each = ~2MB + txs := make([]types.Tx, 10000) + for i := range txs { + txs[i] = make([]byte, 210) + } + + data := &types.Data{ + Txs: txs, + } + + fullness := s.calculateBlockFullness(data) + // Both metrics at or near 1.0 + require.Greater(t, fullness, 0.95) +} + +func TestCalculateBlockFullness_VerySmall(t *testing.T) { + s := &Syncer{} + + data := &types.Data{ + Txs: []types.Tx{[]byte("tx1"), []byte("tx2")}, + } + + fullness := s.calculateBlockFullness(data) + // Very small relative to heuristic limits + require.Less(t, fullness, 0.001) +} + +func TestUpdateDynamicGracePeriod_NoChangeWhenBelowThreshold(t *testing.T) { + initialMultiplier := 1.0 + initialEMA := 0.1 // Well below threshold + + config := forcedInclusionGracePeriodConfig{ + dynamicMinMultiplier: 0.5, + dynamicMaxMultiplier: 3.0, + dynamicFullnessThreshold: 0.8, + dynamicAdjustmentRate: 0.01, // Low adjustment rate + } + + s := &Syncer{ + gracePeriodMultiplier: &atomic.Pointer[float64]{}, + blockFullnessEMA: &atomic.Pointer[float64]{}, + gracePeriodConfig: config, + metrics: common.NopMetrics(), + } + s.gracePeriodMultiplier.Store(&initialMultiplier) + s.blockFullnessEMA.Store(&initialEMA) + + // Update with low fullness - multiplier should stay at 1.0 initially + s.updateDynamicGracePeriod(0.2) + + // With low adjustment rate and starting EMA below threshold, + // multiplier should not change significantly on first call + newMultiplier := *s.gracePeriodMultiplier.Load() + require.InDelta(t, 1.0, newMultiplier, 0.05) +} + +func TestUpdateDynamicGracePeriod_IncreaseOnHighFullness(t *testing.T) { + initialMultiplier := 1.0 + initialEMA := 0.5 + + s := &Syncer{ + gracePeriodConfig: forcedInclusionGracePeriodConfig{ + dynamicMinMultiplier: 0.5, + dynamicMaxMultiplier: 3.0, + dynamicFullnessThreshold: 0.8, + dynamicAdjustmentRate: 0.1, + }, + gracePeriodMultiplier: &atomic.Pointer[float64]{}, + blockFullnessEMA: &atomic.Pointer[float64]{}, + metrics: common.NopMetrics(), + } + s.gracePeriodMultiplier.Store(&initialMultiplier) + s.blockFullnessEMA.Store(&initialEMA) + + // Update multiple times with very high fullness to build up the effect + for i := 0; i < 20; i++ { + s.updateDynamicGracePeriod(0.95) + } + + // EMA should increase + newEMA := *s.blockFullnessEMA.Load() + require.Greater(t, newEMA, initialEMA) + + // Multiplier should increase because EMA is now above threshold + newMultiplier := *s.gracePeriodMultiplier.Load() + require.Greater(t, newMultiplier, initialMultiplier) +} + +func TestUpdateDynamicGracePeriod_DecreaseOnLowFullness(t *testing.T) { + initialMultiplier := 2.0 + initialEMA := 0.9 + + s := &Syncer{ + gracePeriodConfig: forcedInclusionGracePeriodConfig{ + dynamicMinMultiplier: 0.5, + dynamicMaxMultiplier: 3.0, + dynamicFullnessThreshold: 0.8, + dynamicAdjustmentRate: 0.1, + }, + gracePeriodMultiplier: &atomic.Pointer[float64]{}, + blockFullnessEMA: &atomic.Pointer[float64]{}, + metrics: common.NopMetrics(), + } + s.gracePeriodMultiplier.Store(&initialMultiplier) + s.blockFullnessEMA.Store(&initialEMA) + + // Update multiple times with low fullness to build up the effect + for i := 0; i < 20; i++ { + s.updateDynamicGracePeriod(0.2) + } + + // EMA should decrease significantly + newEMA := *s.blockFullnessEMA.Load() + require.Less(t, newEMA, initialEMA) + + // Multiplier should decrease + newMultiplier := *s.gracePeriodMultiplier.Load() + require.Less(t, newMultiplier, initialMultiplier) +} + +func TestUpdateDynamicGracePeriod_ClampToMin(t *testing.T) { + initialMultiplier := 0.6 + initialEMA := 0.1 + + s := &Syncer{ + gracePeriodConfig: forcedInclusionGracePeriodConfig{ + dynamicMinMultiplier: 0.5, + dynamicMaxMultiplier: 3.0, + dynamicFullnessThreshold: 0.8, + dynamicAdjustmentRate: 0.5, // High rate to force clamping + }, + gracePeriodMultiplier: &atomic.Pointer[float64]{}, + blockFullnessEMA: &atomic.Pointer[float64]{}, + metrics: common.NopMetrics(), + } + s.gracePeriodMultiplier.Store(&initialMultiplier) + s.blockFullnessEMA.Store(&initialEMA) + + // Update many times with very low fullness - should eventually clamp to min + for i := 0; i < 50; i++ { + s.updateDynamicGracePeriod(0.0) + } + + newMultiplier := *s.gracePeriodMultiplier.Load() + require.Equal(t, 0.5, newMultiplier) +} + +func TestUpdateDynamicGracePeriod_ClampToMax(t *testing.T) { + initialMultiplier := 2.5 + initialEMA := 0.9 + + s := &Syncer{ + gracePeriodConfig: forcedInclusionGracePeriodConfig{ + dynamicMinMultiplier: 0.5, + dynamicMaxMultiplier: 3.0, + dynamicFullnessThreshold: 0.8, + dynamicAdjustmentRate: 0.5, // High rate to force clamping + }, + gracePeriodMultiplier: &atomic.Pointer[float64]{}, + blockFullnessEMA: &atomic.Pointer[float64]{}, + metrics: common.NopMetrics(), + } + s.gracePeriodMultiplier.Store(&initialMultiplier) + s.blockFullnessEMA.Store(&initialEMA) + + // Update many times with very high fullness - should eventually clamp to max + for i := 0; i < 50; i++ { + s.updateDynamicGracePeriod(1.0) + } + + newMultiplier := *s.gracePeriodMultiplier.Load() + require.Equal(t, 3.0, newMultiplier) +} + +func TestGetEffectiveGracePeriod_WithMultiplier(t *testing.T) { + multiplier := 2.5 + + s := &Syncer{ + gracePeriodConfig: forcedInclusionGracePeriodConfig{ + basePeriod: 2, + dynamicMinMultiplier: 0.5, + dynamicMaxMultiplier: 3.0, + dynamicFullnessThreshold: 0.8, + dynamicAdjustmentRate: 0.05, + }, + gracePeriodMultiplier: &atomic.Pointer[float64]{}, + } + s.gracePeriodMultiplier.Store(&multiplier) + + effective := s.getEffectiveGracePeriod() + // 2 * 2.5 = 5 + require.Equal(t, uint64(5), effective) +} + +func TestGetEffectiveGracePeriod_RoundingUp(t *testing.T) { + multiplier := 2.6 + + s := &Syncer{ + gracePeriodConfig: forcedInclusionGracePeriodConfig{ + basePeriod: 2, + dynamicMinMultiplier: 0.5, + dynamicMaxMultiplier: 3.0, + dynamicFullnessThreshold: 0.8, + dynamicAdjustmentRate: 0.05, + }, + gracePeriodMultiplier: &atomic.Pointer[float64]{}, + } + s.gracePeriodMultiplier.Store(&multiplier) + + effective := s.getEffectiveGracePeriod() + // 2 * 2.6 = 5.2, rounds to 5 + require.Equal(t, uint64(5), effective) +} + +func TestGetEffectiveGracePeriod_EnsuresMinimum(t *testing.T) { + multiplier := 0.3 + + s := &Syncer{ + gracePeriodConfig: forcedInclusionGracePeriodConfig{ + basePeriod: 4, + dynamicMinMultiplier: 0.5, + dynamicMaxMultiplier: 3.0, + dynamicFullnessThreshold: 0.8, + dynamicAdjustmentRate: 0.05, + }, + gracePeriodMultiplier: &atomic.Pointer[float64]{}, + } + s.gracePeriodMultiplier.Store(&multiplier) + + effective := s.getEffectiveGracePeriod() + // 4 * 0.3 = 1.2, but minimum is 4 * 0.5 = 2 + require.Equal(t, uint64(2), effective) +} + +func TestDynamicGracePeriod_Integration_HighCongestion(t *testing.T) { + initialMultiplier := 1.0 + initialEMA := 0.3 + + s := &Syncer{ + gracePeriodConfig: forcedInclusionGracePeriodConfig{ + basePeriod: 2, + dynamicMinMultiplier: 0.5, + dynamicMaxMultiplier: 3.0, + dynamicFullnessThreshold: 0.8, + dynamicAdjustmentRate: 0.1, + }, + gracePeriodMultiplier: &atomic.Pointer[float64]{}, + blockFullnessEMA: &atomic.Pointer[float64]{}, + metrics: common.NopMetrics(), + } + s.gracePeriodMultiplier.Store(&initialMultiplier) + s.blockFullnessEMA.Store(&initialEMA) + + // Simulate processing many blocks with very high fullness (above threshold) + for i := 0; i < 50; i++ { + s.updateDynamicGracePeriod(0.95) + } + + // Multiplier should have increased due to sustained high fullness + finalMultiplier := *s.gracePeriodMultiplier.Load() + require.Greater(t, finalMultiplier, initialMultiplier, "multiplier should increase with sustained congestion") + + // Effective grace period should be higher than base + effectiveGracePeriod := s.getEffectiveGracePeriod() + require.Greater(t, effectiveGracePeriod, s.gracePeriodConfig.basePeriod, "effective grace period should be higher than base") +} + +func TestDynamicGracePeriod_Integration_LowCongestion(t *testing.T) { + initialMultiplier := 2.0 + initialEMA := 0.85 + + s := &Syncer{ + gracePeriodConfig: forcedInclusionGracePeriodConfig{ + basePeriod: 2, + dynamicMinMultiplier: 0.5, + dynamicMaxMultiplier: 3.0, + dynamicFullnessThreshold: 0.8, + dynamicAdjustmentRate: 0.1, + }, + gracePeriodMultiplier: &atomic.Pointer[float64]{}, + blockFullnessEMA: &atomic.Pointer[float64]{}, + metrics: common.NopMetrics(), + } + s.gracePeriodMultiplier.Store(&initialMultiplier) + s.blockFullnessEMA.Store(&initialEMA) + + // Simulate processing many blocks with very low fullness (below threshold) + for i := 0; i < 50; i++ { + s.updateDynamicGracePeriod(0.1) + } + + // Multiplier should have decreased + finalMultiplier := *s.gracePeriodMultiplier.Load() + require.Less(t, finalMultiplier, initialMultiplier, "multiplier should decrease with low congestion") +} + func TestVerifyForcedInclusionTxs_AllTransactionsIncluded(t *testing.T) { ds := dssync.MutexWrap(datastore.NewMapDatastore()) st := store.New(ds) @@ -198,15 +516,28 @@ func TestVerifyForcedInclusionTxs_MissingTransactions(t *testing.T) { return bytes.Equal(ns, namespaceForcedInclusionBz) })).Return(&coreda.GetIDsResult{IDs: [][]byte{}, Timestamp: time.Now()}, nil).Once() - // Now simulate moving to next epoch - should fail if tx still not included - currentState.DAHeight = 1 // Move past epoch end (epoch was [0, 0]) + // Move to next epoch but still within grace period + currentState.DAHeight = 1 // Move to epoch end (epoch was [0, 0]) data2 := makeData(gen.ChainID, 2, 1) data2.Txs[0] = types.Tx([]byte("regular_tx_3")) err = s.verifyForcedInclusionTxs(currentState, data2) + require.NoError(t, err) // Should pass since DAHeight=1 equals grace boundary, not past it + + // Mock DA for height 2 to return no forced inclusion transactions + mockDA.EXPECT().GetIDs(mock.Anything, uint64(2), mock.MatchedBy(func(ns []byte) bool { + return bytes.Equal(ns, namespaceForcedInclusionBz) + })).Return(&coreda.GetIDsResult{IDs: [][]byte{}, Timestamp: time.Now()}, nil).Once() + + // Now move past grace boundary - should fail if tx still not included + currentState.DAHeight = 2 // Move past grace boundary (graceBoundary = 0 + 1*1 = 1) + data3 := makeData(gen.ChainID, 3, 1) + data3.Txs[0] = types.Tx([]byte("regular_tx_4")) + + err = s.verifyForcedInclusionTxs(currentState, data3) require.Error(t, err) require.Contains(t, err.Error(), "sequencer is malicious") - require.Contains(t, err.Error(), "forced inclusion transactions from past epoch(s) not included") + require.Contains(t, err.Error(), "past grace boundary") } func TestVerifyForcedInclusionTxs_PartiallyIncluded(t *testing.T) { @@ -301,15 +632,31 @@ func TestVerifyForcedInclusionTxs_PartiallyIncluded(t *testing.T) { return bytes.Equal(ns, namespaceForcedInclusionBz) })).Return(&coreda.GetIDsResult{IDs: [][]byte{}, Timestamp: time.Now()}, nil).Once() - // Now simulate moving to next epoch - should fail if dataBin2 still not included - currentState.DAHeight = 1 // Move past epoch end (epoch was [0, 0]) + // Move to DAHeight=1 (still within grace period since graceBoundary = 0 + 1*1 = 1) + currentState.DAHeight = 1 data2 := makeData(gen.ChainID, 2, 1) data2.Txs[0] = types.Tx([]byte("regular_tx_3")) + // Verify - should pass since we're at the grace boundary, not past it err = s.verifyForcedInclusionTxs(currentState, data2) + require.NoError(t, err) + + // Mock DA for height 2 (when we move to DAHeight 2) + mockDA.EXPECT().GetIDs(mock.Anything, uint64(2), mock.MatchedBy(func(ns []byte) bool { + return bytes.Equal(ns, namespaceForcedInclusionBz) + })).Return(&coreda.GetIDsResult{IDs: [][]byte{}, Timestamp: time.Now()}, nil).Once() + + // Now simulate moving past grace boundary - should fail if dataBin2 still not included + // With basePeriod=1 and DAEpochForcedInclusion=1, graceBoundary = 0 + (1*1) = 1 + // So we need DAHeight > 1 to trigger the error + currentState.DAHeight = 2 // Move past grace boundary + data3 := makeData(gen.ChainID, 3, 1) + data3.Txs[0] = types.Tx([]byte("regular_tx_4")) + + err = s.verifyForcedInclusionTxs(currentState, data3) require.Error(t, err) require.Contains(t, err.Error(), "sequencer is malicious") - require.Contains(t, err.Error(), "forced inclusion transactions from past epoch(s) not included") + require.Contains(t, err.Error(), "past grace boundary") } func TestVerifyForcedInclusionTxs_NoForcedTransactions(t *testing.T) { @@ -560,7 +907,6 @@ func TestVerifyForcedInclusionTxs_DeferralWithinEpoch(t *testing.T) { pendingCount++ return true }) - require.Equal(t, 1, pendingCount, "should have 1 pending forced inclusion tx") // Mock DA for second verification at same epoch (height 104 - epoch end) mockDA.EXPECT().GetIDs(mock.Anything, uint64(100), mock.MatchedBy(func(ns []byte) bool { @@ -696,68 +1042,102 @@ func TestVerifyForcedInclusionTxs_MaliciousAfterEpochEnd(t *testing.T) { // Verify - should pass, tx can be deferred within epoch err = s.verifyForcedInclusionTxs(currentState, data1) require.NoError(t, err) +} - // Verify that the forced tx is tracked as pending - pendingCount := 0 - s.pendingForcedInclusionTxs.Range(func(key, value any) bool { - pendingCount++ - return true - }) - require.Equal(t, 1, pendingCount, "should have 1 pending forced inclusion tx") +// TestVerifyForcedInclusionTxs_SmoothingExceedsEpoch tests the critical scenario where +// forced inclusion transactions cannot all be included before an epoch ends. +// This demonstrates that the system correctly detects malicious behavior when +// transactions remain pending after the epoch boundary. +func TestVerifyForcedInclusionTxs_SmoothingExceedsEpoch(t *testing.T) { + ds := dssync.MutexWrap(datastore.NewMapDatastore()) + st := store.New(ds) - // Process another block within same epoch - forced tx still not included - // Mock DA for second verification at same epoch (height 102 - epoch end) - mockDA.EXPECT().GetIDs(mock.Anything, uint64(100), mock.MatchedBy(func(ns []byte) bool { - return bytes.Equal(ns, namespaceForcedInclusionBz) - })).Return(&coreda.GetIDsResult{IDs: [][]byte{[]byte("fi1")}, Timestamp: time.Now()}, nil).Once() + cm, err := cache.NewCacheManager(config.DefaultConfig(), zerolog.Nop()) + require.NoError(t, err) - mockDA.EXPECT().Get(mock.Anything, mock.Anything, mock.MatchedBy(func(ns []byte) bool { - return bytes.Equal(ns, namespaceForcedInclusionBz) - })).Return([][]byte{dataBin}, nil).Once() + addr, pub, signer := buildSyncTestSigner(t) + gen := genesis.Genesis{ + ChainID: "tchain", + InitialHeight: 1, + StartTime: time.Now().Add(-time.Second), + ProposerAddress: addr, + DAStartHeight: 100, + DAEpochForcedInclusion: 3, // Epoch: [100, 102] + } - mockDA.EXPECT().GetIDs(mock.Anything, uint64(101), mock.MatchedBy(func(ns []byte) bool { - return bytes.Equal(ns, namespaceForcedInclusionBz) - })).Return(&coreda.GetIDsResult{IDs: [][]byte{}, Timestamp: time.Now()}, nil).Once() + cfg := config.DefaultConfig() + cfg.DA.ForcedInclusionNamespace = "nsForcedInclusion" - mockDA.EXPECT().GetIDs(mock.Anything, uint64(102), mock.MatchedBy(func(ns []byte) bool { - return bytes.Equal(ns, namespaceForcedInclusionBz) - })).Return(&coreda.GetIDsResult{IDs: [][]byte{}, Timestamp: time.Now()}, nil).Once() + mockExec := testmocks.NewMockExecutor(t) + mockExec.EXPECT().InitChain(mock.Anything, mock.Anything, uint64(1), "tchain"). + Return([]byte("app0"), uint64(1024), nil).Once() - data2 := makeData(gen.ChainID, 2, 1) - data2.Txs[0] = types.Tx([]byte("regular_tx_2")) + mockDA := testmocks.NewMockDA(t) - // Still at epoch 100, should still pass - err = s.verifyForcedInclusionTxs(currentState, data2) - require.NoError(t, err) + daClient := da.NewClient(da.Config{ + DA: mockDA, + Logger: zerolog.Nop(), + Namespace: cfg.DA.Namespace, + DataNamespace: cfg.DA.DataNamespace, + ForcedInclusionNamespace: cfg.DA.ForcedInclusionNamespace, + }) + daRetriever := NewDARetriever(daClient, cm, gen, zerolog.Nop()) + fiRetriever := da.NewForcedInclusionRetriever(daClient, gen, zerolog.Nop()) - // Mock DA retrieval for next epoch (DA height 105 - epoch end) - // Epoch boundaries: [103, 105] - // The retriever will fetch heights 103, 104, 105 + s := NewSyncer( + st, + mockExec, + daClient, + cm, + common.NopMetrics(), + cfg, + gen, + common.NewMockBroadcaster[*types.SignedHeader](t), + common.NewMockBroadcaster[*types.Data](t), + zerolog.Nop(), + common.DefaultBlockOptions(), + make(chan error, 1), + ) + s.daRetriever = daRetriever + s.fiRetriever = fiRetriever - // Height 103 (epoch start) - mockDA.EXPECT().GetIDs(mock.Anything, uint64(103), mock.MatchedBy(func(ns []byte) bool { - return bytes.Equal(ns, namespaceForcedInclusionBz) - })).Return(&coreda.GetIDsResult{IDs: [][]byte{}, Timestamp: time.Now()}, nil).Once() + require.NoError(t, s.initializeState()) + s.ctx = context.Background() - // Height 104 (intermediate) - mockDA.EXPECT().GetIDs(mock.Anything, uint64(104), mock.MatchedBy(func(ns []byte) bool { + namespaceForcedInclusionBz := coreda.NamespaceFromString(cfg.DA.GetForcedInclusionNamespace()).Bytes() + + // Create 3 forced inclusion transactions + dataBin1, _ := makeSignedDataBytes(t, gen.ChainID, 10, addr, pub, signer, 2) + dataBin2, _ := makeSignedDataBytes(t, gen.ChainID, 11, addr, pub, signer, 2) + dataBin3, _ := makeSignedDataBytes(t, gen.ChainID, 12, addr, pub, signer, 2) + + // Mock DA retrieval for Epoch 1: [100, 102] + mockDA.EXPECT().GetIDs(mock.Anything, uint64(100), mock.MatchedBy(func(ns []byte) bool { return bytes.Equal(ns, namespaceForcedInclusionBz) - })).Return(&coreda.GetIDsResult{IDs: [][]byte{}, Timestamp: time.Now()}, nil).Once() + })).Return(&coreda.GetIDsResult{ + IDs: [][]byte{[]byte("fi1"), []byte("fi2"), []byte("fi3")}, + Timestamp: time.Now(), + }, nil).Once() - // Height 105 (epoch end) - mockDA.EXPECT().GetIDs(mock.Anything, uint64(105), mock.MatchedBy(func(ns []byte) bool { + mockDA.EXPECT().Get(mock.Anything, mock.Anything, mock.MatchedBy(func(ns []byte) bool { return bytes.Equal(ns, namespaceForcedInclusionBz) - })).Return(&coreda.GetIDsResult{IDs: [][]byte{}, Timestamp: time.Now()}, nil).Once() + })).Return([][]byte{dataBin1, dataBin2, dataBin3}, nil).Once() - // Third block is in the next epoch (at epoch end 105) without including the forced tx - data3 := makeData(gen.ChainID, 3, 1) - data3.Txs[0] = types.Tx([]byte("regular_tx_3")) + for height := uint64(101); height <= 102; height++ { + mockDA.EXPECT().GetIDs(mock.Anything, height, mock.MatchedBy(func(ns []byte) bool { + return bytes.Equal(ns, namespaceForcedInclusionBz) + })).Return(&coreda.GetIDsResult{IDs: [][]byte{}, Timestamp: time.Now()}, nil).Once() + } - currentState.DAHeight = 105 // At epoch end [103, 105], past previous epoch [100, 102] + // Block at DA height 102 (epoch end): Only includes 2 of 3 txs + // The third tx remains pending - legitimate within the epoch + data1 := makeData(gen.ChainID, 1, 2) + data1.Txs[0] = types.Tx(dataBin1) + data1.Txs[1] = types.Tx(dataBin2) - // Verify - should FAIL since forced tx from previous epoch was never included - err = s.verifyForcedInclusionTxs(currentState, data3) - require.Error(t, err) - require.Contains(t, err.Error(), "sequencer is malicious") - require.Contains(t, err.Error(), "forced inclusion transactions from past epoch(s) not included") + currentState := s.GetLastState() + currentState.DAHeight = 102 // At epoch end + + err = s.verifyForcedInclusionTxs(currentState, data1) + require.NoError(t, err, "smoothing within epoch should be allowed") } diff --git a/docs/adr/adr-019-forced-inclusion-mechanism.md b/docs/adr/adr-019-forced-inclusion-mechanism.md index c8b63bb203..d589263f2c 100644 --- a/docs/adr/adr-019-forced-inclusion-mechanism.md +++ b/docs/adr/adr-019-forced-inclusion-mechanism.md @@ -304,7 +304,7 @@ func (s *BasedSequencer) SubmitBatchTxs(ctx context.Context, req SubmitBatchTxsR #### Syncer Verification -Full nodes verify forced inclusion in the sync process with support for transaction smoothing across multiple blocks: +Full nodes verify forced inclusion in the sync process with support for transaction smoothing across multiple blocks and a configurable grace period: ```go func (s *Syncer) verifyForcedInclusionTxs(currentState State, data *Data) error { @@ -347,11 +347,16 @@ func (s *Syncer) verifyForcedInclusionTxs(currentState State, data *Data) error } } - // 5. Check for malicious behavior: pending txs past their epoch boundary + // 5. Check for malicious behavior: pending txs past their grace boundary + // Grace period provides tolerance for temporary chain congestion var maliciousTxs, remainingPending []pendingForcedInclusionTx for _, pending := range stillPending { - // If current DA height is past this epoch's end, these txs MUST have been included - if currentState.DAHeight > pending.EpochEnd { + // Calculate grace boundary: epoch end + (effective grace periods × epoch size) + effectiveGracePeriod := s.getEffectiveGracePeriod() + graceBoundary := pending.EpochEnd + (effectiveGracePeriod * s.genesis.DAEpochForcedInclusion) + + // If current DA height is past the grace boundary, these txs MUST have been included + if currentState.DAHeight > graceBoundary { maliciousTxs = append(maliciousTxs, pending) } else { remainingPending = append(remainingPending, pending) @@ -361,9 +366,9 @@ func (s *Syncer) verifyForcedInclusionTxs(currentState State, data *Data) error // 6. Update pending map with only remaining valid pending txs pendingForcedInclusionTxs = remainingPending - // 7. Reject block if sequencer censored forced txs past epoch boundary + // 7. Reject block if sequencer censored forced txs past grace boundary if len(maliciousTxs) > 0 { - return fmt.Errorf("sequencer is malicious: %d forced inclusion transactions from past epoch(s) not included", len(maliciousTxs)) + return fmt.Errorf("sequencer is malicious: %d forced inclusion transactions past grace boundary not included", len(maliciousTxs)) } return nil @@ -440,6 +445,122 @@ if errors.Is(err, coreda.ErrHeightFromFuture) { } ``` +#### Grace Period for Forced Inclusion + +The grace period mechanism provides tolerance for chain congestion while maintaining censorship resistance: + +**Problem**: If the DA layer experiences temporary unavailability or the chain congestion, the sequencer may be unable to fetch forced inclusion transactions from a completed epoch. Without a grace period, full nodes would immediately flag the sequencer as malicious. + +**Solution**: The grace period mechanism allows forced inclusion transactions from epoch N to be included in subsequent epochs before being flagged as malicious. The grace period is dynamically adjusted based on chain fullness. + +**Grace Boundary Calculation**: + +```go +graceBoundary := epochEnd + (effectiveGracePeriod * DAEpochForcedInclusion) + +// Example with base grace period = 1 epoch, DAEpochForcedInclusion = 50: +// - Epoch N ends at DA height 100 +// - Grace boundary = 100 + (1 * 50) = 150 (adjusted dynamically by chain fullness) +// - Transaction must be included while currentDAHeight <= graceBoundary +// - If currentDAHeight > graceBoundary without inclusion, sequencer is malicious +``` + +**Configuration Recommendations**: + +- **Production (default)**: Base grace period of 1 epoch + - Automatically adjusted based on chain fullness + - Balances censorship resistance with reliability +- **High Security / Reliable DA**: Minimum grace period + - Stricter enforcement when block space is available + - Requires 99.9%+ DA uptime + - Faster detection of censorship +- **Unreliable DA**: Network adjusts grace period dynamically + - Higher tolerance (up to 3x base period) when chain is congested + - Reduced censorship resistance temporarily to avoid false positives + +**Verification Logic**: + +1. Forced inclusion transactions from epoch N are tracked with their epoch boundaries +2. Transactions not immediately included are added to pending queue +3. Each block, full nodes check if pending transactions are past their grace boundary +4. If `currentDAHeight > graceBoundary`, the sequencer is flagged as malicious (strictly greater than) +5. Transactions within the grace period (where `currentDAHeight <= graceBoundary`) remain in pending queue without error + +**Benefits**: + +- Prevents false positives during temporary DA outages +- Maintains censorship resistance (transactions must be included within grace window) +- Configurable trade-off between reliability and security +- Allows networks to adapt to their DA layer's reliability characteristics + +**Examples and Edge Cases**: + +Configuration: `DAEpochForcedInclusion = 50`, Base grace period of 1 epoch (dynamically adjusted) + +_Example 1: Normal Inclusion (Within Same Epoch)_ + +``` +- Forced tx submitted to DA at height 75 (epoch 51-100) +- Sequencer fetches at height 101 (next epoch start) +- Sequencer includes tx in block at DA height 105 +- Result: ✅ Valid - included within same epoch +``` + +_Example 2: Grace Period Usage (Included in Next Epoch)_ + +``` +- Forced tx submitted to DA at height 75 (epoch 51-100) +- Sequencer fetches at height 101 +- DA temporarily unavailable, sequencer cannot fetch +- Sequencer includes tx at DA height 125 (epoch 101-150) +- Grace boundary = 100 + (1 × 50) = 150 +- Result: ✅ Valid - within grace period +``` + +_Example 3: Malicious Sequencer (Past Grace Boundary)_ + +``` +- Forced tx submitted to DA at height 75 (epoch 51-100) +- Sequencer fetches at height 101 +- Sequencer deliberately omits tx +- Block produced at DA height 151 (past grace boundary 150) +- Full node detects: currentDAHeight (151) > graceBoundary (150) +- Result: ❌ Block rejected, sequencer flagged as malicious +``` + +_Example 4: Low Chain Activity (Minimum Grace Period)_ + +``` +- Chain is mostly empty (<20% full) +- Grace period is at minimum (0.5x base period) +- Forced tx submitted at height 75 (epoch 51-100) +- Grace boundary ≈ 100 + (0.5 × 50) = 125 +- Stricter enforcement applied when chain is empty +- Result: Faster censorship detection when block space is available +``` + +_Example 5: Multiple Pending Transactions_ + +``` +- Tx A from epoch ending at height 100, grace boundary 150 +- Tx B from epoch ending at height 150, grace boundary 200 +- Current DA height: 155 +- Tx A not included: ❌ Past grace boundary - malicious +- Tx B not included: ✅ Within grace period - still pending +- Result: Block rejected due to Tx A +``` + +_Example 6: High Chain Activity (Extended Grace Period)_ + +``` +- Chain is highly congested (>80% full) +- Grace period is extended (up to 3x base period) +- Forced tx submitted at height 75 (epoch 51-100) +- Grace boundary ≈ 100 + (3 × 50) = 250 +- Higher tolerance during congestion to avoid false positives +- Result: Better operational reliability when block space is scarce +``` + #### Size Validation and Max Bytes Handling Both sequencers enforce strict size limits to prevent DoS and ensure batches never exceed the DA layer's limits: @@ -539,7 +660,7 @@ type NodeConfig struct { # genesis.json { "chain_id": "my-rollup", - "forced_inclusion_da_epoch": 10 # Scan 10 DA blocks at a time + "da_epoch_forced_inclusion": 10 # Scan 10 DA blocks at a time } # config.toml @@ -557,7 +678,7 @@ based_sequencer = false # Use traditional sequencer # genesis.json { "chain_id": "my-rollup", - "forced_inclusion_da_epoch": 5 # Scan 5 DA blocks at a time + "da_epoch_forced_inclusion": 5 # Scan 5 DA blocks at a time } # config.toml @@ -573,7 +694,6 @@ based_sequencer = true # Use based sequencer #### Single Sequencer Flow -``` 1. Timer triggers GetNextBatch 2. Fetch forced inclusion txs from DA (via DA Retriever) - Only at epoch boundaries @@ -581,11 +701,9 @@ based_sequencer = true # Use based sequencer 3. Get batch from mempool queue 4. Prepend forced txs to batch 5. Return batch for block production -``` #### Based Sequencer Flow -``` 1. Timer triggers GetNextBatch 2. Check transaction queue for buffered txs 3. If queue empty or epoch boundary: @@ -593,19 +711,28 @@ based_sequencer = true # Use based sequencer - Add to queue 4. Create batch from queue (respecting MaxBytes) 5. Return batch for block production -``` ### Full Node Verification Flow -``` 1. Receive block from DA or P2P 2. Before applying block: - a. Fetch forced inclusion txs from DA at block's DA height + a. Fetch forced inclusion txs from DA at block's DA height (epoch-based) b. Build map of transactions in block - c. Verify all forced txs are in block - d. If missing: reject block, flag malicious proposer + c. Check if pending forced txs from previous epochs are included + d. Add any new forced txs not yet included to pending queue + e. Calculate grace boundary for each pending tx (dynamically adjusted by chain fullness): + graceBoundary = epochEnd + (effectiveGracePeriod × DAEpochForcedInclusion) + f. Check if any pending txs are past their grace boundary + g. If txs past grace boundary are not included: reject block, flag malicious proposer + h. If txs within grace period: keep in pending queue, allow block 3. Apply block if verification passes -``` + +**Grace Period Example** (with base grace period = 1 epoch, `DAEpochForcedInclusion = 50`): + +- Forced tx appears in epoch ending at DA height 100 +- Grace boundary = 100 + (1 × 50) = 150 +- Transaction can be included at any DA height from 101 to 150 +- When currentDAHeight > 150 without inclusion, sequencer is flagged as malicious ### Efficiency Considerations @@ -637,96 +764,20 @@ Every `DAEpochForcedInclusion` DA blocks **Attack Vectors**: -- **Censorship**: Mitigated by forced inclusion verification +### Security Considerations + +- **Censorship**: Mitigated by forced inclusion verification with grace period + - Transactions must be included within grace window (epoch + grace period) + - Full nodes detect and reject blocks from malicious sequencers + - Grace period = 0 provides immediate detection but requires high DA reliability + - Grace period = 1+ balances censorship resistance with operational tolerance - **DA Spam**: Limited by DA layer's native spam protection and two-tier blob size limits - **Block Withholding**: Full nodes can fetch and verify from DA independently - **Oversized Batches**: Prevented by strict size validation at multiple levels - -### Testing Strategy - -#### Unit Tests - -1. **DA Retriever**: - - Epoch boundary calculations - - Height from future handling - - Blob size validation - - Empty epoch handling - -2. **Size Validation**: - - Individual blob size validation (absolute limit) - - Cumulative size checking (batch limit) - - Edge cases (empty blobs, exact limits, exceeding limits) - -3. **Single Sequencer**: - - Forced transaction prepending with size constraints - - Batch trimming when forced + batch exceeds MaxBytes - - Trimmed transactions returned to queue via Prepend - - Pending forced inclusion queue management - - DA height tracking - - Error handling - -4. **BatchQueue**: - - Prepend operation (empty queue, with items, after consuming) - - Multiple prepends (LIFO ordering) - - Space reuse before head position - -5. **Based Sequencer**: - - Queue management with size validation - - Batch size limits strictly enforced - - Transaction buffering across batches - - DA-only operation - - Always checking for new forced txs - -6. **Syncer Verification**: - - All forced txs included (pass) - - Missing forced txs (fail) - - No forced txs (pass) - -#### Integration Tests - -1. **Single Sequencer Integration**: - - Submit to mempool and forced inclusion - - Verify both included in block - - Forced txs appear first - -2. **Based Sequencer Integration**: - - Submit only to DA forced inclusion - - Verify block production - - Mempool submissions ignored - -3. **Verification Flow**: - - Full node rejects block missing forced tx - - Full node accepts block with all forced txs - -#### End-to-End Tests - -1. **User Flow**: - - User submits tx to forced inclusion namespace - - Sequencer includes tx in next epoch - - Full nodes verify inclusion - -2. **Based Rollup**: - - Start network with based sequencer - - Submit transactions to DA - - Verify block production and finalization - -3. **Censorship Resistance**: - - Sequencer ignores specific transaction - - User submits to forced inclusion - - Transaction included in next epoch - - Attempting to exclude causes block rejection - -### Breaking Changes - -1. **Sequencer Initialization**: Requires `DARetriever` and `Genesis` parameters -2. **Configuration**: New fields in `DAConfig` and `NodeConfig` -3. **Syncer**: New verification step in block processing - -**Migration Path**: - -- Forced inclusion is optional (enabled when namespace configured) -- Existing deployments work without configuration changes -- Can enable incrementally per network +- **Grace Period Attacks**: + - Malicious sequencer cannot indefinitely delay forced transactions + - Grace boundary is deterministic and enforced by all full nodes + - Longer grace periods extend time to detect censorship (trade-off) ## Status @@ -742,26 +793,28 @@ Accepted and Implemented 4. **Based Rollup Option**: Fully DA-driven transaction ordering available (simplified implementation) 5. **Optional**: Forced inclusion can be disabled for permissioned deployments 6. **Efficient**: Epoch-based fetching minimizes DA queries -7. **Flexible**: Configurable epoch size allows tuning latency vs efficiency +7. **Flexible**: Configurable epoch size and grace period allow tuning latency vs reliability 8. **Robust Size Handling**: Two-tier size validation prevents DoS and DA rejections 9. **Transaction Preservation**: All valid transactions are preserved in queues, nothing is lost 10. **Strict MaxBytes Compliance**: Batches never exceed limits, preventing DA submission failures +11. **DA Fault Tolerance**: Grace period prevents false positives during temporary chain congestion ### Negative 1. **Increased Latency**: Forced transactions subject to epoch boundaries 2. **DA Dependency**: Requires DA layer to support multiple namespaces 3. **Higher DA Costs**: Users pay DA posting fees for forced inclusion -4. **Additional Complexity**: New component (DA Retriever) and verification logic +4. **Additional Complexity**: New component (DA Retriever) and verification logic with grace period tracking 5. **Epoch Configuration**: Requires setting `DAEpochForcedInclusion` in genesis (consensus parameter) +6. **Grace Period Adjustment**: Grace period is dynamically adjusted based on block fullness to balance censorship detection with operational reliability ### Neutral 1. **Two Sequencer Types**: Choice between single (hybrid) and based (DA-only) 2. **Privacy Model Unchanged**: Forced inclusion has same privacy as normal path -3. **Monitoring**: Operators should monitor forced inclusion namespace usage -4. **Documentation**: Users need guidance on when to use forced inclusion -5. **Genesis Parameter**: `DAEpochForcedInclusion` is a consensus parameter fixed at genesis +3. **Monitoring**: Operators should monitor forced inclusion namespace usage and grace period metrics +4. **Documentation**: Users need guidance on when to use forced inclusion and grace period implications +5. **Genesis Parameters**: `DAEpochForcedInclusion` is a consensus parameter fixed at genesis; grace period adjustment is dynamic ## References diff --git a/test/docker-e2e/resiliency_test.go b/test/docker-e2e/resiliency_test.go index 9f35b81c93..873f33a6db 100644 --- a/test/docker-e2e/resiliency_test.go +++ b/test/docker-e2e/resiliency_test.go @@ -5,12 +5,13 @@ package docker_e2e import ( "context" "fmt" - tastoratypes "github.com/celestiaorg/tastora/framework/types" "io" "strings" "testing" "time" + tastoratypes "github.com/celestiaorg/tastora/framework/types" + da "github.com/celestiaorg/tastora/framework/docker/dataavailability" "github.com/celestiaorg/tastora/framework/docker/evstack" "github.com/docker/docker/api/types/container"