From 229da68de4531ededab6122c6c6654d53bececbe Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Mon, 5 Jan 2026 17:24:22 +0100 Subject: [PATCH 01/18] feat(block): async epoch fetching --- apps/evm/cmd/run.go | 13 +- apps/grpc/cmd/run.go | 14 +- apps/testapp/cmd/run.go | 14 +- block/internal/da/async_epoch_fetcher.go | 514 ++++++++++++++++++ block/internal/da/async_epoch_fetcher_test.go | 314 +++++++++++ .../internal/da/forced_inclusion_retriever.go | 26 +- .../da/forced_inclusion_retriever_test.go | 46 +- block/internal/syncing/syncer.go | 14 +- .../syncing/syncer_forced_inclusion_test.go | 38 +- block/public.go | 28 +- pkg/sequencers/single/sequencer.go | 46 +- 11 files changed, 1035 insertions(+), 32 deletions(-) create mode 100644 block/internal/da/async_epoch_fetcher.go create mode 100644 block/internal/da/async_epoch_fetcher_test.go diff --git a/apps/evm/cmd/run.go b/apps/evm/cmd/run.go index dae8e9c3e..16699f18b 100644 --- a/apps/evm/cmd/run.go +++ b/apps/evm/cmd/run.go @@ -167,7 +167,18 @@ func createSequencer( return nil, fmt.Errorf("based sequencer mode requires aggregator mode to be enabled") } - fiRetriever := block.NewForcedInclusionRetriever(daClient, logger, genesis.DAStartHeight, genesis.DAEpochForcedInclusion) + // Create async epoch fetcher for background prefetching + asyncFetcher := block.NewAsyncEpochFetcher( + daClient, + logger, + genesis.DAStartHeight, + genesis.DAEpochForcedInclusion, + 2, // prefetch 2 epochs ahead for based sequencer + 1*time.Second, // check frequently + ) + asyncFetcher.Start() + + fiRetriever := block.NewForcedInclusionRetriever(daClient, logger, genesis.DAStartHeight, genesis.DAEpochForcedInclusion, asyncFetcher) basedSeq, err := based.NewBasedSequencer(fiRetriever, datastore, genesis, logger) if err != nil { return nil, fmt.Errorf("failed to create based sequencer: %w", err) diff --git a/apps/grpc/cmd/run.go b/apps/grpc/cmd/run.go index 021a3bb8f..1106926aa 100644 --- a/apps/grpc/cmd/run.go +++ b/apps/grpc/cmd/run.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "path/filepath" + "time" "github.com/ipfs/go-datastore" "github.com/rs/zerolog" @@ -127,7 +128,18 @@ func createSequencer( return nil, fmt.Errorf("based sequencer mode requires aggregator mode to be enabled") } - fiRetriever := block.NewForcedInclusionRetriever(daClient, logger, genesis.DAStartHeight, genesis.DAEpochForcedInclusion) + // Create async epoch fetcher for background prefetching + asyncFetcher := block.NewAsyncEpochFetcher( + daClient, + logger, + genesis.DAStartHeight, + genesis.DAEpochForcedInclusion, + 2, // prefetch 2 epochs ahead for based sequencer + 1*time.Second, // check frequently + ) + asyncFetcher.Start() + + fiRetriever := block.NewForcedInclusionRetriever(daClient, logger, genesis.DAStartHeight, genesis.DAEpochForcedInclusion, asyncFetcher) basedSeq, err := based.NewBasedSequencer(fiRetriever, datastore, genesis, logger) if err != nil { return nil, fmt.Errorf("failed to create based sequencer: %w", err) diff --git a/apps/testapp/cmd/run.go b/apps/testapp/cmd/run.go index 122ca9279..e4d52eb24 100644 --- a/apps/testapp/cmd/run.go +++ b/apps/testapp/cmd/run.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "path/filepath" + "time" "github.com/ipfs/go-datastore" "github.com/rs/zerolog" @@ -128,7 +129,18 @@ func createSequencer( return nil, fmt.Errorf("based sequencer mode requires aggregator mode to be enabled") } - fiRetriever := block.NewForcedInclusionRetriever(daClient, logger, genesis.DAStartHeight, genesis.DAEpochForcedInclusion) + // Create async epoch fetcher for background prefetching + asyncFetcher := block.NewAsyncEpochFetcher( + daClient, + logger, + genesis.DAStartHeight, + genesis.DAEpochForcedInclusion, + 2, // prefetch 2 epochs ahead for based sequencer + 1*time.Second, // check frequently + ) + asyncFetcher.Start() + + fiRetriever := block.NewForcedInclusionRetriever(daClient, logger, genesis.DAStartHeight, genesis.DAEpochForcedInclusion, asyncFetcher) basedSeq, err := based.NewBasedSequencer(fiRetriever, datastore, genesis, logger) if err != nil { return nil, fmt.Errorf("failed to create based sequencer: %w", err) diff --git a/block/internal/da/async_epoch_fetcher.go b/block/internal/da/async_epoch_fetcher.go new file mode 100644 index 000000000..a05ed342d --- /dev/null +++ b/block/internal/da/async_epoch_fetcher.go @@ -0,0 +1,514 @@ +package da + +import ( + "context" + "errors" + "fmt" + "sync" + "time" + + ds "github.com/ipfs/go-datastore" + dsq "github.com/ipfs/go-datastore/query" + dsync "github.com/ipfs/go-datastore/sync" + "github.com/rs/zerolog" + + datypes "github.com/evstack/ev-node/pkg/da/types" + "github.com/evstack/ev-node/types" +) + +// AsyncEpochFetcher handles background prefetching of DA epoch data +// to speed up processing at epoch boundaries. +type AsyncEpochFetcher struct { + client Client + logger zerolog.Logger + daEpochSize uint64 + daStartHeight uint64 + + // In-memory cache for prefetched epoch data + cache ds.Batching + mu sync.RWMutex + + // Background fetcher control + ctx context.Context + cancel context.CancelFunc + wg sync.WaitGroup + + // Current DA height tracking + currentDAHeight uint64 + heightMu sync.RWMutex + + // Prefetch window - how many epochs ahead to prefetch + prefetchWindow uint64 + + // Polling interval for checking new DA heights + pollInterval time.Duration +} + +// NewAsyncEpochFetcher creates a new async epoch fetcher with in-memory cache. +func NewAsyncEpochFetcher( + client Client, + logger zerolog.Logger, + daStartHeight, daEpochSize uint64, + prefetchWindow uint64, + pollInterval time.Duration, +) *AsyncEpochFetcher { + if prefetchWindow == 0 { + prefetchWindow = 1 // Default: prefetch next epoch + } + if pollInterval == 0 { + pollInterval = 2 * time.Second // Default polling interval + } + + ctx, cancel := context.WithCancel(context.Background()) + + return &AsyncEpochFetcher{ + client: client, + logger: logger.With().Str("component", "async_epoch_fetcher").Logger(), + daStartHeight: daStartHeight, + daEpochSize: daEpochSize, + cache: dsync.MutexWrap(ds.NewMapDatastore()), + ctx: ctx, + cancel: cancel, + currentDAHeight: daStartHeight, + prefetchWindow: prefetchWindow, + pollInterval: pollInterval, + } +} + +// Start begins the background prefetching process. +func (f *AsyncEpochFetcher) Start() { + f.wg.Add(1) + go f.backgroundFetchLoop() + f.logger.Info(). + Uint64("da_start_height", f.daStartHeight). + Uint64("da_epoch_size", f.daEpochSize). + Uint64("prefetch_window", f.prefetchWindow). + Dur("poll_interval", f.pollInterval). + Msg("async epoch fetcher started") +} + +// Stop gracefully stops the background prefetching process. +func (f *AsyncEpochFetcher) Stop() { + f.logger.Info().Msg("stopping async epoch fetcher") + f.cancel() + f.wg.Wait() + f.logger.Info().Msg("async epoch fetcher stopped") +} + +// SetDAHeight updates the current DA height being processed. +// This is called by sequencers to inform the fetcher of progress. +func (f *AsyncEpochFetcher) SetDAHeight(height uint64) { + f.heightMu.Lock() + defer f.heightMu.Unlock() + + if height > f.currentDAHeight { + f.currentDAHeight = height + f.logger.Debug().Uint64("da_height", height).Msg("updated current DA height") + } +} + +// GetDAHeight returns the current DA height. +func (f *AsyncEpochFetcher) GetDAHeight() uint64 { + f.heightMu.RLock() + defer f.heightMu.RUnlock() + return f.currentDAHeight +} + +// GetCachedEpoch retrieves a cached epoch from memory. +// Returns nil if the epoch is not cached. +func (f *AsyncEpochFetcher) GetCachedEpoch(ctx context.Context, daHeight uint64) (*ForcedInclusionEvent, error) { + if !f.client.HasForcedInclusionNamespace() { + return nil, ErrForceInclusionNotConfigured + } + + if daHeight < f.daStartHeight { + return nil, fmt.Errorf("DA height %d is before the configured start height %d", daHeight, f.daStartHeight) + } + + epochStart, epochEnd, _ := types.CalculateEpochBoundaries(daHeight, f.daStartHeight, f.daEpochSize) + + // Only return cached data at epoch end + if daHeight != epochEnd { + return &ForcedInclusionEvent{ + StartDaHeight: daHeight, + EndDaHeight: daHeight, + Txs: [][]byte{}, + }, nil + } + + // Try to get from cache + key := ds.NewKey(fmt.Sprintf("/epoch/%d", epochEnd)) + + f.mu.RLock() + data, err := f.cache.Get(ctx, key) + f.mu.RUnlock() + + if err != nil { + if errors.Is(err, ds.ErrNotFound) { + return nil, nil // Not cached yet + } + return nil, fmt.Errorf("failed to get cached epoch: %w", err) + } + + // Deserialize the cached event + event, err := deserializeForcedInclusionEvent(data) + if err != nil { + return nil, fmt.Errorf("failed to deserialize cached epoch: %w", err) + } + + f.logger.Debug(). + Uint64("da_height", daHeight). + Uint64("epoch_start", epochStart). + Uint64("epoch_end", epochEnd). + Int("tx_count", len(event.Txs)). + Msg("retrieved epoch from cache") + + return event, nil +} + +// backgroundFetchLoop runs in the background and prefetches epochs ahead of time. +func (f *AsyncEpochFetcher) backgroundFetchLoop() { + defer f.wg.Done() + + ticker := time.NewTicker(f.pollInterval) + defer ticker.Stop() + + for { + select { + case <-f.ctx.Done(): + return + case <-ticker.C: + f.prefetchEpochs() + } + } +} + +// prefetchEpochs prefetches epochs within the prefetch window. +func (f *AsyncEpochFetcher) prefetchEpochs() { + if !f.client.HasForcedInclusionNamespace() { + return + } + + currentHeight := f.GetDAHeight() + + // Calculate the current epoch and epochs to prefetch + _, currentEpochEnd, _ := types.CalculateEpochBoundaries(currentHeight, f.daStartHeight, f.daEpochSize) + + // Prefetch upcoming epochs + for i := uint64(0); i < f.prefetchWindow; i++ { + targetEpochEnd := currentEpochEnd + (i * f.daEpochSize) + + // Check if already cached + key := ds.NewKey(fmt.Sprintf("/epoch/%d", targetEpochEnd)) + f.mu.RLock() + _, err := f.cache.Get(f.ctx, key) + f.mu.RUnlock() + + if err == nil { + // Already cached + continue + } + + // Fetch and cache the epoch + f.fetchAndCacheEpoch(targetEpochEnd) + } + + // Clean up old epochs from cache to prevent memory growth + f.cleanupOldEpochs(currentEpochEnd) +} + +// fetchAndCacheEpoch fetches an epoch and stores it in the cache. +func (f *AsyncEpochFetcher) fetchAndCacheEpoch(epochEnd uint64) { + epochStart := epochEnd - (f.daEpochSize - 1) + if epochStart < f.daStartHeight { + epochStart = f.daStartHeight + } + + f.logger.Debug(). + Uint64("epoch_start", epochStart). + Uint64("epoch_end", epochEnd). + Msg("prefetching epoch") + + event := &ForcedInclusionEvent{ + StartDaHeight: epochStart, + EndDaHeight: epochEnd, + Txs: [][]byte{}, + } + + // Fetch epoch end first to check availability + epochEndResult := f.client.Retrieve(f.ctx, epochEnd, f.client.GetForcedInclusionNamespace()) + if epochEndResult.Code == datypes.StatusHeightFromFuture { + f.logger.Debug(). + Uint64("epoch_end", epochEnd). + Msg("epoch end height not yet available - will retry") + return + } + + epochStartResult := epochEndResult + if epochStart != epochEnd { + epochStartResult = f.client.Retrieve(f.ctx, epochStart, f.client.GetForcedInclusionNamespace()) + if epochStartResult.Code == datypes.StatusHeightFromFuture { + f.logger.Debug(). + Uint64("epoch_start", epochStart). + Msg("epoch start height not yet available - will retry") + return + } + } + + // Process all heights in the epoch + var processErrs error + err := f.processForcedInclusionBlobs(event, epochStartResult, epochStart) + processErrs = errors.Join(processErrs, err) + + // Process heights between start and end (exclusive) + for epochHeight := epochStart + 1; epochHeight < epochEnd; epochHeight++ { + result := f.client.Retrieve(f.ctx, epochHeight, f.client.GetForcedInclusionNamespace()) + err = f.processForcedInclusionBlobs(event, result, epochHeight) + processErrs = errors.Join(processErrs, err) + } + + // Process epoch end (only if different from start) + if epochEnd != epochStart { + err = f.processForcedInclusionBlobs(event, epochEndResult, epochEnd) + processErrs = errors.Join(processErrs, err) + } + + if processErrs != nil { + f.logger.Warn(). + Uint64("epoch_start", epochStart). + Uint64("epoch_end", epochEnd). + Err(processErrs). + Msg("failed to prefetch epoch - will retry") + return + } + + // Serialize and cache the event + data, err := serializeForcedInclusionEvent(event) + if err != nil { + f.logger.Error(). + Err(err). + Uint64("epoch_end", epochEnd). + Msg("failed to serialize epoch for caching") + return + } + + key := ds.NewKey(fmt.Sprintf("/epoch/%d", epochEnd)) + f.mu.Lock() + err = f.cache.Put(f.ctx, key, data) + f.mu.Unlock() + + if err != nil { + f.logger.Error(). + Err(err). + Uint64("epoch_end", epochEnd). + Msg("failed to cache epoch") + return + } + + f.logger.Info(). + Uint64("epoch_start", epochStart). + Uint64("epoch_end", epochEnd). + Int("tx_count", len(event.Txs)). + Msg("successfully prefetched and cached epoch") +} + +// processForcedInclusionBlobs processes blobs from a single DA height for forced inclusion. +func (f *AsyncEpochFetcher) processForcedInclusionBlobs( + event *ForcedInclusionEvent, + result datypes.ResultRetrieve, + height uint64, +) error { + if result.Code == datypes.StatusNotFound { + f.logger.Debug().Uint64("height", height).Msg("no forced inclusion blobs at height") + return nil + } + + if result.Code != datypes.StatusSuccess { + return fmt.Errorf("failed to retrieve forced inclusion blobs at height %d: %s", height, result.Message) + } + + // Process each blob as a transaction + for _, blob := range result.Data { + if len(blob) > 0 { + event.Txs = append(event.Txs, blob) + } + } + + if result.Timestamp.After(event.Timestamp) { + event.Timestamp = result.Timestamp + } + + f.logger.Debug(). + Uint64("height", height). + Int("blob_count", len(result.Data)). + Msg("processed forced inclusion blobs for prefetch") + + return nil +} + +// cleanupOldEpochs removes epochs older than the current epoch from cache. +func (f *AsyncEpochFetcher) cleanupOldEpochs(currentEpochEnd uint64) { + // Remove epochs older than current - 1 + // Keep current and previous in case of reorgs or restarts + cleanupThreshold := currentEpochEnd - f.daEpochSize + + f.mu.Lock() + defer f.mu.Unlock() + + // Query all keys + query := dsq.Query{Prefix: "/epoch/"} + results, err := f.cache.Query(f.ctx, query) + if err != nil { + f.logger.Warn().Err(err).Msg("failed to query cache for cleanup") + return + } + defer results.Close() + + for result := range results.Next() { + if result.Error != nil { + continue + } + + key := ds.NewKey(result.Key) + // Extract epoch end from key + var epochEnd uint64 + _, err := fmt.Sscanf(key.String(), "/epoch/%d", &epochEnd) + if err != nil { + continue + } + + if epochEnd < cleanupThreshold { + if err := f.cache.Delete(f.ctx, key); err != nil { + f.logger.Warn(). + Err(err). + Uint64("epoch_end", epochEnd). + Msg("failed to delete old epoch from cache") + } else { + f.logger.Debug(). + Uint64("epoch_end", epochEnd). + Msg("cleaned up old epoch from cache") + } + } + } +} + +// serializeForcedInclusionEvent serializes an event to bytes. +// Simple format: timestamp (int64) + startHeight (uint64) + endHeight (uint64) + txCount (uint32) + txs +func serializeForcedInclusionEvent(event *ForcedInclusionEvent) ([]byte, error) { + // Calculate total size + size := 8 + 8 + 8 + 4 // timestamp + startHeight + endHeight + txCount + for _, tx := range event.Txs { + size += 4 + len(tx) // txLen + tx + } + + buf := make([]byte, size) + offset := 0 + + // Timestamp + writeUint64(buf[offset:], uint64(event.Timestamp.Unix())) + offset += 8 + + // StartDaHeight + writeUint64(buf[offset:], event.StartDaHeight) + offset += 8 + + // EndDaHeight + writeUint64(buf[offset:], event.EndDaHeight) + offset += 8 + + // TxCount + writeUint32(buf[offset:], uint32(len(event.Txs))) + offset += 4 + + // Txs + for _, tx := range event.Txs { + writeUint32(buf[offset:], uint32(len(tx))) + offset += 4 + copy(buf[offset:], tx) + offset += len(tx) + } + + return buf, nil +} + +// deserializeForcedInclusionEvent deserializes bytes to an event. +func deserializeForcedInclusionEvent(data []byte) (*ForcedInclusionEvent, error) { + if len(data) < 28 { + return nil, errors.New("invalid data: too short") + } + + offset := 0 + event := &ForcedInclusionEvent{} + + // Timestamp + timestamp := readUint64(data[offset:]) + event.Timestamp = time.Unix(int64(timestamp), 0).UTC() + offset += 8 + + // StartDaHeight + event.StartDaHeight = readUint64(data[offset:]) + offset += 8 + + // EndDaHeight + event.EndDaHeight = readUint64(data[offset:]) + offset += 8 + + // TxCount + txCount := readUint32(data[offset:]) + offset += 4 + + // Txs + event.Txs = make([][]byte, txCount) + for i := uint32(0); i < txCount; i++ { + if offset+4 > len(data) { + return nil, errors.New("invalid data: unexpected end while reading tx length") + } + txLen := readUint32(data[offset:]) + offset += 4 + + if offset+int(txLen) > len(data) { + return nil, errors.New("invalid data: unexpected end while reading tx") + } + event.Txs[i] = make([]byte, txLen) + copy(event.Txs[i], data[offset:offset+int(txLen)]) + offset += int(txLen) + } + + return event, nil +} + +func writeUint64(buf []byte, val uint64) { + buf[0] = byte(val) + buf[1] = byte(val >> 8) + buf[2] = byte(val >> 16) + buf[3] = byte(val >> 24) + buf[4] = byte(val >> 32) + buf[5] = byte(val >> 40) + buf[6] = byte(val >> 48) + buf[7] = byte(val >> 56) +} + +func readUint64(buf []byte) uint64 { + return uint64(buf[0]) | + uint64(buf[1])<<8 | + uint64(buf[2])<<16 | + uint64(buf[3])<<24 | + uint64(buf[4])<<32 | + uint64(buf[5])<<40 | + uint64(buf[6])<<48 | + uint64(buf[7])<<56 +} + +func writeUint32(buf []byte, val uint32) { + buf[0] = byte(val) + buf[1] = byte(val >> 8) + buf[2] = byte(val >> 16) + buf[3] = byte(val >> 24) +} + +func readUint32(buf []byte) uint32 { + return uint32(buf[0]) | + uint32(buf[1])<<8 | + uint32(buf[2])<<16 | + uint32(buf[3])<<24 +} diff --git a/block/internal/da/async_epoch_fetcher_test.go b/block/internal/da/async_epoch_fetcher_test.go new file mode 100644 index 000000000..07e561e14 --- /dev/null +++ b/block/internal/da/async_epoch_fetcher_test.go @@ -0,0 +1,314 @@ +package da + +import ( + "context" + "testing" + "time" + + "github.com/rs/zerolog" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + + datypes "github.com/evstack/ev-node/pkg/da/types" +) + +// MockClient is a mock implementation of the Client interface +type MockClient struct { + mock.Mock +} + +func (m *MockClient) Submit(ctx context.Context, data [][]byte, gasPrice float64, namespace []byte, options []byte) datypes.ResultSubmit { + args := m.Called(ctx, data, gasPrice, namespace, options) + return args.Get(0).(datypes.ResultSubmit) +} + +func (m *MockClient) Retrieve(ctx context.Context, height uint64, namespace []byte) datypes.ResultRetrieve { + args := m.Called(ctx, height, namespace) + return args.Get(0).(datypes.ResultRetrieve) +} + +func (m *MockClient) Get(ctx context.Context, ids []datypes.ID, namespace []byte) ([]datypes.Blob, error) { + args := m.Called(ctx, ids, namespace) + return args.Get(0).([]datypes.Blob), args.Error(1) +} + +func (m *MockClient) GetHeaderNamespace() []byte { + args := m.Called() + return args.Get(0).([]byte) +} + +func (m *MockClient) GetDataNamespace() []byte { + args := m.Called() + return args.Get(0).([]byte) +} + +func (m *MockClient) GetForcedInclusionNamespace() []byte { + args := m.Called() + return args.Get(0).([]byte) +} + +func (m *MockClient) HasForcedInclusionNamespace() bool { + args := m.Called() + return args.Bool(0) +} + +func TestAsyncEpochFetcher_Creation(t *testing.T) { + client := &MockClient{} + logger := zerolog.Nop() + + fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 2, 100*time.Millisecond) + assert.NotNil(t, fetcher) + assert.Equal(t, uint64(100), fetcher.daStartHeight) + assert.Equal(t, uint64(10), fetcher.daEpochSize) + assert.Equal(t, uint64(2), fetcher.prefetchWindow) + assert.Equal(t, 100*time.Millisecond, fetcher.pollInterval) +} + +func TestAsyncEpochFetcher_SetAndGetDAHeight(t *testing.T) { + client := &MockClient{} + logger := zerolog.Nop() + + fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, time.Second) + + assert.Equal(t, uint64(100), fetcher.GetDAHeight()) + + fetcher.SetDAHeight(150) + assert.Equal(t, uint64(150), fetcher.GetDAHeight()) + + // Should not decrease + fetcher.SetDAHeight(120) + assert.Equal(t, uint64(150), fetcher.GetDAHeight()) +} + +func TestAsyncEpochFetcher_GetCachedEpoch_NotAtEpochEnd(t *testing.T) { + client := &MockClient{} + fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() + client.On("HasForcedInclusionNamespace").Return(true) + client.On("GetForcedInclusionNamespace").Return(fiNs) + + logger := zerolog.Nop() + fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, time.Second) + + ctx := context.Background() + + // Height 105 is not an epoch end (100, 109, 118, etc. are epoch ends for size 10) + event, err := fetcher.GetCachedEpoch(ctx, 105) + require.NoError(t, err) + assert.NotNil(t, event) + assert.Equal(t, 0, len(event.Txs)) + assert.Equal(t, uint64(105), event.StartDaHeight) + assert.Equal(t, uint64(105), event.EndDaHeight) +} + +func TestAsyncEpochFetcher_GetCachedEpoch_NoNamespace(t *testing.T) { + client := &MockClient{} + client.On("HasForcedInclusionNamespace").Return(false) + + logger := zerolog.Nop() + fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, time.Second) + + ctx := context.Background() + _, err := fetcher.GetCachedEpoch(ctx, 109) + assert.ErrorIs(t, err, ErrForceInclusionNotConfigured) +} + +func TestAsyncEpochFetcher_GetCachedEpoch_CacheMiss(t *testing.T) { + client := &MockClient{} + fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() + client.On("HasForcedInclusionNamespace").Return(true) + client.On("GetForcedInclusionNamespace").Return(fiNs) + + logger := zerolog.Nop() + fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, time.Second) + + ctx := context.Background() + + // Epoch end at 109, but nothing cached + event, err := fetcher.GetCachedEpoch(ctx, 109) + require.NoError(t, err) + assert.Nil(t, event) // Cache miss +} + +func TestAsyncEpochFetcher_FetchAndCache(t *testing.T) { + testBlobs := [][]byte{ + []byte("tx1"), + []byte("tx2"), + []byte("tx3"), + } + + client := &MockClient{} + fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() + client.On("HasForcedInclusionNamespace").Return(true) + client.On("GetForcedInclusionNamespace").Return(fiNs) + + // Mock Retrieve calls for epoch [100, 109] + for height := uint64(100); height <= 109; height++ { + if height == 100 { + client.On("Retrieve", mock.Anything, height, fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{ + Code: datypes.StatusSuccess, + Timestamp: time.Unix(1000, 0), + }, + Data: testBlobs, + }).Once() + } else { + client.On("Retrieve", mock.Anything, height, fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{Code: datypes.StatusNotFound}, + }).Once() + } + } + + logger := zerolog.Nop() + fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, time.Second) + + // Manually trigger fetch + fetcher.fetchAndCacheEpoch(109) + + // Now try to get from cache + ctx := context.Background() + event, err := fetcher.GetCachedEpoch(ctx, 109) + require.NoError(t, err) + assert.NotNil(t, event) + assert.Equal(t, 3, len(event.Txs)) + assert.Equal(t, testBlobs[0], event.Txs[0]) + assert.Equal(t, uint64(100), event.StartDaHeight) + assert.Equal(t, uint64(109), event.EndDaHeight) +} + +func TestAsyncEpochFetcher_BackgroundPrefetch(t *testing.T) { + testBlobs := [][]byte{ + []byte("tx1"), + []byte("tx2"), + } + + client := &MockClient{} + fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() + client.On("HasForcedInclusionNamespace").Return(true) + client.On("GetForcedInclusionNamespace").Return(fiNs) + + // Mock for current epoch [100, 109] + for height := uint64(100); height <= 109; height++ { + if height == 105 { + client.On("Retrieve", mock.Anything, height, fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{ + Code: datypes.StatusSuccess, + Timestamp: time.Unix(2000, 0), + }, + Data: testBlobs, + }).Maybe() + } else { + client.On("Retrieve", mock.Anything, height, fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{Code: datypes.StatusNotFound}, + }).Maybe() + } + } + + // Mock for next epoch [110, 119] + for height := uint64(110); height <= 119; height++ { + client.On("Retrieve", mock.Anything, height, fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{Code: datypes.StatusNotFound}, + }).Maybe() + } + + logger := zerolog.Nop() + fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, 50*time.Millisecond) + + fetcher.SetDAHeight(100) + fetcher.Start() + defer fetcher.Stop() + + // Wait for background prefetch to happen + time.Sleep(200 * time.Millisecond) + + // Check if epoch was prefetched + ctx := context.Background() + event, err := fetcher.GetCachedEpoch(ctx, 109) + require.NoError(t, err) + assert.NotNil(t, event) + assert.Equal(t, 2, len(event.Txs)) +} + +func TestAsyncEpochFetcher_Serialization(t *testing.T) { + event := &ForcedInclusionEvent{ + Timestamp: time.Unix(12345, 0).UTC(), + StartDaHeight: 100, + EndDaHeight: 109, + Txs: [][]byte{ + []byte("transaction1"), + []byte("tx2"), + []byte("another_transaction"), + }, + } + + // Serialize + data, err := serializeForcedInclusionEvent(event) + require.NoError(t, err) + assert.Greater(t, len(data), 0) + + // Deserialize + decoded, err := deserializeForcedInclusionEvent(data) + require.NoError(t, err) + assert.Equal(t, event.Timestamp.Unix(), decoded.Timestamp.Unix()) + assert.Equal(t, event.StartDaHeight, decoded.StartDaHeight) + assert.Equal(t, event.EndDaHeight, decoded.EndDaHeight) + assert.Equal(t, len(event.Txs), len(decoded.Txs)) + for i := range event.Txs { + assert.Equal(t, event.Txs[i], decoded.Txs[i]) + } +} + +func TestAsyncEpochFetcher_SerializationEmpty(t *testing.T) { + event := &ForcedInclusionEvent{ + Timestamp: time.Unix(0, 0).UTC(), + StartDaHeight: 100, + EndDaHeight: 100, + Txs: [][]byte{}, + } + + data, err := serializeForcedInclusionEvent(event) + require.NoError(t, err) + + decoded, err := deserializeForcedInclusionEvent(data) + require.NoError(t, err) + assert.Equal(t, uint64(100), decoded.StartDaHeight) + assert.Equal(t, 0, len(decoded.Txs)) +} + +func TestAsyncEpochFetcher_HeightFromFuture(t *testing.T) { + client := &MockClient{} + fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() + client.On("HasForcedInclusionNamespace").Return(true) + client.On("GetForcedInclusionNamespace").Return(fiNs) + + // Epoch end not available yet + client.On("Retrieve", mock.Anything, uint64(109), fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{Code: datypes.StatusHeightFromFuture}, + }).Once() + + logger := zerolog.Nop() + fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, time.Second) + + // This should not panic and should handle gracefully + fetcher.fetchAndCacheEpoch(109) + + // Cache should be empty + ctx := context.Background() + event, err := fetcher.GetCachedEpoch(ctx, 109) + require.NoError(t, err) + assert.Nil(t, event) +} + +func TestAsyncEpochFetcher_StopGracefully(t *testing.T) { + client := &MockClient{} + client.On("HasForcedInclusionNamespace").Return(false) + + logger := zerolog.Nop() + fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, 50*time.Millisecond) + + fetcher.Start() + time.Sleep(100 * time.Millisecond) + + // Should stop gracefully without panic + fetcher.Stop() +} diff --git a/block/internal/da/forced_inclusion_retriever.go b/block/internal/da/forced_inclusion_retriever.go index 651c071a3..306ab632b 100644 --- a/block/internal/da/forced_inclusion_retriever.go +++ b/block/internal/da/forced_inclusion_retriever.go @@ -21,6 +21,7 @@ type ForcedInclusionRetriever struct { logger zerolog.Logger daEpochSize uint64 daStartHeight uint64 + asyncFetcher *AsyncEpochFetcher // Required for async prefetching } // ForcedInclusionEvent contains forced inclusion transactions retrieved from DA. @@ -32,21 +33,25 @@ type ForcedInclusionEvent struct { } // NewForcedInclusionRetriever creates a new forced inclusion retriever. +// The asyncFetcher parameter is required for background prefetching of DA epoch data. func NewForcedInclusionRetriever( client Client, logger zerolog.Logger, daStartHeight, daEpochSize uint64, + asyncFetcher *AsyncEpochFetcher, ) *ForcedInclusionRetriever { return &ForcedInclusionRetriever{ client: client, logger: logger.With().Str("component", "forced_inclusion_retriever").Logger(), daStartHeight: daStartHeight, daEpochSize: daEpochSize, + asyncFetcher: asyncFetcher, } } // RetrieveForcedIncludedTxs retrieves forced inclusion transactions at the given DA height. // It respects epoch boundaries and only fetches at epoch start. +// If an async fetcher is configured, it will try to use cached data first for better performance. func (r *ForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Context, daHeight uint64) (*ForcedInclusionEvent, error) { // when daStartHeight is not set or no namespace is configured, we retrieve nothing. if !r.client.HasForcedInclusionNamespace() { @@ -72,6 +77,25 @@ func (r *ForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Context }, nil } + // Try to get from async fetcher cache first + cachedEvent, err := r.asyncFetcher.GetCachedEpoch(ctx, daHeight) + if err == nil && cachedEvent != nil { + r.logger.Debug(). + Uint64("da_height", daHeight). + Uint64("epoch_start", epochStart). + Uint64("epoch_end", epochEnd). + Int("tx_count", len(cachedEvent.Txs)). + Msg("using cached epoch data from async fetcher") + return cachedEvent, nil + } + // Cache miss or error, fall through to sync fetch + if err != nil { + r.logger.Debug(). + Err(err). + Uint64("da_height", daHeight). + Msg("failed to get cached epoch, falling back to sync fetch") + } + event := &ForcedInclusionEvent{ StartDaHeight: epochStart, EndDaHeight: epochEnd, @@ -105,7 +129,7 @@ func (r *ForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Context Msg("retrieving forced included transactions from DA") var processErrs error - err := r.processForcedInclusionBlobs(event, epochStartResult, epochStart) + err = r.processForcedInclusionBlobs(event, epochStartResult, epochStart) processErrs = errors.Join(processErrs, err) // Process heights between start and end (exclusive) diff --git a/block/internal/da/forced_inclusion_retriever_test.go b/block/internal/da/forced_inclusion_retriever_test.go index 6f35d0783..a83a22f3a 100644 --- a/block/internal/da/forced_inclusion_retriever_test.go +++ b/block/internal/da/forced_inclusion_retriever_test.go @@ -14,6 +14,18 @@ import ( "github.com/evstack/ev-node/test/mocks" ) +// createTestAsyncFetcher creates a minimal async fetcher for tests (without starting it) +func createTestAsyncFetcher(client Client, gen genesis.Genesis) *AsyncEpochFetcher { + return NewAsyncEpochFetcher( + client, + zerolog.Nop(), + gen.DAStartHeight, + gen.DAEpochForcedInclusion, + 1, // prefetch 1 epoch + 1*time.Second, // poll interval (doesn't matter for tests) + ) +} + func TestNewForcedInclusionRetriever(t *testing.T) { client := mocks.NewMockClient(t) client.On("HasForcedInclusionNamespace").Return(true).Maybe() @@ -24,7 +36,8 @@ func TestNewForcedInclusionRetriever(t *testing.T) { DAEpochForcedInclusion: 10, } - retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + asyncFetcher := createTestAsyncFetcher(client, gen) + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) assert.Assert(t, retriever != nil) } @@ -37,7 +50,8 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_NoNamespace(t *testi DAEpochForcedInclusion: 10, } - retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + asyncFetcher := createTestAsyncFetcher(client, gen) + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) ctx := context.Background() _, err := retriever.RetrieveForcedIncludedTxs(ctx, 100) @@ -56,7 +70,8 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_NotAtEpochStart(t *t DAEpochForcedInclusion: 10, } - retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + asyncFetcher := createTestAsyncFetcher(client, gen) + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) ctx := context.Background() // Height 105 is not an epoch start (100, 110, 120, etc. are epoch starts) @@ -77,19 +92,20 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_EpochStartSuccess(t client := mocks.NewMockClient(t) fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() - client.On("HasForcedInclusionNamespace").Return(true).Once() + client.On("HasForcedInclusionNamespace").Return(true).Maybe() client.On("GetForcedInclusionNamespace").Return(fiNs).Maybe() client.On("Retrieve", mock.Anything, mock.Anything, fiNs).Return(datypes.ResultRetrieve{ BaseResult: datypes.BaseResult{Code: datypes.StatusSuccess, IDs: []datypes.ID{[]byte("id1"), []byte("id2"), []byte("id3")}, Timestamp: time.Now()}, Data: testBlobs, - }).Once() + }).Maybe() gen := genesis.Genesis{ DAStartHeight: 100, DAEpochForcedInclusion: 1, // Single height epoch } - retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + asyncFetcher := createTestAsyncFetcher(client, gen) + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) ctx := context.Background() // Height 100 is an epoch start @@ -105,7 +121,7 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_EpochStartSuccess(t func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_EpochStartNotAvailable(t *testing.T) { client := mocks.NewMockClient(t) fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() - client.On("HasForcedInclusionNamespace").Return(true).Once() + client.On("HasForcedInclusionNamespace").Return(true).Maybe() client.On("GetForcedInclusionNamespace").Return(fiNs).Maybe() client.On("Retrieve", mock.Anything, uint64(109), fiNs).Return(datypes.ResultRetrieve{ BaseResult: datypes.BaseResult{Code: datypes.StatusHeightFromFuture}, @@ -116,7 +132,8 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_EpochStartNotAvailab DAEpochForcedInclusion: 10, } - retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + asyncFetcher := createTestAsyncFetcher(client, gen) + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) ctx := context.Background() // Epoch boundaries: [100, 109] - retrieval happens at epoch end (109) @@ -128,7 +145,7 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_EpochStartNotAvailab func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_NoBlobsAtHeight(t *testing.T) { client := mocks.NewMockClient(t) fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() - client.On("HasForcedInclusionNamespace").Return(true).Once() + client.On("HasForcedInclusionNamespace").Return(true).Maybe() client.On("GetForcedInclusionNamespace").Return(fiNs).Maybe() client.On("Retrieve", mock.Anything, uint64(100), fiNs).Return(datypes.ResultRetrieve{ BaseResult: datypes.BaseResult{Code: datypes.StatusNotFound}, @@ -139,7 +156,8 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_NoBlobsAtHeight(t *t DAEpochForcedInclusion: 1, // Single height epoch } - retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + asyncFetcher := createTestAsyncFetcher(client, gen) + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) ctx := context.Background() event, err := retriever.RetrieveForcedIncludedTxs(ctx, 100) @@ -157,7 +175,7 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_MultiHeightEpoch(t * client := mocks.NewMockClient(t) fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() - client.On("HasForcedInclusionNamespace").Return(true).Once() + client.On("HasForcedInclusionNamespace").Return(true).Maybe() client.On("GetForcedInclusionNamespace").Return(fiNs).Maybe() client.On("Retrieve", mock.Anything, uint64(102), fiNs).Return(datypes.ResultRetrieve{ BaseResult: datypes.BaseResult{Code: datypes.StatusSuccess, Timestamp: time.Now()}, @@ -177,7 +195,8 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_MultiHeightEpoch(t * DAEpochForcedInclusion: 3, // Epoch: 100-102 } - retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + asyncFetcher := createTestAsyncFetcher(client, gen) + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) ctx := context.Background() // Epoch boundaries: [100, 102] - retrieval happens at epoch end (102) @@ -201,7 +220,8 @@ func TestForcedInclusionRetriever_processForcedInclusionBlobs(t *testing.T) { DAEpochForcedInclusion: 10, } - retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + asyncFetcher := createTestAsyncFetcher(client, gen) + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) tests := []struct { name string diff --git a/block/internal/syncing/syncer.go b/block/internal/syncing/syncer.go index 266bc55e4..265ff15be 100644 --- a/block/internal/syncing/syncer.go +++ b/block/internal/syncing/syncer.go @@ -187,7 +187,19 @@ func (s *Syncer) Start(ctx context.Context) error { // Initialize handlers s.daRetriever = NewDARetriever(s.daClient, s.cache, s.genesis, s.logger) - s.fiRetriever = da.NewForcedInclusionRetriever(s.daClient, s.logger, s.genesis.DAStartHeight, s.genesis.DAEpochForcedInclusion) + + // Create async epoch fetcher for background prefetching + asyncFetcher := da.NewAsyncEpochFetcher( + s.daClient, + s.logger, + s.genesis.DAStartHeight, + s.genesis.DAEpochForcedInclusion, + 1, // prefetch 1 epoch ahead + 2*time.Second, // check every 2 seconds + ) + asyncFetcher.Start() + + s.fiRetriever = da.NewForcedInclusionRetriever(s.daClient, s.logger, s.genesis.DAStartHeight, s.genesis.DAEpochForcedInclusion, asyncFetcher) s.p2pHandler = NewP2PHandler(s.headerStore.Store(), s.dataStore.Store(), s.cache, s.genesis, s.logger) if currentHeight, err := s.store.Height(s.ctx); err != nil { s.logger.Error().Err(err).Msg("failed to set initial processed height for p2p handler") diff --git a/block/internal/syncing/syncer_forced_inclusion_test.go b/block/internal/syncing/syncer_forced_inclusion_test.go index 8b0622a42..29df5e0ad 100644 --- a/block/internal/syncing/syncer_forced_inclusion_test.go +++ b/block/internal/syncing/syncer_forced_inclusion_test.go @@ -14,7 +14,7 @@ import ( "github.com/evstack/ev-node/block/internal/cache" "github.com/evstack/ev-node/block/internal/common" - da "github.com/evstack/ev-node/block/internal/da" + "github.com/evstack/ev-node/block/internal/da" "github.com/evstack/ev-node/pkg/config" datypes "github.com/evstack/ev-node/pkg/da/types" "github.com/evstack/ev-node/pkg/genesis" @@ -23,6 +23,18 @@ import ( "github.com/evstack/ev-node/types" ) +// createTestAsyncFetcher creates a minimal async fetcher for tests (without starting it) +func createTestAsyncFetcherForSyncer(client da.Client, gen genesis.Genesis) *da.AsyncEpochFetcher { + return da.NewAsyncEpochFetcher( + client, + zerolog.Nop(), + gen.DAStartHeight, + gen.DAEpochForcedInclusion, + 1, // prefetch 1 epoch + 1*time.Second, // poll interval (doesn't matter for tests) + ) +} + func TestCalculateBlockFullness_HalfFull(t *testing.T) { s := &Syncer{} @@ -370,7 +382,8 @@ func TestVerifyForcedInclusionTxs_AllTransactionsIncluded(t *testing.T) { client.On("GetForcedInclusionNamespace").Return([]byte(cfg.DA.ForcedInclusionNamespace)).Maybe() client.On("HasForcedInclusionNamespace").Return(true).Maybe() daRetriever := NewDARetriever(client, cm, gen, zerolog.Nop()) - fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + asyncFetcher := createTestAsyncFetcherForSyncer(client, gen) + fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) s := NewSyncer( st, @@ -443,7 +456,8 @@ func TestVerifyForcedInclusionTxs_MissingTransactions(t *testing.T) { client.On("GetForcedInclusionNamespace").Return([]byte(cfg.DA.ForcedInclusionNamespace)).Maybe() client.On("HasForcedInclusionNamespace").Return(true).Maybe() daRetriever := NewDARetriever(client, cm, gen, zerolog.Nop()) - fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + asyncFetcher := createTestAsyncFetcherForSyncer(client, gen) + fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) s := NewSyncer( st, @@ -546,7 +560,8 @@ func TestVerifyForcedInclusionTxs_PartiallyIncluded(t *testing.T) { client.On("GetForcedInclusionNamespace").Return([]byte(cfg.DA.ForcedInclusionNamespace)).Maybe() client.On("HasForcedInclusionNamespace").Return(true).Maybe() daRetriever := NewDARetriever(client, cm, gen, zerolog.Nop()) - fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + asyncFetcher := createTestAsyncFetcherForSyncer(client, gen) + fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) s := NewSyncer( st, @@ -653,7 +668,8 @@ func TestVerifyForcedInclusionTxs_NoForcedTransactions(t *testing.T) { client.On("GetForcedInclusionNamespace").Return([]byte(cfg.DA.ForcedInclusionNamespace)).Maybe() client.On("HasForcedInclusionNamespace").Return(true).Maybe() daRetriever := NewDARetriever(client, cm, gen, zerolog.Nop()) - fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + asyncFetcher := createTestAsyncFetcherForSyncer(client, gen) + fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) s := NewSyncer( st, @@ -719,7 +735,8 @@ func TestVerifyForcedInclusionTxs_NamespaceNotConfigured(t *testing.T) { client.On("GetForcedInclusionNamespace").Return([]byte(nil)).Maybe() client.On("HasForcedInclusionNamespace").Return(false).Maybe() daRetriever := NewDARetriever(client, cm, gen, zerolog.Nop()) - fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + asyncFetcher := createTestAsyncFetcherForSyncer(client, gen) + fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) s := NewSyncer( st, @@ -784,7 +801,8 @@ func TestVerifyForcedInclusionTxs_DeferralWithinEpoch(t *testing.T) { client.On("GetForcedInclusionNamespace").Return([]byte(cfg.DA.ForcedInclusionNamespace)).Maybe() client.On("HasForcedInclusionNamespace").Return(true).Maybe() daRetriever := NewDARetriever(client, cm, gen, zerolog.Nop()) - fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + asyncFetcher := createTestAsyncFetcherForSyncer(client, gen) + fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) s := NewSyncer( st, @@ -907,7 +925,8 @@ func TestVerifyForcedInclusionTxs_MaliciousAfterEpochEnd(t *testing.T) { client.On("GetForcedInclusionNamespace").Return([]byte(cfg.DA.ForcedInclusionNamespace)).Maybe() client.On("HasForcedInclusionNamespace").Return(true).Maybe() daRetriever := NewDARetriever(client, cm, gen, zerolog.Nop()) - fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + asyncFetcher := createTestAsyncFetcherForSyncer(client, gen) + fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) s := NewSyncer( st, @@ -996,7 +1015,8 @@ func TestVerifyForcedInclusionTxs_SmoothingExceedsEpoch(t *testing.T) { client.On("HasForcedInclusionNamespace").Return(true).Maybe() daRetriever := NewDARetriever(client, cm, gen, zerolog.Nop()) - fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + asyncFetcher := createTestAsyncFetcherForSyncer(client, gen) + fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) s := NewSyncer( st, diff --git a/block/public.go b/block/public.go index 61a1e068a..4350b4eee 100644 --- a/block/public.go +++ b/block/public.go @@ -2,6 +2,7 @@ package block import ( "context" + "time" "github.com/evstack/ev-node/block/internal/common" "github.com/evstack/ev-node/block/internal/da" @@ -65,16 +66,39 @@ var ErrForceInclusionNotConfigured = da.ErrForceInclusionNotConfigured // ForcedInclusionEvent represents forced inclusion transactions retrieved from DA type ForcedInclusionEvent = da.ForcedInclusionEvent +// AsyncEpochFetcher provides background prefetching of DA epoch data +type AsyncEpochFetcher = da.AsyncEpochFetcher + // ForcedInclusionRetriever defines the interface for retrieving forced inclusion transactions from DA type ForcedInclusionRetriever interface { RetrieveForcedIncludedTxs(ctx context.Context, daHeight uint64) (*da.ForcedInclusionEvent, error) } -// NewForcedInclusionRetriever creates a new forced inclusion retriever +// NewAsyncEpochFetcher creates a new async epoch fetcher for background prefetching. +// Parameters: +// - client: DA client for fetching data +// - logger: structured logger +// - daStartHeight: genesis DA start height +// - daEpochSize: number of DA blocks per epoch +// - prefetchWindow: how many epochs ahead to prefetch (1-2 recommended) +// - pollInterval: how often to check for new epochs to prefetch +func NewAsyncEpochFetcher( + client DAClient, + logger zerolog.Logger, + daStartHeight, daEpochSize uint64, + prefetchWindow uint64, + pollInterval time.Duration, +) *AsyncEpochFetcher { + return da.NewAsyncEpochFetcher(client, logger, daStartHeight, daEpochSize, prefetchWindow, pollInterval) +} + +// NewForcedInclusionRetriever creates a new forced inclusion retriever. +// The asyncFetcher parameter is required for background prefetching of DA epoch data. func NewForcedInclusionRetriever( client DAClient, logger zerolog.Logger, daStartHeight, daEpochSize uint64, + asyncFetcher *AsyncEpochFetcher, ) ForcedInclusionRetriever { - return da.NewForcedInclusionRetriever(client, logger, daStartHeight, daEpochSize) + return da.NewForcedInclusionRetriever(client, logger, daStartHeight, daEpochSize, asyncFetcher) } diff --git a/pkg/sequencers/single/sequencer.go b/pkg/sequencers/single/sequencer.go index 45baf17cb..19d840130 100644 --- a/pkg/sequencers/single/sequencer.go +++ b/pkg/sequencers/single/sequencer.go @@ -40,6 +40,7 @@ type Sequencer struct { queue *BatchQueue // single queue for immediate availability // Forced inclusion support + asyncFetcher *block.AsyncEpochFetcher fiRetriever block.ForcedInclusionRetriever daHeight atomic.Uint64 daStartHeight atomic.Uint64 @@ -91,7 +92,18 @@ func NewSequencer( TxIndex: 0, } - s.fiRetriever = block.NewForcedInclusionRetriever(daClient, logger, s.GetDAHeight(), genesis.DAEpochForcedInclusion) + // Create async epoch fetcher for background prefetching + s.asyncFetcher = block.NewAsyncEpochFetcher( + daClient, + logger, + s.GetDAHeight(), + genesis.DAEpochForcedInclusion, + 1, // prefetch 1 epoch ahead + 3*time.Second, // check every 3 seconds + ) + s.asyncFetcher.Start() + + s.fiRetriever = block.NewForcedInclusionRetriever(daClient, logger, s.GetDAHeight(), genesis.DAEpochForcedInclusion, s.asyncFetcher) } else { return nil, fmt.Errorf("failed to load checkpoint from DB: %w", err) } @@ -106,7 +118,20 @@ func NewSequencer( Msg("resuming from checkpoint within DA epoch") } - s.fiRetriever = block.NewForcedInclusionRetriever(daClient, logger, s.getInitialDAStartHeight(context.Background()), genesis.DAEpochForcedInclusion) + initialDAHeight := s.getInitialDAStartHeight(context.Background()) + + // Create async epoch fetcher for background prefetching + s.asyncFetcher = block.NewAsyncEpochFetcher( + daClient, + logger, + initialDAHeight, + genesis.DAEpochForcedInclusion, + 1, // prefetch 1 epoch ahead + 3*time.Second, // check every 3 seconds + ) + s.asyncFetcher.Start() + + s.fiRetriever = block.NewForcedInclusionRetriever(daClient, logger, initialDAHeight, genesis.DAEpochForcedInclusion, s.asyncFetcher) } return s, nil @@ -175,7 +200,22 @@ func (c *Sequencer) GetNextBatch(ctx context.Context, req coresequencer.GetNextB TxIndex: 0, } - c.fiRetriever = block.NewForcedInclusionRetriever(c.daClient, c.logger, c.getInitialDAStartHeight(ctx), c.genesis.DAEpochForcedInclusion) + initialDAHeight := c.getInitialDAStartHeight(ctx) + + // Create async epoch fetcher if not already created + if c.asyncFetcher == nil { + c.asyncFetcher = block.NewAsyncEpochFetcher( + c.daClient, + c.logger, + initialDAHeight, + c.genesis.DAEpochForcedInclusion, + 1, // prefetch 1 epoch ahead + 3*time.Second, // check every 3 seconds + ) + c.asyncFetcher.Start() + } + + c.fiRetriever = block.NewForcedInclusionRetriever(c.daClient, c.logger, initialDAHeight, c.genesis.DAEpochForcedInclusion, c.asyncFetcher) } // If we have no cached transactions or we've consumed all from the current cache, From c727394bf882bedf49e1a836c88129e262ccc6ec Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Wed, 7 Jan 2026 23:49:37 +0100 Subject: [PATCH 02/18] simplify --- apps/evm/cmd/run.go | 14 +--- apps/grpc/cmd/run.go | 15 +---- apps/testapp/cmd/run.go | 15 +---- pkg/sequencers/based/sequencer.go | 17 ++++- pkg/sequencers/based/sequencer_test.go | 88 +++++++++++++++++++++++++- pkg/sequencers/single/sequencer.go | 60 +++++------------- 6 files changed, 120 insertions(+), 89 deletions(-) diff --git a/apps/evm/cmd/run.go b/apps/evm/cmd/run.go index 16699f18b..dced7d0d9 100644 --- a/apps/evm/cmd/run.go +++ b/apps/evm/cmd/run.go @@ -167,19 +167,7 @@ func createSequencer( return nil, fmt.Errorf("based sequencer mode requires aggregator mode to be enabled") } - // Create async epoch fetcher for background prefetching - asyncFetcher := block.NewAsyncEpochFetcher( - daClient, - logger, - genesis.DAStartHeight, - genesis.DAEpochForcedInclusion, - 2, // prefetch 2 epochs ahead for based sequencer - 1*time.Second, // check frequently - ) - asyncFetcher.Start() - - fiRetriever := block.NewForcedInclusionRetriever(daClient, logger, genesis.DAStartHeight, genesis.DAEpochForcedInclusion, asyncFetcher) - basedSeq, err := based.NewBasedSequencer(fiRetriever, datastore, genesis, logger) + basedSeq, err := based.NewBasedSequencer(daClient, datastore, genesis, logger) if err != nil { return nil, fmt.Errorf("failed to create based sequencer: %w", err) } diff --git a/apps/grpc/cmd/run.go b/apps/grpc/cmd/run.go index 1106926aa..791d8be42 100644 --- a/apps/grpc/cmd/run.go +++ b/apps/grpc/cmd/run.go @@ -4,7 +4,6 @@ import ( "context" "fmt" "path/filepath" - "time" "github.com/ipfs/go-datastore" "github.com/rs/zerolog" @@ -128,19 +127,7 @@ func createSequencer( return nil, fmt.Errorf("based sequencer mode requires aggregator mode to be enabled") } - // Create async epoch fetcher for background prefetching - asyncFetcher := block.NewAsyncEpochFetcher( - daClient, - logger, - genesis.DAStartHeight, - genesis.DAEpochForcedInclusion, - 2, // prefetch 2 epochs ahead for based sequencer - 1*time.Second, // check frequently - ) - asyncFetcher.Start() - - fiRetriever := block.NewForcedInclusionRetriever(daClient, logger, genesis.DAStartHeight, genesis.DAEpochForcedInclusion, asyncFetcher) - basedSeq, err := based.NewBasedSequencer(fiRetriever, datastore, genesis, logger) + basedSeq, err := based.NewBasedSequencer(daClient, datastore, genesis, logger) if err != nil { return nil, fmt.Errorf("failed to create based sequencer: %w", err) } diff --git a/apps/testapp/cmd/run.go b/apps/testapp/cmd/run.go index e4d52eb24..41f6b7808 100644 --- a/apps/testapp/cmd/run.go +++ b/apps/testapp/cmd/run.go @@ -4,7 +4,6 @@ import ( "context" "fmt" "path/filepath" - "time" "github.com/ipfs/go-datastore" "github.com/rs/zerolog" @@ -129,19 +128,7 @@ func createSequencer( return nil, fmt.Errorf("based sequencer mode requires aggregator mode to be enabled") } - // Create async epoch fetcher for background prefetching - asyncFetcher := block.NewAsyncEpochFetcher( - daClient, - logger, - genesis.DAStartHeight, - genesis.DAEpochForcedInclusion, - 2, // prefetch 2 epochs ahead for based sequencer - 1*time.Second, // check frequently - ) - asyncFetcher.Start() - - fiRetriever := block.NewForcedInclusionRetriever(daClient, logger, genesis.DAStartHeight, genesis.DAEpochForcedInclusion, asyncFetcher) - basedSeq, err := based.NewBasedSequencer(fiRetriever, datastore, genesis, logger) + basedSeq, err := based.NewBasedSequencer(daClient, datastore, genesis, logger) if err != nil { return nil, fmt.Errorf("failed to create based sequencer: %w", err) } diff --git a/pkg/sequencers/based/sequencer.go b/pkg/sequencers/based/sequencer.go index feaca53f5..2b9753417 100644 --- a/pkg/sequencers/based/sequencer.go +++ b/pkg/sequencers/based/sequencer.go @@ -25,6 +25,7 @@ var _ coresequencer.Sequencer = (*BasedSequencer)(nil) type BasedSequencer struct { logger zerolog.Logger + asyncFetcher *block.AsyncEpochFetcher fiRetriever block.ForcedInclusionRetriever daHeight atomic.Uint64 checkpointStore *seqcommon.CheckpointStore @@ -38,7 +39,7 @@ type BasedSequencer struct { // NewBasedSequencer creates a new based sequencer instance func NewBasedSequencer( - fiRetriever block.ForcedInclusionRetriever, + daClient block.FullDAClient, db ds.Batching, genesis genesis.Genesis, logger zerolog.Logger, @@ -46,7 +47,6 @@ func NewBasedSequencer( bs := &BasedSequencer{ logger: logger.With().Str("component", "based_sequencer").Logger(), checkpointStore: seqcommon.NewCheckpointStore(db, ds.NewKey("/based/checkpoint")), - fiRetriever: fiRetriever, } // based sequencers need community consensus about the da start height given no submission are done bs.SetDAHeight(genesis.DAStartHeight) @@ -78,6 +78,19 @@ func NewBasedSequencer( } } + // Create async epoch fetcher for background prefetching (created once) + bs.asyncFetcher = block.NewAsyncEpochFetcher( + daClient, + logger, + genesis.DAStartHeight, + genesis.DAEpochForcedInclusion, + 2, // prefetch 2 epochs ahead for based sequencer + 1*time.Second, // check frequently + ) + bs.asyncFetcher.Start() + + bs.fiRetriever = block.NewForcedInclusionRetriever(daClient, logger, genesis.DAStartHeight, genesis.DAEpochForcedInclusion, bs.asyncFetcher) + return bs, nil } diff --git a/pkg/sequencers/based/sequencer_test.go b/pkg/sequencers/based/sequencer_test.go index 60525f09c..1e4808bf3 100644 --- a/pkg/sequencers/based/sequencer_test.go +++ b/pkg/sequencers/based/sequencer_test.go @@ -31,6 +31,71 @@ func (m *MockForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Con return args.Get(0).(*block.ForcedInclusionEvent), args.Error(1) } +// MockDAClient is a mock implementation of block.FullDAClient for testing +type MockDAClient struct { + mock.Mock + retriever *MockForcedInclusionRetriever +} + +// Client interface methods +func (m *MockDAClient) Submit(ctx context.Context, data [][]byte, gasPrice float64, namespace []byte, options []byte) datypes.ResultSubmit { + args := m.Called(ctx, data, gasPrice, namespace, options) + return args.Get(0).(datypes.ResultSubmit) +} + +func (m *MockDAClient) Retrieve(ctx context.Context, height uint64, namespace []byte) datypes.ResultRetrieve { + args := m.Called(ctx, height, namespace) + return args.Get(0).(datypes.ResultRetrieve) +} + +func (m *MockDAClient) Get(ctx context.Context, ids []datypes.ID, namespace []byte) ([]datypes.Blob, error) { + args := m.Called(ctx, ids, namespace) + if args.Get(0) == nil { + return nil, args.Error(1) + } + return args.Get(0).([]datypes.Blob), args.Error(1) +} + +func (m *MockDAClient) GetHeaderNamespace() []byte { + args := m.Called() + if args.Get(0) == nil { + return nil + } + return args.Get(0).([]byte) +} + +func (m *MockDAClient) GetDataNamespace() []byte { + args := m.Called() + if args.Get(0) == nil { + return nil + } + return args.Get(0).([]byte) +} + +func (m *MockDAClient) GetForcedInclusionNamespace() []byte { + args := m.Called() + if args.Get(0) == nil { + return nil + } + return args.Get(0).([]byte) +} + +func (m *MockDAClient) HasForcedInclusionNamespace() bool { + args := m.Called() + return args.Get(0).(bool) +} + +// Verifier interface methods +func (m *MockDAClient) GetProofs(ctx context.Context, ids []datypes.ID, namespace []byte) ([]datypes.Proof, error) { + args := m.Called(ctx, ids, namespace) + return args.Get(0).([]datypes.Proof), args.Error(1) +} + +func (m *MockDAClient) Validate(ctx context.Context, ids []datypes.ID, proofs []datypes.Proof, namespace []byte) ([]bool, error) { + args := m.Called(ctx, ids, proofs, namespace) + return args.Get(0).([]bool), args.Error(1) +} + // createTestSequencer is a helper function to create a sequencer for testing func createTestSequencer(t *testing.T, mockRetriever *MockForcedInclusionRetriever, gen genesis.Genesis) *BasedSequencer { t.Helper() @@ -38,8 +103,15 @@ func createTestSequencer(t *testing.T, mockRetriever *MockForcedInclusionRetriev // Create in-memory datastore db := syncds.MutexWrap(ds.NewMapDatastore()) - seq, err := NewBasedSequencer(mockRetriever, db, gen, zerolog.Nop()) + // Create mock DA client that wraps the retriever + mockDAClient := &MockDAClient{retriever: mockRetriever} + + seq, err := NewBasedSequencer(mockDAClient, db, gen, zerolog.Nop()) require.NoError(t, err) + + // Replace the fiRetriever with our mock so tests work as before + seq.fiRetriever = mockRetriever + return seq } @@ -454,10 +526,16 @@ func TestBasedSequencer_CheckpointPersistence(t *testing.T) { // Create persistent datastore db := syncds.MutexWrap(ds.NewMapDatastore()) + // Create mock DA client + mockDAClient := &MockDAClient{retriever: mockRetriever} + // Create first sequencer - seq1, err := NewBasedSequencer(mockRetriever, db, gen, zerolog.Nop()) + seq1, err := NewBasedSequencer(mockDAClient, db, gen, zerolog.Nop()) require.NoError(t, err) + // Replace the fiRetriever with our mock so tests work as before + seq1.fiRetriever = mockRetriever + req := coresequencer.GetNextBatchRequest{ MaxBytes: 1000000, LastBatchData: nil, @@ -470,9 +548,13 @@ func TestBasedSequencer_CheckpointPersistence(t *testing.T) { assert.Equal(t, 2, len(resp.Batch.Transactions)) // Create a new sequencer with the same datastore (simulating restart) - seq2, err := NewBasedSequencer(mockRetriever, db, gen, zerolog.Nop()) + mockDAClient2 := &MockDAClient{retriever: mockRetriever} + seq2, err := NewBasedSequencer(mockDAClient2, db, gen, zerolog.Nop()) require.NoError(t, err) + // Replace the fiRetriever with our mock so tests work as before + seq2.fiRetriever = mockRetriever + // Checkpoint should be loaded from DB assert.Equal(t, uint64(101), seq2.checkpoint.DAHeight) assert.Equal(t, uint64(0), seq2.checkpoint.TxIndex) diff --git a/pkg/sequencers/single/sequencer.go b/pkg/sequencers/single/sequencer.go index 19d840130..8e8cba656 100644 --- a/pkg/sequencers/single/sequencer.go +++ b/pkg/sequencers/single/sequencer.go @@ -91,19 +91,6 @@ func NewSequencer( DAHeight: s.GetDAHeight(), TxIndex: 0, } - - // Create async epoch fetcher for background prefetching - s.asyncFetcher = block.NewAsyncEpochFetcher( - daClient, - logger, - s.GetDAHeight(), - genesis.DAEpochForcedInclusion, - 1, // prefetch 1 epoch ahead - 3*time.Second, // check every 3 seconds - ) - s.asyncFetcher.Start() - - s.fiRetriever = block.NewForcedInclusionRetriever(daClient, logger, s.GetDAHeight(), genesis.DAEpochForcedInclusion, s.asyncFetcher) } else { return nil, fmt.Errorf("failed to load checkpoint from DB: %w", err) } @@ -117,22 +104,26 @@ func NewSequencer( Uint64("da_height", checkpoint.DAHeight). Msg("resuming from checkpoint within DA epoch") } + } - initialDAHeight := s.getInitialDAStartHeight(context.Background()) + // Determine initial DA height for forced inclusion + initialDAHeight := s.getInitialDAStartHeight(context.Background()) + if initialDAHeight == 0 { + initialDAHeight = s.GetDAHeight() + } - // Create async epoch fetcher for background prefetching - s.asyncFetcher = block.NewAsyncEpochFetcher( - daClient, - logger, - initialDAHeight, - genesis.DAEpochForcedInclusion, - 1, // prefetch 1 epoch ahead - 3*time.Second, // check every 3 seconds - ) - s.asyncFetcher.Start() + // Create async epoch fetcher for background prefetching (created once) + s.asyncFetcher = block.NewAsyncEpochFetcher( + daClient, + logger, + initialDAHeight, + genesis.DAEpochForcedInclusion, + 1, // prefetch 1 epoch ahead + 3*time.Second, // check every 3 seconds + ) + s.asyncFetcher.Start() - s.fiRetriever = block.NewForcedInclusionRetriever(daClient, logger, initialDAHeight, genesis.DAEpochForcedInclusion, s.asyncFetcher) - } + s.fiRetriever = block.NewForcedInclusionRetriever(daClient, logger, initialDAHeight, genesis.DAEpochForcedInclusion, s.asyncFetcher) return s, nil } @@ -199,23 +190,6 @@ func (c *Sequencer) GetNextBatch(ctx context.Context, req coresequencer.GetNextB DAHeight: daHeight, TxIndex: 0, } - - initialDAHeight := c.getInitialDAStartHeight(ctx) - - // Create async epoch fetcher if not already created - if c.asyncFetcher == nil { - c.asyncFetcher = block.NewAsyncEpochFetcher( - c.daClient, - c.logger, - initialDAHeight, - c.genesis.DAEpochForcedInclusion, - 1, // prefetch 1 epoch ahead - 3*time.Second, // check every 3 seconds - ) - c.asyncFetcher.Start() - } - - c.fiRetriever = block.NewForcedInclusionRetriever(c.daClient, c.logger, initialDAHeight, c.genesis.DAEpochForcedInclusion, c.asyncFetcher) } // If we have no cached transactions or we've consumed all from the current cache, From d08b3bfa8ae3294558696ae63b12dc8b98f74ab8 Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Thu, 8 Jan 2026 11:21:24 +0100 Subject: [PATCH 03/18] chore: fix unit tests --- block/internal/da/async_epoch_fetcher_test.go | 60 ++++--------------- pkg/p2p/utils_test.go | 8 ++- 2 files changed, 18 insertions(+), 50 deletions(-) diff --git a/block/internal/da/async_epoch_fetcher_test.go b/block/internal/da/async_epoch_fetcher_test.go index 07e561e14..d8a117a54 100644 --- a/block/internal/da/async_epoch_fetcher_test.go +++ b/block/internal/da/async_epoch_fetcher_test.go @@ -11,50 +11,12 @@ import ( "github.com/stretchr/testify/require" datypes "github.com/evstack/ev-node/pkg/da/types" -) - -// MockClient is a mock implementation of the Client interface -type MockClient struct { - mock.Mock -} - -func (m *MockClient) Submit(ctx context.Context, data [][]byte, gasPrice float64, namespace []byte, options []byte) datypes.ResultSubmit { - args := m.Called(ctx, data, gasPrice, namespace, options) - return args.Get(0).(datypes.ResultSubmit) -} - -func (m *MockClient) Retrieve(ctx context.Context, height uint64, namespace []byte) datypes.ResultRetrieve { - args := m.Called(ctx, height, namespace) - return args.Get(0).(datypes.ResultRetrieve) -} - -func (m *MockClient) Get(ctx context.Context, ids []datypes.ID, namespace []byte) ([]datypes.Blob, error) { - args := m.Called(ctx, ids, namespace) - return args.Get(0).([]datypes.Blob), args.Error(1) -} -func (m *MockClient) GetHeaderNamespace() []byte { - args := m.Called() - return args.Get(0).([]byte) -} - -func (m *MockClient) GetDataNamespace() []byte { - args := m.Called() - return args.Get(0).([]byte) -} - -func (m *MockClient) GetForcedInclusionNamespace() []byte { - args := m.Called() - return args.Get(0).([]byte) -} - -func (m *MockClient) HasForcedInclusionNamespace() bool { - args := m.Called() - return args.Bool(0) -} + mocks "github.com/evstack/ev-node/test/mocks" +) func TestAsyncEpochFetcher_Creation(t *testing.T) { - client := &MockClient{} + client := &mocks.MockClient{} logger := zerolog.Nop() fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 2, 100*time.Millisecond) @@ -66,7 +28,7 @@ func TestAsyncEpochFetcher_Creation(t *testing.T) { } func TestAsyncEpochFetcher_SetAndGetDAHeight(t *testing.T) { - client := &MockClient{} + client := &mocks.MockClient{} logger := zerolog.Nop() fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, time.Second) @@ -82,7 +44,7 @@ func TestAsyncEpochFetcher_SetAndGetDAHeight(t *testing.T) { } func TestAsyncEpochFetcher_GetCachedEpoch_NotAtEpochEnd(t *testing.T) { - client := &MockClient{} + client := &mocks.MockClient{} fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() client.On("HasForcedInclusionNamespace").Return(true) client.On("GetForcedInclusionNamespace").Return(fiNs) @@ -102,7 +64,7 @@ func TestAsyncEpochFetcher_GetCachedEpoch_NotAtEpochEnd(t *testing.T) { } func TestAsyncEpochFetcher_GetCachedEpoch_NoNamespace(t *testing.T) { - client := &MockClient{} + client := &mocks.MockClient{} client.On("HasForcedInclusionNamespace").Return(false) logger := zerolog.Nop() @@ -114,7 +76,7 @@ func TestAsyncEpochFetcher_GetCachedEpoch_NoNamespace(t *testing.T) { } func TestAsyncEpochFetcher_GetCachedEpoch_CacheMiss(t *testing.T) { - client := &MockClient{} + client := &mocks.MockClient{} fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() client.On("HasForcedInclusionNamespace").Return(true) client.On("GetForcedInclusionNamespace").Return(fiNs) @@ -137,7 +99,7 @@ func TestAsyncEpochFetcher_FetchAndCache(t *testing.T) { []byte("tx3"), } - client := &MockClient{} + client := &mocks.MockClient{} fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() client.On("HasForcedInclusionNamespace").Return(true) client.On("GetForcedInclusionNamespace").Return(fiNs) @@ -182,7 +144,7 @@ func TestAsyncEpochFetcher_BackgroundPrefetch(t *testing.T) { []byte("tx2"), } - client := &MockClient{} + client := &mocks.MockClient{} fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() client.On("HasForcedInclusionNamespace").Return(true) client.On("GetForcedInclusionNamespace").Return(fiNs) @@ -276,7 +238,7 @@ func TestAsyncEpochFetcher_SerializationEmpty(t *testing.T) { } func TestAsyncEpochFetcher_HeightFromFuture(t *testing.T) { - client := &MockClient{} + client := &mocks.MockClient{} fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() client.On("HasForcedInclusionNamespace").Return(true) client.On("GetForcedInclusionNamespace").Return(fiNs) @@ -300,7 +262,7 @@ func TestAsyncEpochFetcher_HeightFromFuture(t *testing.T) { } func TestAsyncEpochFetcher_StopGracefully(t *testing.T) { - client := &MockClient{} + client := &mocks.MockClient{} client.On("HasForcedInclusionNamespace").Return(false) logger := zerolog.Nop() diff --git a/pkg/p2p/utils_test.go b/pkg/p2p/utils_test.go index dadb151c7..5bd5664dd 100644 --- a/pkg/p2p/utils_test.go +++ b/pkg/p2p/utils_test.go @@ -113,13 +113,19 @@ func startTestNetwork(ctx context.Context, t *testing.T, n int, conf map[int]hos nodeKey, err := key.GenerateNodeKey() require.NoError(err) + // Use chainID from conf if specified, otherwise default to "test-chain" + chainID := "test-chain" + if descr, ok := conf[i]; ok && descr.chainID != "" { + chainID = descr.chainID + } + client, err := NewClient( config.P2PConfig{ Peers: seeds[i], }, nodeKey.PrivKey, sync.MutexWrap(datastore.NewMapDatastore()), - "test-chain", + chainID, logger, NopMetrics(), ) From 7f789eeffa57e7342cd4863fc8ffb196c1150bbd Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Thu, 8 Jan 2026 17:42:16 +0100 Subject: [PATCH 04/18] chore: use interfaces --- block/internal/da/async_epoch_fetcher.go | 55 ++++++----------- block/internal/da/async_epoch_fetcher_test.go | 61 +++++++------------ .../internal/da/forced_inclusion_retriever.go | 6 +- .../da/forced_inclusion_retriever_test.go | 4 +- block/internal/syncing/syncer.go | 4 +- .../syncing/syncer_forced_inclusion_test.go | 2 +- block/public.go | 10 ++- pkg/sequencers/based/sequencer.go | 2 +- pkg/sequencers/single/sequencer.go | 2 +- 9 files changed, 57 insertions(+), 89 deletions(-) diff --git a/block/internal/da/async_epoch_fetcher.go b/block/internal/da/async_epoch_fetcher.go index a05ed342d..72204cf1f 100644 --- a/block/internal/da/async_epoch_fetcher.go +++ b/block/internal/da/async_epoch_fetcher.go @@ -16,9 +16,16 @@ import ( "github.com/evstack/ev-node/types" ) -// AsyncEpochFetcher handles background prefetching of DA epoch data +// AsyncEpochFetcher provides background prefetching of DA epoch data +type AsyncEpochFetcher interface { + Start() + Stop() + GetCachedEpoch(ctx context.Context, daHeight uint64) (*ForcedInclusionEvent, error) +} + +// asyncEpochFetcher handles background prefetching of DA epoch data // to speed up processing at epoch boundaries. -type AsyncEpochFetcher struct { +type asyncEpochFetcher struct { client Client logger zerolog.Logger daEpochSize uint64 @@ -35,7 +42,6 @@ type AsyncEpochFetcher struct { // Current DA height tracking currentDAHeight uint64 - heightMu sync.RWMutex // Prefetch window - how many epochs ahead to prefetch prefetchWindow uint64 @@ -51,7 +57,7 @@ func NewAsyncEpochFetcher( daStartHeight, daEpochSize uint64, prefetchWindow uint64, pollInterval time.Duration, -) *AsyncEpochFetcher { +) AsyncEpochFetcher { if prefetchWindow == 0 { prefetchWindow = 1 // Default: prefetch next epoch } @@ -61,7 +67,7 @@ func NewAsyncEpochFetcher( ctx, cancel := context.WithCancel(context.Background()) - return &AsyncEpochFetcher{ + return &asyncEpochFetcher{ client: client, logger: logger.With().Str("component", "async_epoch_fetcher").Logger(), daStartHeight: daStartHeight, @@ -76,7 +82,7 @@ func NewAsyncEpochFetcher( } // Start begins the background prefetching process. -func (f *AsyncEpochFetcher) Start() { +func (f *asyncEpochFetcher) Start() { f.wg.Add(1) go f.backgroundFetchLoop() f.logger.Info(). @@ -88,35 +94,16 @@ func (f *AsyncEpochFetcher) Start() { } // Stop gracefully stops the background prefetching process. -func (f *AsyncEpochFetcher) Stop() { +func (f *asyncEpochFetcher) Stop() { f.logger.Info().Msg("stopping async epoch fetcher") f.cancel() f.wg.Wait() f.logger.Info().Msg("async epoch fetcher stopped") } -// SetDAHeight updates the current DA height being processed. -// This is called by sequencers to inform the fetcher of progress. -func (f *AsyncEpochFetcher) SetDAHeight(height uint64) { - f.heightMu.Lock() - defer f.heightMu.Unlock() - - if height > f.currentDAHeight { - f.currentDAHeight = height - f.logger.Debug().Uint64("da_height", height).Msg("updated current DA height") - } -} - -// GetDAHeight returns the current DA height. -func (f *AsyncEpochFetcher) GetDAHeight() uint64 { - f.heightMu.RLock() - defer f.heightMu.RUnlock() - return f.currentDAHeight -} - // GetCachedEpoch retrieves a cached epoch from memory. // Returns nil if the epoch is not cached. -func (f *AsyncEpochFetcher) GetCachedEpoch(ctx context.Context, daHeight uint64) (*ForcedInclusionEvent, error) { +func (f *asyncEpochFetcher) GetCachedEpoch(ctx context.Context, daHeight uint64) (*ForcedInclusionEvent, error) { if !f.client.HasForcedInclusionNamespace() { return nil, ErrForceInclusionNotConfigured } @@ -167,7 +154,7 @@ func (f *AsyncEpochFetcher) GetCachedEpoch(ctx context.Context, daHeight uint64) } // backgroundFetchLoop runs in the background and prefetches epochs ahead of time. -func (f *AsyncEpochFetcher) backgroundFetchLoop() { +func (f *asyncEpochFetcher) backgroundFetchLoop() { defer f.wg.Done() ticker := time.NewTicker(f.pollInterval) @@ -184,15 +171,13 @@ func (f *AsyncEpochFetcher) backgroundFetchLoop() { } // prefetchEpochs prefetches epochs within the prefetch window. -func (f *AsyncEpochFetcher) prefetchEpochs() { +func (f *asyncEpochFetcher) prefetchEpochs() { if !f.client.HasForcedInclusionNamespace() { return } - currentHeight := f.GetDAHeight() - // Calculate the current epoch and epochs to prefetch - _, currentEpochEnd, _ := types.CalculateEpochBoundaries(currentHeight, f.daStartHeight, f.daEpochSize) + _, currentEpochEnd, _ := types.CalculateEpochBoundaries(f.currentDAHeight, f.daStartHeight, f.daEpochSize) // Prefetch upcoming epochs for i := uint64(0); i < f.prefetchWindow; i++ { @@ -218,7 +203,7 @@ func (f *AsyncEpochFetcher) prefetchEpochs() { } // fetchAndCacheEpoch fetches an epoch and stores it in the cache. -func (f *AsyncEpochFetcher) fetchAndCacheEpoch(epochEnd uint64) { +func (f *asyncEpochFetcher) fetchAndCacheEpoch(epochEnd uint64) { epochStart := epochEnd - (f.daEpochSize - 1) if epochStart < f.daStartHeight { epochStart = f.daStartHeight @@ -313,7 +298,7 @@ func (f *AsyncEpochFetcher) fetchAndCacheEpoch(epochEnd uint64) { } // processForcedInclusionBlobs processes blobs from a single DA height for forced inclusion. -func (f *AsyncEpochFetcher) processForcedInclusionBlobs( +func (f *asyncEpochFetcher) processForcedInclusionBlobs( event *ForcedInclusionEvent, result datypes.ResultRetrieve, height uint64, @@ -347,7 +332,7 @@ func (f *AsyncEpochFetcher) processForcedInclusionBlobs( } // cleanupOldEpochs removes epochs older than the current epoch from cache. -func (f *AsyncEpochFetcher) cleanupOldEpochs(currentEpochEnd uint64) { +func (f *asyncEpochFetcher) cleanupOldEpochs(currentEpochEnd uint64) { // Remove epochs older than current - 1 // Keep current and previous in case of reorgs or restarts cleanupThreshold := currentEpochEnd - f.daEpochSize diff --git a/block/internal/da/async_epoch_fetcher_test.go b/block/internal/da/async_epoch_fetcher_test.go index d8a117a54..1034956b6 100644 --- a/block/internal/da/async_epoch_fetcher_test.go +++ b/block/internal/da/async_epoch_fetcher_test.go @@ -15,34 +15,6 @@ import ( mocks "github.com/evstack/ev-node/test/mocks" ) -func TestAsyncEpochFetcher_Creation(t *testing.T) { - client := &mocks.MockClient{} - logger := zerolog.Nop() - - fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 2, 100*time.Millisecond) - assert.NotNil(t, fetcher) - assert.Equal(t, uint64(100), fetcher.daStartHeight) - assert.Equal(t, uint64(10), fetcher.daEpochSize) - assert.Equal(t, uint64(2), fetcher.prefetchWindow) - assert.Equal(t, 100*time.Millisecond, fetcher.pollInterval) -} - -func TestAsyncEpochFetcher_SetAndGetDAHeight(t *testing.T) { - client := &mocks.MockClient{} - logger := zerolog.Nop() - - fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, time.Second) - - assert.Equal(t, uint64(100), fetcher.GetDAHeight()) - - fetcher.SetDAHeight(150) - assert.Equal(t, uint64(150), fetcher.GetDAHeight()) - - // Should not decrease - fetcher.SetDAHeight(120) - assert.Equal(t, uint64(150), fetcher.GetDAHeight()) -} - func TestAsyncEpochFetcher_GetCachedEpoch_NotAtEpochEnd(t *testing.T) { client := &mocks.MockClient{} fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() @@ -122,16 +94,27 @@ func TestAsyncEpochFetcher_FetchAndCache(t *testing.T) { } logger := zerolog.Nop() - fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, time.Second) - - // Manually trigger fetch - fetcher.fetchAndCacheEpoch(109) + // Use a short poll interval for faster test execution + fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, 50*time.Millisecond) + fetcher.Start() + defer fetcher.Stop() - // Now try to get from cache + // Wait for the background fetch to complete by polling the cache ctx := context.Background() - event, err := fetcher.GetCachedEpoch(ctx, 109) - require.NoError(t, err) - assert.NotNil(t, event) + var event *ForcedInclusionEvent + var err error + + // Poll for up to 2 seconds for the epoch to be cached + for i := 0; i < 40; i++ { + event, err = fetcher.GetCachedEpoch(ctx, 109) + require.NoError(t, err) + if event != nil { + break + } + time.Sleep(50 * time.Millisecond) + } + + require.NotNil(t, event, "epoch should be cached after background fetch") assert.Equal(t, 3, len(event.Txs)) assert.Equal(t, testBlobs[0], event.Txs[0]) assert.Equal(t, uint64(100), event.StartDaHeight) @@ -176,7 +159,6 @@ func TestAsyncEpochFetcher_BackgroundPrefetch(t *testing.T) { logger := zerolog.Nop() fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, 50*time.Millisecond) - fetcher.SetDAHeight(100) fetcher.Start() defer fetcher.Stop() @@ -250,9 +232,8 @@ func TestAsyncEpochFetcher_HeightFromFuture(t *testing.T) { logger := zerolog.Nop() fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, time.Second) - - // This should not panic and should handle gracefully - fetcher.fetchAndCacheEpoch(109) + fetcher.Start() + defer fetcher.Stop() // Cache should be empty ctx := context.Background() diff --git a/block/internal/da/forced_inclusion_retriever.go b/block/internal/da/forced_inclusion_retriever.go index 306ab632b..d146477f7 100644 --- a/block/internal/da/forced_inclusion_retriever.go +++ b/block/internal/da/forced_inclusion_retriever.go @@ -21,7 +21,7 @@ type ForcedInclusionRetriever struct { logger zerolog.Logger daEpochSize uint64 daStartHeight uint64 - asyncFetcher *AsyncEpochFetcher // Required for async prefetching + asyncFetcher AsyncEpochFetcher // Required for async prefetching } // ForcedInclusionEvent contains forced inclusion transactions retrieved from DA. @@ -33,12 +33,11 @@ type ForcedInclusionEvent struct { } // NewForcedInclusionRetriever creates a new forced inclusion retriever. -// The asyncFetcher parameter is required for background prefetching of DA epoch data. func NewForcedInclusionRetriever( client Client, logger zerolog.Logger, daStartHeight, daEpochSize uint64, - asyncFetcher *AsyncEpochFetcher, + asyncFetcher AsyncEpochFetcher, ) *ForcedInclusionRetriever { return &ForcedInclusionRetriever{ client: client, @@ -51,7 +50,6 @@ func NewForcedInclusionRetriever( // RetrieveForcedIncludedTxs retrieves forced inclusion transactions at the given DA height. // It respects epoch boundaries and only fetches at epoch start. -// If an async fetcher is configured, it will try to use cached data first for better performance. func (r *ForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Context, daHeight uint64) (*ForcedInclusionEvent, error) { // when daStartHeight is not set or no namespace is configured, we retrieve nothing. if !r.client.HasForcedInclusionNamespace() { diff --git a/block/internal/da/forced_inclusion_retriever_test.go b/block/internal/da/forced_inclusion_retriever_test.go index a83a22f3a..eb8e6f7fd 100644 --- a/block/internal/da/forced_inclusion_retriever_test.go +++ b/block/internal/da/forced_inclusion_retriever_test.go @@ -14,8 +14,8 @@ import ( "github.com/evstack/ev-node/test/mocks" ) -// createTestAsyncFetcher creates a minimal async fetcher for tests (without starting it) -func createTestAsyncFetcher(client Client, gen genesis.Genesis) *AsyncEpochFetcher { +// createTestAsyncFetcher creates a minimal async fetcher for tests +func createTestAsyncFetcher(client Client, gen genesis.Genesis) AsyncEpochFetcher { return NewAsyncEpochFetcher( client, zerolog.Nop(), diff --git a/block/internal/syncing/syncer.go b/block/internal/syncing/syncer.go index 265ff15be..4ff5c54ab 100644 --- a/block/internal/syncing/syncer.go +++ b/block/internal/syncing/syncer.go @@ -194,8 +194,8 @@ func (s *Syncer) Start(ctx context.Context) error { s.logger, s.genesis.DAStartHeight, s.genesis.DAEpochForcedInclusion, - 1, // prefetch 1 epoch ahead - 2*time.Second, // check every 2 seconds + 1, // prefetch 1 epoch ahead + s.config.DA.BlockTime.Duration, ) asyncFetcher.Start() diff --git a/block/internal/syncing/syncer_forced_inclusion_test.go b/block/internal/syncing/syncer_forced_inclusion_test.go index 29df5e0ad..4c1967b61 100644 --- a/block/internal/syncing/syncer_forced_inclusion_test.go +++ b/block/internal/syncing/syncer_forced_inclusion_test.go @@ -24,7 +24,7 @@ import ( ) // createTestAsyncFetcher creates a minimal async fetcher for tests (without starting it) -func createTestAsyncFetcherForSyncer(client da.Client, gen genesis.Genesis) *da.AsyncEpochFetcher { +func createTestAsyncFetcherForSyncer(client da.Client, gen genesis.Genesis) da.AsyncEpochFetcher { return da.NewAsyncEpochFetcher( client, zerolog.Nop(), diff --git a/block/public.go b/block/public.go index 4350b4eee..38de92380 100644 --- a/block/public.go +++ b/block/public.go @@ -67,7 +67,11 @@ var ErrForceInclusionNotConfigured = da.ErrForceInclusionNotConfigured type ForcedInclusionEvent = da.ForcedInclusionEvent // AsyncEpochFetcher provides background prefetching of DA epoch data -type AsyncEpochFetcher = da.AsyncEpochFetcher +type AsyncEpochFetcher interface { + Start() + Stop() + GetCachedEpoch(ctx context.Context, daHeight uint64) (*da.ForcedInclusionEvent, error) +} // ForcedInclusionRetriever defines the interface for retrieving forced inclusion transactions from DA type ForcedInclusionRetriever interface { @@ -88,7 +92,7 @@ func NewAsyncEpochFetcher( daStartHeight, daEpochSize uint64, prefetchWindow uint64, pollInterval time.Duration, -) *AsyncEpochFetcher { +) AsyncEpochFetcher { return da.NewAsyncEpochFetcher(client, logger, daStartHeight, daEpochSize, prefetchWindow, pollInterval) } @@ -98,7 +102,7 @@ func NewForcedInclusionRetriever( client DAClient, logger zerolog.Logger, daStartHeight, daEpochSize uint64, - asyncFetcher *AsyncEpochFetcher, + asyncFetcher AsyncEpochFetcher, ) ForcedInclusionRetriever { return da.NewForcedInclusionRetriever(client, logger, daStartHeight, daEpochSize, asyncFetcher) } diff --git a/pkg/sequencers/based/sequencer.go b/pkg/sequencers/based/sequencer.go index 2b9753417..a59608524 100644 --- a/pkg/sequencers/based/sequencer.go +++ b/pkg/sequencers/based/sequencer.go @@ -25,7 +25,7 @@ var _ coresequencer.Sequencer = (*BasedSequencer)(nil) type BasedSequencer struct { logger zerolog.Logger - asyncFetcher *block.AsyncEpochFetcher + asyncFetcher block.AsyncEpochFetcher fiRetriever block.ForcedInclusionRetriever daHeight atomic.Uint64 checkpointStore *seqcommon.CheckpointStore diff --git a/pkg/sequencers/single/sequencer.go b/pkg/sequencers/single/sequencer.go index 8e8cba656..ed1112589 100644 --- a/pkg/sequencers/single/sequencer.go +++ b/pkg/sequencers/single/sequencer.go @@ -40,7 +40,7 @@ type Sequencer struct { queue *BatchQueue // single queue for immediate availability // Forced inclusion support - asyncFetcher *block.AsyncEpochFetcher + asyncFetcher block.AsyncEpochFetcher fiRetriever block.ForcedInclusionRetriever daHeight atomic.Uint64 daStartHeight atomic.Uint64 From 8cd87132750ed04e97706ec90b9c358b583d1993 Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Thu, 8 Jan 2026 18:09:47 +0100 Subject: [PATCH 05/18] updates --- apps/evm/cmd/run.go | 4 +-- apps/grpc/cmd/run.go | 4 +-- apps/testapp/cmd/run.go | 4 +-- block/internal/da/async_epoch_fetcher.go | 6 +---- block/internal/da/async_epoch_fetcher_test.go | 2 +- block/public.go | 7 +++-- pkg/sequencers/based/sequencer.go | 6 +++-- pkg/sequencers/based/sequencer_test.go | 7 ++--- pkg/sequencers/single/sequencer.go | 9 ++++--- pkg/sequencers/single/sequencer_test.go | 27 ++++++++++--------- 10 files changed, 38 insertions(+), 38 deletions(-) diff --git a/apps/evm/cmd/run.go b/apps/evm/cmd/run.go index dced7d0d9..7e1b8ae33 100644 --- a/apps/evm/cmd/run.go +++ b/apps/evm/cmd/run.go @@ -167,7 +167,7 @@ func createSequencer( return nil, fmt.Errorf("based sequencer mode requires aggregator mode to be enabled") } - basedSeq, err := based.NewBasedSequencer(daClient, datastore, genesis, logger) + basedSeq, err := based.NewBasedSequencer(daClient, nodeConfig, datastore, genesis, logger) if err != nil { return nil, fmt.Errorf("failed to create based sequencer: %w", err) } @@ -184,8 +184,8 @@ func createSequencer( logger, datastore, daClient, + nodeConfig, []byte(genesis.ChainID), - nodeConfig.Node.BlockTime.Duration, 1000, genesis, ) diff --git a/apps/grpc/cmd/run.go b/apps/grpc/cmd/run.go index 791d8be42..2cd4d5610 100644 --- a/apps/grpc/cmd/run.go +++ b/apps/grpc/cmd/run.go @@ -127,7 +127,7 @@ func createSequencer( return nil, fmt.Errorf("based sequencer mode requires aggregator mode to be enabled") } - basedSeq, err := based.NewBasedSequencer(daClient, datastore, genesis, logger) + basedSeq, err := based.NewBasedSequencer(daClient, nodeConfig, datastore, genesis, logger) if err != nil { return nil, fmt.Errorf("failed to create based sequencer: %w", err) } @@ -144,8 +144,8 @@ func createSequencer( logger, datastore, daClient, + nodeConfig, []byte(genesis.ChainID), - nodeConfig.Node.BlockTime.Duration, 1000, genesis, ) diff --git a/apps/testapp/cmd/run.go b/apps/testapp/cmd/run.go index 41f6b7808..bc8ab889d 100644 --- a/apps/testapp/cmd/run.go +++ b/apps/testapp/cmd/run.go @@ -128,7 +128,7 @@ func createSequencer( return nil, fmt.Errorf("based sequencer mode requires aggregator mode to be enabled") } - basedSeq, err := based.NewBasedSequencer(daClient, datastore, genesis, logger) + basedSeq, err := based.NewBasedSequencer(daClient, nodeConfig, datastore, genesis, logger) if err != nil { return nil, fmt.Errorf("failed to create based sequencer: %w", err) } @@ -145,8 +145,8 @@ func createSequencer( logger, datastore, daClient, + nodeConfig, []byte(genesis.ChainID), - nodeConfig.Node.BlockTime.Duration, 1000, genesis, ) diff --git a/block/internal/da/async_epoch_fetcher.go b/block/internal/da/async_epoch_fetcher.go index 72204cf1f..cd039cfdd 100644 --- a/block/internal/da/async_epoch_fetcher.go +++ b/block/internal/da/async_epoch_fetcher.go @@ -204,11 +204,7 @@ func (f *asyncEpochFetcher) prefetchEpochs() { // fetchAndCacheEpoch fetches an epoch and stores it in the cache. func (f *asyncEpochFetcher) fetchAndCacheEpoch(epochEnd uint64) { - epochStart := epochEnd - (f.daEpochSize - 1) - if epochStart < f.daStartHeight { - epochStart = f.daStartHeight - } - + epochStart := max(epochEnd-(f.daEpochSize-1), f.daStartHeight) f.logger.Debug(). Uint64("epoch_start", epochStart). Uint64("epoch_end", epochEnd). diff --git a/block/internal/da/async_epoch_fetcher_test.go b/block/internal/da/async_epoch_fetcher_test.go index 1034956b6..6f485a595 100644 --- a/block/internal/da/async_epoch_fetcher_test.go +++ b/block/internal/da/async_epoch_fetcher_test.go @@ -116,7 +116,7 @@ func TestAsyncEpochFetcher_FetchAndCache(t *testing.T) { require.NotNil(t, event, "epoch should be cached after background fetch") assert.Equal(t, 3, len(event.Txs)) - assert.Equal(t, testBlobs[0], event.Txs[0]) + assert.Equal(t, testBlobs[0], event.Txs[0]) //nolint:gosec // false positive assert.Equal(t, uint64(100), event.StartDaHeight) assert.Equal(t, uint64(109), event.EndDaHeight) } diff --git a/block/public.go b/block/public.go index 38de92380..67b85d482 100644 --- a/block/public.go +++ b/block/public.go @@ -2,7 +2,6 @@ package block import ( "context" - "time" "github.com/evstack/ev-node/block/internal/common" "github.com/evstack/ev-node/block/internal/da" @@ -81,19 +80,19 @@ type ForcedInclusionRetriever interface { // NewAsyncEpochFetcher creates a new async epoch fetcher for background prefetching. // Parameters: // - client: DA client for fetching data +// - config: Ev-node config // - logger: structured logger // - daStartHeight: genesis DA start height // - daEpochSize: number of DA blocks per epoch // - prefetchWindow: how many epochs ahead to prefetch (1-2 recommended) -// - pollInterval: how often to check for new epochs to prefetch func NewAsyncEpochFetcher( client DAClient, + cfg config.Config, logger zerolog.Logger, daStartHeight, daEpochSize uint64, prefetchWindow uint64, - pollInterval time.Duration, ) AsyncEpochFetcher { - return da.NewAsyncEpochFetcher(client, logger, daStartHeight, daEpochSize, prefetchWindow, pollInterval) + return da.NewAsyncEpochFetcher(client, logger, daStartHeight, daEpochSize, prefetchWindow, cfg.DA.BlockTime.Duration) } // NewForcedInclusionRetriever creates a new forced inclusion retriever. diff --git a/pkg/sequencers/based/sequencer.go b/pkg/sequencers/based/sequencer.go index a59608524..64e514f62 100644 --- a/pkg/sequencers/based/sequencer.go +++ b/pkg/sequencers/based/sequencer.go @@ -12,6 +12,7 @@ import ( "github.com/evstack/ev-node/block" coresequencer "github.com/evstack/ev-node/core/sequencer" + "github.com/evstack/ev-node/pkg/config" datypes "github.com/evstack/ev-node/pkg/da/types" "github.com/evstack/ev-node/pkg/genesis" seqcommon "github.com/evstack/ev-node/pkg/sequencers/common" @@ -40,6 +41,7 @@ type BasedSequencer struct { // NewBasedSequencer creates a new based sequencer instance func NewBasedSequencer( daClient block.FullDAClient, + cfg config.Config, db ds.Batching, genesis genesis.Genesis, logger zerolog.Logger, @@ -81,11 +83,11 @@ func NewBasedSequencer( // Create async epoch fetcher for background prefetching (created once) bs.asyncFetcher = block.NewAsyncEpochFetcher( daClient, + cfg, logger, genesis.DAStartHeight, genesis.DAEpochForcedInclusion, - 2, // prefetch 2 epochs ahead for based sequencer - 1*time.Second, // check frequently + 2, // prefetch 2 epochs ahead for based sequencer ) bs.asyncFetcher.Start() diff --git a/pkg/sequencers/based/sequencer_test.go b/pkg/sequencers/based/sequencer_test.go index 1e4808bf3..0e274edd8 100644 --- a/pkg/sequencers/based/sequencer_test.go +++ b/pkg/sequencers/based/sequencer_test.go @@ -14,6 +14,7 @@ import ( "github.com/evstack/ev-node/block" coresequencer "github.com/evstack/ev-node/core/sequencer" + "github.com/evstack/ev-node/pkg/config" datypes "github.com/evstack/ev-node/pkg/da/types" "github.com/evstack/ev-node/pkg/genesis" ) @@ -106,7 +107,7 @@ func createTestSequencer(t *testing.T, mockRetriever *MockForcedInclusionRetriev // Create mock DA client that wraps the retriever mockDAClient := &MockDAClient{retriever: mockRetriever} - seq, err := NewBasedSequencer(mockDAClient, db, gen, zerolog.Nop()) + seq, err := NewBasedSequencer(mockDAClient, config.DefaultConfig(), db, gen, zerolog.Nop()) require.NoError(t, err) // Replace the fiRetriever with our mock so tests work as before @@ -530,7 +531,7 @@ func TestBasedSequencer_CheckpointPersistence(t *testing.T) { mockDAClient := &MockDAClient{retriever: mockRetriever} // Create first sequencer - seq1, err := NewBasedSequencer(mockDAClient, db, gen, zerolog.Nop()) + seq1, err := NewBasedSequencer(mockDAClient, config.DefaultConfig(), db, gen, zerolog.Nop()) require.NoError(t, err) // Replace the fiRetriever with our mock so tests work as before @@ -549,7 +550,7 @@ func TestBasedSequencer_CheckpointPersistence(t *testing.T) { // Create a new sequencer with the same datastore (simulating restart) mockDAClient2 := &MockDAClient{retriever: mockRetriever} - seq2, err := NewBasedSequencer(mockDAClient2, db, gen, zerolog.Nop()) + seq2, err := NewBasedSequencer(mockDAClient2, config.DefaultConfig(), db, gen, zerolog.Nop()) require.NoError(t, err) // Replace the fiRetriever with our mock so tests work as before diff --git a/pkg/sequencers/single/sequencer.go b/pkg/sequencers/single/sequencer.go index ed1112589..c0fafdc41 100644 --- a/pkg/sequencers/single/sequencer.go +++ b/pkg/sequencers/single/sequencer.go @@ -16,6 +16,7 @@ import ( "github.com/evstack/ev-node/block" coresequencer "github.com/evstack/ev-node/core/sequencer" + "github.com/evstack/ev-node/pkg/config" datypes "github.com/evstack/ev-node/pkg/da/types" "github.com/evstack/ev-node/pkg/genesis" seqcommon "github.com/evstack/ev-node/pkg/sequencers/common" @@ -56,8 +57,8 @@ func NewSequencer( logger zerolog.Logger, db ds.Batching, daClient block.FullDAClient, + cfg config.Config, id []byte, - batchTime time.Duration, maxQueueSize int, genesis genesis.Genesis, ) (*Sequencer, error) { @@ -65,7 +66,7 @@ func NewSequencer( db: db, logger: logger, daClient: daClient, - batchTime: batchTime, + batchTime: cfg.Node.BlockTime.Duration, Id: id, queue: NewBatchQueue(db, "batches", maxQueueSize), checkpointStore: seqcommon.NewCheckpointStore(db, ds.NewKey("/single/checkpoint")), @@ -115,11 +116,11 @@ func NewSequencer( // Create async epoch fetcher for background prefetching (created once) s.asyncFetcher = block.NewAsyncEpochFetcher( daClient, + cfg, logger, initialDAHeight, genesis.DAEpochForcedInclusion, - 1, // prefetch 1 epoch ahead - 3*time.Second, // check every 3 seconds + 1, // prefetch 1 epoch ahead ) s.asyncFetcher.Start() diff --git a/pkg/sequencers/single/sequencer_test.go b/pkg/sequencers/single/sequencer_test.go index 3c0afdbf5..cf145672d 100644 --- a/pkg/sequencers/single/sequencer_test.go +++ b/pkg/sequencers/single/sequencer_test.go @@ -15,6 +15,7 @@ import ( "github.com/evstack/ev-node/block" coresequencer "github.com/evstack/ev-node/core/sequencer" + "github.com/evstack/ev-node/pkg/config" datypes "github.com/evstack/ev-node/pkg/da/types" "github.com/evstack/ev-node/pkg/genesis" damocks "github.com/evstack/ev-node/test/mocks" @@ -64,8 +65,8 @@ func newTestSequencer(t *testing.T, db ds.Batching, daClient block.FullDAClient) logger, db, daClient, + config.DefaultConfig(), []byte("test"), - 1*time.Second, 0, // unlimited queue gen, ) @@ -78,7 +79,7 @@ func TestSequencer_SubmitBatchTxs(t *testing.T) { db := ds.NewMapDatastore() Id := []byte("test1") logger := zerolog.Nop() - seq, err := NewSequencer(logger, db, dummyDA, Id, 10*time.Second, 1000, genesis.Genesis{}) + seq, err := NewSequencer(logger, db, dummyDA, config.DefaultConfig(), Id, 1000, genesis.Genesis{}) if err != nil { t.Fatalf("Failed to create sequencer: %v", err) } @@ -127,7 +128,7 @@ func TestSequencer_SubmitBatchTxs_EmptyBatch(t *testing.T) { db := ds.NewMapDatastore() Id := []byte("test1") logger := zerolog.Nop() - seq, err := NewSequencer(logger, db, dummyDA, Id, 10*time.Second, 1000, genesis.Genesis{}) + seq, err := NewSequencer(logger, db, dummyDA, config.DefaultConfig(), Id, 1000, genesis.Genesis{}) require.NoError(t, err, "Failed to create sequencer") defer func() { err := db.Close() @@ -178,8 +179,8 @@ func TestSequencer_GetNextBatch_NoLastBatch(t *testing.T) { logger, db, dummyDA, + config.DefaultConfig(), []byte("test"), - 1*time.Second, 0, // unlimited queue gen, ) @@ -298,7 +299,7 @@ func TestSequencer_GetNextBatch_BeforeDASubmission(t *testing.T) { dummyDA := newDummyDA(100_000_000) db := ds.NewMapDatastore() logger := zerolog.Nop() - seq, err := NewSequencer(logger, db, dummyDA, []byte("test1"), 1*time.Second, 1000, genesis.Genesis{}) + seq, err := NewSequencer(logger, db, dummyDA, config.DefaultConfig(), []byte("test1"), 1000, genesis.Genesis{}) if err != nil { t.Fatalf("Failed to create sequencer: %v", err) } @@ -376,8 +377,8 @@ func TestSequencer_GetNextBatch_ForcedInclusionAndBatch_MaxBytes(t *testing.T) { logger, db, mockDA, + config.DefaultConfig(), []byte("test-chain"), - 1*time.Second, 100, gen, ) @@ -469,8 +470,8 @@ func TestSequencer_GetNextBatch_ForcedInclusion_ExceedsMaxBytes(t *testing.T) { logger, db, mockDA, + config.DefaultConfig(), []byte("test-chain"), - 1*time.Second, 100, gen, ) @@ -544,8 +545,8 @@ func TestSequencer_GetNextBatch_AlwaysCheckPendingForcedInclusion(t *testing.T) logger, db, mockDA, + config.DefaultConfig(), []byte("test-chain"), - 1*time.Second, 100, gen, ) @@ -618,8 +619,8 @@ func TestSequencer_QueueLimit_Integration(t *testing.T) { logger, db, dummyDA, + config.DefaultConfig(), []byte("test"), - time.Second, 2, // Very small limit for testing gen, ) @@ -733,8 +734,8 @@ func TestSequencer_DAFailureAndQueueThrottling_Integration(t *testing.T) { logger, db, dummyDA, + config.DefaultConfig(), []byte("test-chain"), - 100*time.Millisecond, queueSize, genesis.Genesis{}, ) @@ -878,8 +879,8 @@ func TestSequencer_CheckpointPersistence_CrashRecovery(t *testing.T) { logger, db, mockDA, + config.DefaultConfig(), []byte("test-chain"), - 1*time.Second, 100, gen, ) @@ -918,8 +919,8 @@ func TestSequencer_CheckpointPersistence_CrashRecovery(t *testing.T) { logger, db, mockDA, + config.DefaultConfig(), []byte("test-chain"), - 1*time.Second, 100, gen, ) @@ -979,8 +980,8 @@ func TestSequencer_GetNextBatch_EmptyDABatch_IncreasesDAHeight(t *testing.T) { zerolog.Nop(), db, mockDA, + config.DefaultConfig(), []byte("test"), - 1*time.Second, 1000, gen, ) From 9480a1465a2380082ff287b1233cc4c6a2ca85a4 Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Thu, 8 Jan 2026 20:07:19 +0100 Subject: [PATCH 06/18] wip --- block/internal/da/async_block_fetcher.go | 444 ++++++++++++++++ block/internal/da/async_block_fetcher_test.go | 240 +++++++++ block/internal/da/async_epoch_fetcher.go | 495 ------------------ block/internal/da/async_epoch_fetcher_test.go | 257 --------- .../internal/da/forced_inclusion_retriever.go | 134 +++-- .../da/forced_inclusion_retriever_test.go | 18 +- block/internal/syncing/syncer.go | 9 +- .../syncing/syncer_forced_inclusion_test.go | 9 +- block/public.go | 25 +- pkg/sequencers/based/sequencer.go | 5 +- pkg/sequencers/single/sequencer.go | 13 +- 11 files changed, 803 insertions(+), 846 deletions(-) create mode 100644 block/internal/da/async_block_fetcher.go create mode 100644 block/internal/da/async_block_fetcher_test.go delete mode 100644 block/internal/da/async_epoch_fetcher.go delete mode 100644 block/internal/da/async_epoch_fetcher_test.go diff --git a/block/internal/da/async_block_fetcher.go b/block/internal/da/async_block_fetcher.go new file mode 100644 index 000000000..6f0aea3d5 --- /dev/null +++ b/block/internal/da/async_block_fetcher.go @@ -0,0 +1,444 @@ +package da + +import ( + "context" + "errors" + "fmt" + "sync" + "time" + + ds "github.com/ipfs/go-datastore" + dsq "github.com/ipfs/go-datastore/query" + dsync "github.com/ipfs/go-datastore/sync" + "github.com/rs/zerolog" + + "github.com/evstack/ev-node/pkg/config" + datypes "github.com/evstack/ev-node/pkg/da/types" +) + +// AsyncBlockFetcher provides background prefetching of DA blocks +type AsyncBlockFetcher interface { + Start() + Stop() + GetCachedBlock(ctx context.Context, daHeight uint64) (*BlockData, error) + UpdateCurrentHeight(height uint64) +} + +// BlockData contains data retrieved from a single DA height +type BlockData struct { + Height uint64 + Timestamp time.Time + Blobs [][]byte +} + +// asyncBlockFetcher handles background prefetching of individual DA blocks +// to speed up forced inclusion processing. +type asyncBlockFetcher struct { + client Client + logger zerolog.Logger + daStartHeight uint64 + + // In-memory cache for prefetched block data + cache ds.Batching + mu sync.RWMutex + + // Background fetcher control + ctx context.Context + cancel context.CancelFunc + wg sync.WaitGroup + + // Current DA height tracking + currentDAHeight uint64 + heightMu sync.RWMutex + + // Prefetch window - how many blocks ahead to prefetch + prefetchWindow uint64 + + // Polling interval for checking new DA heights + pollInterval time.Duration +} + +// NewAsyncBlockFetcher creates a new async block fetcher with in-memory cache. +func NewAsyncBlockFetcher( + client Client, + logger zerolog.Logger, + config config.Config, + daStartHeight uint64, + prefetchWindow uint64, +) AsyncBlockFetcher { + if prefetchWindow == 0 { + prefetchWindow = 10 // Default: prefetch next 10 blocks + } + + ctx, cancel := context.WithCancel(context.Background()) + + return &asyncBlockFetcher{ + client: client, + logger: logger.With().Str("component", "async_block_fetcher").Logger(), + daStartHeight: daStartHeight, + cache: dsync.MutexWrap(ds.NewMapDatastore()), + ctx: ctx, + cancel: cancel, + currentDAHeight: daStartHeight, + prefetchWindow: prefetchWindow, + pollInterval: config.DA.BlockTime.Duration, + } +} + +// Start begins the background prefetching process. +func (f *asyncBlockFetcher) Start() { + f.wg.Add(1) + go f.backgroundFetchLoop() + f.logger.Info(). + Uint64("da_start_height", f.daStartHeight). + Uint64("prefetch_window", f.prefetchWindow). + Dur("poll_interval", f.pollInterval). + Msg("async block fetcher started") +} + +// Stop gracefully stops the background prefetching process. +func (f *asyncBlockFetcher) Stop() { + f.logger.Info().Msg("stopping async block fetcher") + f.cancel() + f.wg.Wait() + f.logger.Info().Msg("async block fetcher stopped") +} + +// UpdateCurrentHeight updates the current DA height for prefetching. +func (f *asyncBlockFetcher) UpdateCurrentHeight(height uint64) { + f.heightMu.Lock() + defer f.heightMu.Unlock() + + if height > f.currentDAHeight { + f.currentDAHeight = height + f.logger.Debug(). + Uint64("new_height", height). + Msg("updated current DA height") + } +} + +// GetCachedBlock retrieves a cached block from memory. +// Returns nil if the block is not cached. +func (f *asyncBlockFetcher) GetCachedBlock(ctx context.Context, daHeight uint64) (*BlockData, error) { + if !f.client.HasForcedInclusionNamespace() { + return nil, ErrForceInclusionNotConfigured + } + + if daHeight < f.daStartHeight { + return nil, fmt.Errorf("DA height %d is before the configured start height %d", daHeight, f.daStartHeight) + } + + // Try to get from cache + key := ds.NewKey(fmt.Sprintf("/block/%d", daHeight)) + + f.mu.RLock() + data, err := f.cache.Get(ctx, key) + f.mu.RUnlock() + + if err != nil { + if errors.Is(err, ds.ErrNotFound) { + return nil, nil // Not cached yet + } + return nil, fmt.Errorf("failed to get cached block: %w", err) + } + + // Deserialize the cached block + block, err := deserializeBlockData(data) + if err != nil { + return nil, fmt.Errorf("failed to deserialize cached block: %w", err) + } + + f.logger.Debug(). + Uint64("da_height", daHeight). + Int("blob_count", len(block.Blobs)). + Msg("retrieved block from cache") + + return block, nil +} + +// backgroundFetchLoop runs in the background and prefetches blocks ahead of time. +func (f *asyncBlockFetcher) backgroundFetchLoop() { + defer f.wg.Done() + + ticker := time.NewTicker(f.pollInterval) + defer ticker.Stop() + + for { + select { + case <-f.ctx.Done(): + return + case <-ticker.C: + f.prefetchBlocks() + } + } +} + +// prefetchBlocks prefetches blocks within the prefetch window. +func (f *asyncBlockFetcher) prefetchBlocks() { + if !f.client.HasForcedInclusionNamespace() { + return + } + + f.heightMu.RLock() + currentHeight := f.currentDAHeight + f.heightMu.RUnlock() + + // Prefetch upcoming blocks + for i := uint64(0); i < f.prefetchWindow; i++ { + targetHeight := currentHeight + i + + // Check if already cached + key := ds.NewKey(fmt.Sprintf("/block/%d", targetHeight)) + f.mu.RLock() + _, err := f.cache.Get(f.ctx, key) + f.mu.RUnlock() + + if err == nil { + // Already cached + continue + } + + // Fetch and cache the block + f.fetchAndCacheBlock(targetHeight) + } + + // Clean up old blocks from cache to prevent memory growth + f.cleanupOldBlocks(currentHeight) +} + +// fetchAndCacheBlock fetches a block and stores it in the cache. +func (f *asyncBlockFetcher) fetchAndCacheBlock(height uint64) { + f.logger.Debug(). + Uint64("height", height). + Msg("prefetching block") + + result := f.client.Retrieve(f.ctx, height, f.client.GetForcedInclusionNamespace()) + + if result.Code == datypes.StatusHeightFromFuture { + f.logger.Debug(). + Uint64("height", height). + Msg("block height not yet available - will retry") + return + } + + block := &BlockData{ + Height: height, + Timestamp: result.Timestamp, + Blobs: [][]byte{}, + } + + if result.Code == datypes.StatusNotFound { + f.logger.Debug(). + Uint64("height", height). + Msg("no forced inclusion blobs at height") + // Cache empty result to avoid re-fetching + } else if result.Code == datypes.StatusSuccess { + // Process each blob + for _, blob := range result.Data { + if len(blob) > 0 { + block.Blobs = append(block.Blobs, blob) + } + } + f.logger.Debug(). + Uint64("height", height). + Int("blob_count", len(result.Data)). + Msg("processed forced inclusion blobs for prefetch") + } else { + f.logger.Warn(). + Uint64("height", height). + Str("status", result.Message). + Msg("failed to retrieve block - will retry") + return + } + + // Serialize and cache the block + data, err := serializeBlockData(block) + if err != nil { + f.logger.Error(). + Err(err). + Uint64("height", height). + Msg("failed to serialize block for caching") + return + } + + key := ds.NewKey(fmt.Sprintf("/block/%d", height)) + f.mu.Lock() + err = f.cache.Put(f.ctx, key, data) + f.mu.Unlock() + + if err != nil { + f.logger.Error(). + Err(err). + Uint64("height", height). + Msg("failed to cache block") + return + } + + f.logger.Info(). + Uint64("height", height). + Int("blob_count", len(block.Blobs)). + Msg("successfully prefetched and cached block") +} + +// cleanupOldBlocks removes blocks older than a threshold from cache. +func (f *asyncBlockFetcher) cleanupOldBlocks(currentHeight uint64) { + // Remove blocks older than current - prefetchWindow + // Keep some history in case of reorgs or restarts + if currentHeight < f.prefetchWindow { + return + } + + cleanupThreshold := currentHeight - f.prefetchWindow + + f.mu.Lock() + defer f.mu.Unlock() + + // Query all keys + query := dsq.Query{Prefix: "/block/"} + results, err := f.cache.Query(f.ctx, query) + if err != nil { + f.logger.Warn().Err(err).Msg("failed to query cache for cleanup") + return + } + defer results.Close() + + for result := range results.Next() { + if result.Error != nil { + continue + } + + key := ds.NewKey(result.Key) + // Extract height from key + var height uint64 + _, err := fmt.Sscanf(key.String(), "/block/%d", &height) + if err != nil { + continue + } + + if height < cleanupThreshold { + if err := f.cache.Delete(f.ctx, key); err != nil { + f.logger.Warn(). + Err(err). + Uint64("height", height). + Msg("failed to delete old block from cache") + } else { + f.logger.Debug(). + Uint64("height", height). + Msg("cleaned up old block from cache") + } + } + } +} + +// serializeBlockData serializes block data to bytes. +// Format: timestamp (int64) + height (uint64) + blobCount (uint32) + blobs +func serializeBlockData(block *BlockData) ([]byte, error) { + // Calculate total size + size := 8 + 8 + 4 // timestamp + height + blobCount + for _, blob := range block.Blobs { + size += 4 + len(blob) // blobLen + blob + } + + buf := make([]byte, size) + offset := 0 + + // Timestamp + writeUint64(buf[offset:], uint64(block.Timestamp.Unix())) + offset += 8 + + // Height + writeUint64(buf[offset:], block.Height) + offset += 8 + + // BlobCount + writeUint32(buf[offset:], uint32(len(block.Blobs))) + offset += 4 + + // Blobs + for _, blob := range block.Blobs { + writeUint32(buf[offset:], uint32(len(blob))) + offset += 4 + copy(buf[offset:], blob) + offset += len(blob) + } + + return buf, nil +} + +// deserializeBlockData deserializes bytes to block data. +func deserializeBlockData(data []byte) (*BlockData, error) { + if len(data) < 20 { + return nil, errors.New("invalid data: too short") + } + + offset := 0 + block := &BlockData{} + + // Timestamp + timestamp := readUint64(data[offset:]) + block.Timestamp = time.Unix(int64(timestamp), 0).UTC() + offset += 8 + + // Height + block.Height = readUint64(data[offset:]) + offset += 8 + + // BlobCount + blobCount := readUint32(data[offset:]) + offset += 4 + + // Blobs + block.Blobs = make([][]byte, blobCount) + for i := uint32(0); i < blobCount; i++ { + if offset+4 > len(data) { + return nil, errors.New("invalid data: unexpected end while reading blob length") + } + blobLen := readUint32(data[offset:]) + offset += 4 + + if offset+int(blobLen) > len(data) { + return nil, errors.New("invalid data: unexpected end while reading blob") + } + block.Blobs[i] = make([]byte, blobLen) + copy(block.Blobs[i], data[offset:offset+int(blobLen)]) + offset += int(blobLen) + } + + return block, nil +} + +func writeUint64(buf []byte, val uint64) { + buf[0] = byte(val) + buf[1] = byte(val >> 8) + buf[2] = byte(val >> 16) + buf[3] = byte(val >> 24) + buf[4] = byte(val >> 32) + buf[5] = byte(val >> 40) + buf[6] = byte(val >> 48) + buf[7] = byte(val >> 56) +} + +func readUint64(buf []byte) uint64 { + return uint64(buf[0]) | + uint64(buf[1])<<8 | + uint64(buf[2])<<16 | + uint64(buf[3])<<24 | + uint64(buf[4])<<32 | + uint64(buf[5])<<40 | + uint64(buf[6])<<48 | + uint64(buf[7])<<56 +} + +func writeUint32(buf []byte, val uint32) { + buf[0] = byte(val) + buf[1] = byte(val >> 8) + buf[2] = byte(val >> 16) + buf[3] = byte(val >> 24) +} + +func readUint32(buf []byte) uint32 { + return uint32(buf[0]) | + uint32(buf[1])<<8 | + uint32(buf[2])<<16 | + uint32(buf[3])<<24 +} diff --git a/block/internal/da/async_block_fetcher_test.go b/block/internal/da/async_block_fetcher_test.go new file mode 100644 index 000000000..24ee24049 --- /dev/null +++ b/block/internal/da/async_block_fetcher_test.go @@ -0,0 +1,240 @@ +package da + +import ( + "context" + "testing" + "time" + + "github.com/rs/zerolog" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + + "github.com/evstack/ev-node/pkg/config" + datypes "github.com/evstack/ev-node/pkg/da/types" + + mocks "github.com/evstack/ev-node/test/mocks" +) + +func TestAsyncBlockFetcher_GetCachedBlock_NoNamespace(t *testing.T) { + client := &mocks.MockClient{} + client.On("HasForcedInclusionNamespace").Return(false) + + logger := zerolog.Nop() + fetcher := NewAsyncBlockFetcher(client, logger, config.DefaultConfig(), 100, 10) + + ctx := context.Background() + _, err := fetcher.GetCachedBlock(ctx, 100) + assert.ErrorIs(t, err, ErrForceInclusionNotConfigured) +} + +func TestAsyncBlockFetcher_GetCachedBlock_CacheMiss(t *testing.T) { + client := &mocks.MockClient{} + fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() + client.On("HasForcedInclusionNamespace").Return(true) + client.On("GetForcedInclusionNamespace").Return(fiNs) + + logger := zerolog.Nop() + fetcher := NewAsyncBlockFetcher(client, logger, config.DefaultConfig(), 100, 10) + + ctx := context.Background() + + // Nothing cached yet + block, err := fetcher.GetCachedBlock(ctx, 100) + require.NoError(t, err) + assert.Nil(t, block) // Cache miss +} + +func TestAsyncBlockFetcher_FetchAndCache(t *testing.T) { + testBlobs := [][]byte{ + []byte("tx1"), + []byte("tx2"), + []byte("tx3"), + } + + client := &mocks.MockClient{} + fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() + client.On("HasForcedInclusionNamespace").Return(true) + client.On("GetForcedInclusionNamespace").Return(fiNs) + + // Mock Retrieve call for height 100 + client.On("Retrieve", mock.Anything, uint64(100), fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{ + Code: datypes.StatusSuccess, + Timestamp: time.Unix(1000, 0), + }, + Data: testBlobs, + }).Once() + + // Mock other heights that will be prefetched + for height := uint64(101); height <= 109; height++ { + client.On("Retrieve", mock.Anything, height, fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{Code: datypes.StatusNotFound}, + }).Maybe() + } + + logger := zerolog.Nop() + // Use a short poll interval for faster test execution + fetcher := NewAsyncBlockFetcher(client, logger, config.DefaultConfig(), 100, 10) + fetcher.Start() + defer fetcher.Stop() + + // Update current height to trigger prefetch + fetcher.UpdateCurrentHeight(100) + + // Wait for the background fetch to complete by polling the cache + ctx := context.Background() + var block *BlockData + var err error + + // Poll for up to 2 seconds for the block to be cached + for i := 0; i < 40; i++ { + block, err = fetcher.GetCachedBlock(ctx, 100) + require.NoError(t, err) + if block != nil { + break + } + time.Sleep(50 * time.Millisecond) + } + + require.NotNil(t, block, "block should be cached after background fetch") + assert.Equal(t, uint64(100), block.Height) + assert.Equal(t, 3, len(block.Blobs)) + assert.Equal(t, testBlobs[0], block.Blobs[0]) + assert.Equal(t, testBlobs[1], block.Blobs[1]) + assert.Equal(t, testBlobs[2], block.Blobs[2]) +} + +func TestAsyncBlockFetcher_BackgroundPrefetch(t *testing.T) { + testBlobs := [][]byte{ + []byte("tx1"), + []byte("tx2"), + } + + client := &mocks.MockClient{} + fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() + client.On("HasForcedInclusionNamespace").Return(true) + client.On("GetForcedInclusionNamespace").Return(fiNs) + + // Mock for heights 100-110 (current + prefetch window) + for height := uint64(100); height <= 110; height++ { + if height == 105 { + client.On("Retrieve", mock.Anything, height, fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{ + Code: datypes.StatusSuccess, + Timestamp: time.Unix(2000, 0), + }, + Data: testBlobs, + }).Maybe() + } else { + client.On("Retrieve", mock.Anything, height, fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{Code: datypes.StatusNotFound}, + }).Maybe() + } + } + + logger := zerolog.Nop() + fetcher := NewAsyncBlockFetcher(client, logger, config.DefaultConfig(), 100, 10) + + fetcher.Start() + defer fetcher.Stop() + + // Update current height to trigger prefetch + fetcher.UpdateCurrentHeight(100) + + // Wait for background prefetch to happen + time.Sleep(200 * time.Millisecond) + + // Check if block was prefetched + ctx := context.Background() + block, err := fetcher.GetCachedBlock(ctx, 105) + require.NoError(t, err) + assert.NotNil(t, block) + assert.Equal(t, uint64(105), block.Height) + assert.Equal(t, 2, len(block.Blobs)) +} + +func TestAsyncBlockFetcher_HeightFromFuture(t *testing.T) { + client := &mocks.MockClient{} + fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() + client.On("HasForcedInclusionNamespace").Return(true) + client.On("GetForcedInclusionNamespace").Return(fiNs) + + // Height not available yet + client.On("Retrieve", mock.Anything, uint64(100), fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{Code: datypes.StatusHeightFromFuture}, + }).Maybe() + + logger := zerolog.Nop() + fetcher := NewAsyncBlockFetcher(client, logger, config.DefaultConfig(), 100, 10) + fetcher.Start() + defer fetcher.Stop() + + fetcher.UpdateCurrentHeight(100) + + // Wait a bit + time.Sleep(100 * time.Millisecond) + + // Cache should be empty + ctx := context.Background() + block, err := fetcher.GetCachedBlock(ctx, 100) + require.NoError(t, err) + assert.Nil(t, block) +} + +func TestAsyncBlockFetcher_StopGracefully(t *testing.T) { + client := &mocks.MockClient{} + client.On("HasForcedInclusionNamespace").Return(false) + + logger := zerolog.Nop() + fetcher := NewAsyncBlockFetcher(client, logger, config.DefaultConfig(), 100, 10) + + fetcher.Start() + time.Sleep(100 * time.Millisecond) + + // Should stop gracefully without panic + fetcher.Stop() +} + +func TestBlockData_Serialization(t *testing.T) { + block := &BlockData{ + Height: 100, + Timestamp: time.Unix(12345, 0).UTC(), + Blobs: [][]byte{ + []byte("blob1"), + []byte("blob2"), + []byte("another_blob"), + }, + } + + // Serialize + data, err := serializeBlockData(block) + require.NoError(t, err) + assert.Greater(t, len(data), 0) + + // Deserialize + decoded, err := deserializeBlockData(data) + require.NoError(t, err) + assert.Equal(t, block.Timestamp.Unix(), decoded.Timestamp.Unix()) + assert.Equal(t, block.Height, decoded.Height) + assert.Equal(t, len(block.Blobs), len(decoded.Blobs)) + for i := range block.Blobs { + assert.Equal(t, block.Blobs[i], decoded.Blobs[i]) + } +} + +func TestBlockData_SerializationEmpty(t *testing.T) { + block := &BlockData{ + Height: 100, + Timestamp: time.Unix(0, 0).UTC(), + Blobs: [][]byte{}, + } + + data, err := serializeBlockData(block) + require.NoError(t, err) + + decoded, err := deserializeBlockData(data) + require.NoError(t, err) + assert.Equal(t, uint64(100), decoded.Height) + assert.Equal(t, 0, len(decoded.Blobs)) +} diff --git a/block/internal/da/async_epoch_fetcher.go b/block/internal/da/async_epoch_fetcher.go deleted file mode 100644 index cd039cfdd..000000000 --- a/block/internal/da/async_epoch_fetcher.go +++ /dev/null @@ -1,495 +0,0 @@ -package da - -import ( - "context" - "errors" - "fmt" - "sync" - "time" - - ds "github.com/ipfs/go-datastore" - dsq "github.com/ipfs/go-datastore/query" - dsync "github.com/ipfs/go-datastore/sync" - "github.com/rs/zerolog" - - datypes "github.com/evstack/ev-node/pkg/da/types" - "github.com/evstack/ev-node/types" -) - -// AsyncEpochFetcher provides background prefetching of DA epoch data -type AsyncEpochFetcher interface { - Start() - Stop() - GetCachedEpoch(ctx context.Context, daHeight uint64) (*ForcedInclusionEvent, error) -} - -// asyncEpochFetcher handles background prefetching of DA epoch data -// to speed up processing at epoch boundaries. -type asyncEpochFetcher struct { - client Client - logger zerolog.Logger - daEpochSize uint64 - daStartHeight uint64 - - // In-memory cache for prefetched epoch data - cache ds.Batching - mu sync.RWMutex - - // Background fetcher control - ctx context.Context - cancel context.CancelFunc - wg sync.WaitGroup - - // Current DA height tracking - currentDAHeight uint64 - - // Prefetch window - how many epochs ahead to prefetch - prefetchWindow uint64 - - // Polling interval for checking new DA heights - pollInterval time.Duration -} - -// NewAsyncEpochFetcher creates a new async epoch fetcher with in-memory cache. -func NewAsyncEpochFetcher( - client Client, - logger zerolog.Logger, - daStartHeight, daEpochSize uint64, - prefetchWindow uint64, - pollInterval time.Duration, -) AsyncEpochFetcher { - if prefetchWindow == 0 { - prefetchWindow = 1 // Default: prefetch next epoch - } - if pollInterval == 0 { - pollInterval = 2 * time.Second // Default polling interval - } - - ctx, cancel := context.WithCancel(context.Background()) - - return &asyncEpochFetcher{ - client: client, - logger: logger.With().Str("component", "async_epoch_fetcher").Logger(), - daStartHeight: daStartHeight, - daEpochSize: daEpochSize, - cache: dsync.MutexWrap(ds.NewMapDatastore()), - ctx: ctx, - cancel: cancel, - currentDAHeight: daStartHeight, - prefetchWindow: prefetchWindow, - pollInterval: pollInterval, - } -} - -// Start begins the background prefetching process. -func (f *asyncEpochFetcher) Start() { - f.wg.Add(1) - go f.backgroundFetchLoop() - f.logger.Info(). - Uint64("da_start_height", f.daStartHeight). - Uint64("da_epoch_size", f.daEpochSize). - Uint64("prefetch_window", f.prefetchWindow). - Dur("poll_interval", f.pollInterval). - Msg("async epoch fetcher started") -} - -// Stop gracefully stops the background prefetching process. -func (f *asyncEpochFetcher) Stop() { - f.logger.Info().Msg("stopping async epoch fetcher") - f.cancel() - f.wg.Wait() - f.logger.Info().Msg("async epoch fetcher stopped") -} - -// GetCachedEpoch retrieves a cached epoch from memory. -// Returns nil if the epoch is not cached. -func (f *asyncEpochFetcher) GetCachedEpoch(ctx context.Context, daHeight uint64) (*ForcedInclusionEvent, error) { - if !f.client.HasForcedInclusionNamespace() { - return nil, ErrForceInclusionNotConfigured - } - - if daHeight < f.daStartHeight { - return nil, fmt.Errorf("DA height %d is before the configured start height %d", daHeight, f.daStartHeight) - } - - epochStart, epochEnd, _ := types.CalculateEpochBoundaries(daHeight, f.daStartHeight, f.daEpochSize) - - // Only return cached data at epoch end - if daHeight != epochEnd { - return &ForcedInclusionEvent{ - StartDaHeight: daHeight, - EndDaHeight: daHeight, - Txs: [][]byte{}, - }, nil - } - - // Try to get from cache - key := ds.NewKey(fmt.Sprintf("/epoch/%d", epochEnd)) - - f.mu.RLock() - data, err := f.cache.Get(ctx, key) - f.mu.RUnlock() - - if err != nil { - if errors.Is(err, ds.ErrNotFound) { - return nil, nil // Not cached yet - } - return nil, fmt.Errorf("failed to get cached epoch: %w", err) - } - - // Deserialize the cached event - event, err := deserializeForcedInclusionEvent(data) - if err != nil { - return nil, fmt.Errorf("failed to deserialize cached epoch: %w", err) - } - - f.logger.Debug(). - Uint64("da_height", daHeight). - Uint64("epoch_start", epochStart). - Uint64("epoch_end", epochEnd). - Int("tx_count", len(event.Txs)). - Msg("retrieved epoch from cache") - - return event, nil -} - -// backgroundFetchLoop runs in the background and prefetches epochs ahead of time. -func (f *asyncEpochFetcher) backgroundFetchLoop() { - defer f.wg.Done() - - ticker := time.NewTicker(f.pollInterval) - defer ticker.Stop() - - for { - select { - case <-f.ctx.Done(): - return - case <-ticker.C: - f.prefetchEpochs() - } - } -} - -// prefetchEpochs prefetches epochs within the prefetch window. -func (f *asyncEpochFetcher) prefetchEpochs() { - if !f.client.HasForcedInclusionNamespace() { - return - } - - // Calculate the current epoch and epochs to prefetch - _, currentEpochEnd, _ := types.CalculateEpochBoundaries(f.currentDAHeight, f.daStartHeight, f.daEpochSize) - - // Prefetch upcoming epochs - for i := uint64(0); i < f.prefetchWindow; i++ { - targetEpochEnd := currentEpochEnd + (i * f.daEpochSize) - - // Check if already cached - key := ds.NewKey(fmt.Sprintf("/epoch/%d", targetEpochEnd)) - f.mu.RLock() - _, err := f.cache.Get(f.ctx, key) - f.mu.RUnlock() - - if err == nil { - // Already cached - continue - } - - // Fetch and cache the epoch - f.fetchAndCacheEpoch(targetEpochEnd) - } - - // Clean up old epochs from cache to prevent memory growth - f.cleanupOldEpochs(currentEpochEnd) -} - -// fetchAndCacheEpoch fetches an epoch and stores it in the cache. -func (f *asyncEpochFetcher) fetchAndCacheEpoch(epochEnd uint64) { - epochStart := max(epochEnd-(f.daEpochSize-1), f.daStartHeight) - f.logger.Debug(). - Uint64("epoch_start", epochStart). - Uint64("epoch_end", epochEnd). - Msg("prefetching epoch") - - event := &ForcedInclusionEvent{ - StartDaHeight: epochStart, - EndDaHeight: epochEnd, - Txs: [][]byte{}, - } - - // Fetch epoch end first to check availability - epochEndResult := f.client.Retrieve(f.ctx, epochEnd, f.client.GetForcedInclusionNamespace()) - if epochEndResult.Code == datypes.StatusHeightFromFuture { - f.logger.Debug(). - Uint64("epoch_end", epochEnd). - Msg("epoch end height not yet available - will retry") - return - } - - epochStartResult := epochEndResult - if epochStart != epochEnd { - epochStartResult = f.client.Retrieve(f.ctx, epochStart, f.client.GetForcedInclusionNamespace()) - if epochStartResult.Code == datypes.StatusHeightFromFuture { - f.logger.Debug(). - Uint64("epoch_start", epochStart). - Msg("epoch start height not yet available - will retry") - return - } - } - - // Process all heights in the epoch - var processErrs error - err := f.processForcedInclusionBlobs(event, epochStartResult, epochStart) - processErrs = errors.Join(processErrs, err) - - // Process heights between start and end (exclusive) - for epochHeight := epochStart + 1; epochHeight < epochEnd; epochHeight++ { - result := f.client.Retrieve(f.ctx, epochHeight, f.client.GetForcedInclusionNamespace()) - err = f.processForcedInclusionBlobs(event, result, epochHeight) - processErrs = errors.Join(processErrs, err) - } - - // Process epoch end (only if different from start) - if epochEnd != epochStart { - err = f.processForcedInclusionBlobs(event, epochEndResult, epochEnd) - processErrs = errors.Join(processErrs, err) - } - - if processErrs != nil { - f.logger.Warn(). - Uint64("epoch_start", epochStart). - Uint64("epoch_end", epochEnd). - Err(processErrs). - Msg("failed to prefetch epoch - will retry") - return - } - - // Serialize and cache the event - data, err := serializeForcedInclusionEvent(event) - if err != nil { - f.logger.Error(). - Err(err). - Uint64("epoch_end", epochEnd). - Msg("failed to serialize epoch for caching") - return - } - - key := ds.NewKey(fmt.Sprintf("/epoch/%d", epochEnd)) - f.mu.Lock() - err = f.cache.Put(f.ctx, key, data) - f.mu.Unlock() - - if err != nil { - f.logger.Error(). - Err(err). - Uint64("epoch_end", epochEnd). - Msg("failed to cache epoch") - return - } - - f.logger.Info(). - Uint64("epoch_start", epochStart). - Uint64("epoch_end", epochEnd). - Int("tx_count", len(event.Txs)). - Msg("successfully prefetched and cached epoch") -} - -// processForcedInclusionBlobs processes blobs from a single DA height for forced inclusion. -func (f *asyncEpochFetcher) processForcedInclusionBlobs( - event *ForcedInclusionEvent, - result datypes.ResultRetrieve, - height uint64, -) error { - if result.Code == datypes.StatusNotFound { - f.logger.Debug().Uint64("height", height).Msg("no forced inclusion blobs at height") - return nil - } - - if result.Code != datypes.StatusSuccess { - return fmt.Errorf("failed to retrieve forced inclusion blobs at height %d: %s", height, result.Message) - } - - // Process each blob as a transaction - for _, blob := range result.Data { - if len(blob) > 0 { - event.Txs = append(event.Txs, blob) - } - } - - if result.Timestamp.After(event.Timestamp) { - event.Timestamp = result.Timestamp - } - - f.logger.Debug(). - Uint64("height", height). - Int("blob_count", len(result.Data)). - Msg("processed forced inclusion blobs for prefetch") - - return nil -} - -// cleanupOldEpochs removes epochs older than the current epoch from cache. -func (f *asyncEpochFetcher) cleanupOldEpochs(currentEpochEnd uint64) { - // Remove epochs older than current - 1 - // Keep current and previous in case of reorgs or restarts - cleanupThreshold := currentEpochEnd - f.daEpochSize - - f.mu.Lock() - defer f.mu.Unlock() - - // Query all keys - query := dsq.Query{Prefix: "/epoch/"} - results, err := f.cache.Query(f.ctx, query) - if err != nil { - f.logger.Warn().Err(err).Msg("failed to query cache for cleanup") - return - } - defer results.Close() - - for result := range results.Next() { - if result.Error != nil { - continue - } - - key := ds.NewKey(result.Key) - // Extract epoch end from key - var epochEnd uint64 - _, err := fmt.Sscanf(key.String(), "/epoch/%d", &epochEnd) - if err != nil { - continue - } - - if epochEnd < cleanupThreshold { - if err := f.cache.Delete(f.ctx, key); err != nil { - f.logger.Warn(). - Err(err). - Uint64("epoch_end", epochEnd). - Msg("failed to delete old epoch from cache") - } else { - f.logger.Debug(). - Uint64("epoch_end", epochEnd). - Msg("cleaned up old epoch from cache") - } - } - } -} - -// serializeForcedInclusionEvent serializes an event to bytes. -// Simple format: timestamp (int64) + startHeight (uint64) + endHeight (uint64) + txCount (uint32) + txs -func serializeForcedInclusionEvent(event *ForcedInclusionEvent) ([]byte, error) { - // Calculate total size - size := 8 + 8 + 8 + 4 // timestamp + startHeight + endHeight + txCount - for _, tx := range event.Txs { - size += 4 + len(tx) // txLen + tx - } - - buf := make([]byte, size) - offset := 0 - - // Timestamp - writeUint64(buf[offset:], uint64(event.Timestamp.Unix())) - offset += 8 - - // StartDaHeight - writeUint64(buf[offset:], event.StartDaHeight) - offset += 8 - - // EndDaHeight - writeUint64(buf[offset:], event.EndDaHeight) - offset += 8 - - // TxCount - writeUint32(buf[offset:], uint32(len(event.Txs))) - offset += 4 - - // Txs - for _, tx := range event.Txs { - writeUint32(buf[offset:], uint32(len(tx))) - offset += 4 - copy(buf[offset:], tx) - offset += len(tx) - } - - return buf, nil -} - -// deserializeForcedInclusionEvent deserializes bytes to an event. -func deserializeForcedInclusionEvent(data []byte) (*ForcedInclusionEvent, error) { - if len(data) < 28 { - return nil, errors.New("invalid data: too short") - } - - offset := 0 - event := &ForcedInclusionEvent{} - - // Timestamp - timestamp := readUint64(data[offset:]) - event.Timestamp = time.Unix(int64(timestamp), 0).UTC() - offset += 8 - - // StartDaHeight - event.StartDaHeight = readUint64(data[offset:]) - offset += 8 - - // EndDaHeight - event.EndDaHeight = readUint64(data[offset:]) - offset += 8 - - // TxCount - txCount := readUint32(data[offset:]) - offset += 4 - - // Txs - event.Txs = make([][]byte, txCount) - for i := uint32(0); i < txCount; i++ { - if offset+4 > len(data) { - return nil, errors.New("invalid data: unexpected end while reading tx length") - } - txLen := readUint32(data[offset:]) - offset += 4 - - if offset+int(txLen) > len(data) { - return nil, errors.New("invalid data: unexpected end while reading tx") - } - event.Txs[i] = make([]byte, txLen) - copy(event.Txs[i], data[offset:offset+int(txLen)]) - offset += int(txLen) - } - - return event, nil -} - -func writeUint64(buf []byte, val uint64) { - buf[0] = byte(val) - buf[1] = byte(val >> 8) - buf[2] = byte(val >> 16) - buf[3] = byte(val >> 24) - buf[4] = byte(val >> 32) - buf[5] = byte(val >> 40) - buf[6] = byte(val >> 48) - buf[7] = byte(val >> 56) -} - -func readUint64(buf []byte) uint64 { - return uint64(buf[0]) | - uint64(buf[1])<<8 | - uint64(buf[2])<<16 | - uint64(buf[3])<<24 | - uint64(buf[4])<<32 | - uint64(buf[5])<<40 | - uint64(buf[6])<<48 | - uint64(buf[7])<<56 -} - -func writeUint32(buf []byte, val uint32) { - buf[0] = byte(val) - buf[1] = byte(val >> 8) - buf[2] = byte(val >> 16) - buf[3] = byte(val >> 24) -} - -func readUint32(buf []byte) uint32 { - return uint32(buf[0]) | - uint32(buf[1])<<8 | - uint32(buf[2])<<16 | - uint32(buf[3])<<24 -} diff --git a/block/internal/da/async_epoch_fetcher_test.go b/block/internal/da/async_epoch_fetcher_test.go deleted file mode 100644 index 6f485a595..000000000 --- a/block/internal/da/async_epoch_fetcher_test.go +++ /dev/null @@ -1,257 +0,0 @@ -package da - -import ( - "context" - "testing" - "time" - - "github.com/rs/zerolog" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - - datypes "github.com/evstack/ev-node/pkg/da/types" - - mocks "github.com/evstack/ev-node/test/mocks" -) - -func TestAsyncEpochFetcher_GetCachedEpoch_NotAtEpochEnd(t *testing.T) { - client := &mocks.MockClient{} - fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() - client.On("HasForcedInclusionNamespace").Return(true) - client.On("GetForcedInclusionNamespace").Return(fiNs) - - logger := zerolog.Nop() - fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, time.Second) - - ctx := context.Background() - - // Height 105 is not an epoch end (100, 109, 118, etc. are epoch ends for size 10) - event, err := fetcher.GetCachedEpoch(ctx, 105) - require.NoError(t, err) - assert.NotNil(t, event) - assert.Equal(t, 0, len(event.Txs)) - assert.Equal(t, uint64(105), event.StartDaHeight) - assert.Equal(t, uint64(105), event.EndDaHeight) -} - -func TestAsyncEpochFetcher_GetCachedEpoch_NoNamespace(t *testing.T) { - client := &mocks.MockClient{} - client.On("HasForcedInclusionNamespace").Return(false) - - logger := zerolog.Nop() - fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, time.Second) - - ctx := context.Background() - _, err := fetcher.GetCachedEpoch(ctx, 109) - assert.ErrorIs(t, err, ErrForceInclusionNotConfigured) -} - -func TestAsyncEpochFetcher_GetCachedEpoch_CacheMiss(t *testing.T) { - client := &mocks.MockClient{} - fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() - client.On("HasForcedInclusionNamespace").Return(true) - client.On("GetForcedInclusionNamespace").Return(fiNs) - - logger := zerolog.Nop() - fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, time.Second) - - ctx := context.Background() - - // Epoch end at 109, but nothing cached - event, err := fetcher.GetCachedEpoch(ctx, 109) - require.NoError(t, err) - assert.Nil(t, event) // Cache miss -} - -func TestAsyncEpochFetcher_FetchAndCache(t *testing.T) { - testBlobs := [][]byte{ - []byte("tx1"), - []byte("tx2"), - []byte("tx3"), - } - - client := &mocks.MockClient{} - fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() - client.On("HasForcedInclusionNamespace").Return(true) - client.On("GetForcedInclusionNamespace").Return(fiNs) - - // Mock Retrieve calls for epoch [100, 109] - for height := uint64(100); height <= 109; height++ { - if height == 100 { - client.On("Retrieve", mock.Anything, height, fiNs).Return(datypes.ResultRetrieve{ - BaseResult: datypes.BaseResult{ - Code: datypes.StatusSuccess, - Timestamp: time.Unix(1000, 0), - }, - Data: testBlobs, - }).Once() - } else { - client.On("Retrieve", mock.Anything, height, fiNs).Return(datypes.ResultRetrieve{ - BaseResult: datypes.BaseResult{Code: datypes.StatusNotFound}, - }).Once() - } - } - - logger := zerolog.Nop() - // Use a short poll interval for faster test execution - fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, 50*time.Millisecond) - fetcher.Start() - defer fetcher.Stop() - - // Wait for the background fetch to complete by polling the cache - ctx := context.Background() - var event *ForcedInclusionEvent - var err error - - // Poll for up to 2 seconds for the epoch to be cached - for i := 0; i < 40; i++ { - event, err = fetcher.GetCachedEpoch(ctx, 109) - require.NoError(t, err) - if event != nil { - break - } - time.Sleep(50 * time.Millisecond) - } - - require.NotNil(t, event, "epoch should be cached after background fetch") - assert.Equal(t, 3, len(event.Txs)) - assert.Equal(t, testBlobs[0], event.Txs[0]) //nolint:gosec // false positive - assert.Equal(t, uint64(100), event.StartDaHeight) - assert.Equal(t, uint64(109), event.EndDaHeight) -} - -func TestAsyncEpochFetcher_BackgroundPrefetch(t *testing.T) { - testBlobs := [][]byte{ - []byte("tx1"), - []byte("tx2"), - } - - client := &mocks.MockClient{} - fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() - client.On("HasForcedInclusionNamespace").Return(true) - client.On("GetForcedInclusionNamespace").Return(fiNs) - - // Mock for current epoch [100, 109] - for height := uint64(100); height <= 109; height++ { - if height == 105 { - client.On("Retrieve", mock.Anything, height, fiNs).Return(datypes.ResultRetrieve{ - BaseResult: datypes.BaseResult{ - Code: datypes.StatusSuccess, - Timestamp: time.Unix(2000, 0), - }, - Data: testBlobs, - }).Maybe() - } else { - client.On("Retrieve", mock.Anything, height, fiNs).Return(datypes.ResultRetrieve{ - BaseResult: datypes.BaseResult{Code: datypes.StatusNotFound}, - }).Maybe() - } - } - - // Mock for next epoch [110, 119] - for height := uint64(110); height <= 119; height++ { - client.On("Retrieve", mock.Anything, height, fiNs).Return(datypes.ResultRetrieve{ - BaseResult: datypes.BaseResult{Code: datypes.StatusNotFound}, - }).Maybe() - } - - logger := zerolog.Nop() - fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, 50*time.Millisecond) - - fetcher.Start() - defer fetcher.Stop() - - // Wait for background prefetch to happen - time.Sleep(200 * time.Millisecond) - - // Check if epoch was prefetched - ctx := context.Background() - event, err := fetcher.GetCachedEpoch(ctx, 109) - require.NoError(t, err) - assert.NotNil(t, event) - assert.Equal(t, 2, len(event.Txs)) -} - -func TestAsyncEpochFetcher_Serialization(t *testing.T) { - event := &ForcedInclusionEvent{ - Timestamp: time.Unix(12345, 0).UTC(), - StartDaHeight: 100, - EndDaHeight: 109, - Txs: [][]byte{ - []byte("transaction1"), - []byte("tx2"), - []byte("another_transaction"), - }, - } - - // Serialize - data, err := serializeForcedInclusionEvent(event) - require.NoError(t, err) - assert.Greater(t, len(data), 0) - - // Deserialize - decoded, err := deserializeForcedInclusionEvent(data) - require.NoError(t, err) - assert.Equal(t, event.Timestamp.Unix(), decoded.Timestamp.Unix()) - assert.Equal(t, event.StartDaHeight, decoded.StartDaHeight) - assert.Equal(t, event.EndDaHeight, decoded.EndDaHeight) - assert.Equal(t, len(event.Txs), len(decoded.Txs)) - for i := range event.Txs { - assert.Equal(t, event.Txs[i], decoded.Txs[i]) - } -} - -func TestAsyncEpochFetcher_SerializationEmpty(t *testing.T) { - event := &ForcedInclusionEvent{ - Timestamp: time.Unix(0, 0).UTC(), - StartDaHeight: 100, - EndDaHeight: 100, - Txs: [][]byte{}, - } - - data, err := serializeForcedInclusionEvent(event) - require.NoError(t, err) - - decoded, err := deserializeForcedInclusionEvent(data) - require.NoError(t, err) - assert.Equal(t, uint64(100), decoded.StartDaHeight) - assert.Equal(t, 0, len(decoded.Txs)) -} - -func TestAsyncEpochFetcher_HeightFromFuture(t *testing.T) { - client := &mocks.MockClient{} - fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() - client.On("HasForcedInclusionNamespace").Return(true) - client.On("GetForcedInclusionNamespace").Return(fiNs) - - // Epoch end not available yet - client.On("Retrieve", mock.Anything, uint64(109), fiNs).Return(datypes.ResultRetrieve{ - BaseResult: datypes.BaseResult{Code: datypes.StatusHeightFromFuture}, - }).Once() - - logger := zerolog.Nop() - fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, time.Second) - fetcher.Start() - defer fetcher.Stop() - - // Cache should be empty - ctx := context.Background() - event, err := fetcher.GetCachedEpoch(ctx, 109) - require.NoError(t, err) - assert.Nil(t, event) -} - -func TestAsyncEpochFetcher_StopGracefully(t *testing.T) { - client := &mocks.MockClient{} - client.On("HasForcedInclusionNamespace").Return(false) - - logger := zerolog.Nop() - fetcher := NewAsyncEpochFetcher(client, logger, 100, 10, 1, 50*time.Millisecond) - - fetcher.Start() - time.Sleep(100 * time.Millisecond) - - // Should stop gracefully without panic - fetcher.Stop() -} diff --git a/block/internal/da/forced_inclusion_retriever.go b/block/internal/da/forced_inclusion_retriever.go index d146477f7..cd477aa48 100644 --- a/block/internal/da/forced_inclusion_retriever.go +++ b/block/internal/da/forced_inclusion_retriever.go @@ -21,7 +21,7 @@ type ForcedInclusionRetriever struct { logger zerolog.Logger daEpochSize uint64 daStartHeight uint64 - asyncFetcher AsyncEpochFetcher // Required for async prefetching + asyncFetcher AsyncBlockFetcher // Fetches individual blocks in background } // ForcedInclusionEvent contains forced inclusion transactions retrieved from DA. @@ -37,7 +37,7 @@ func NewForcedInclusionRetriever( client Client, logger zerolog.Logger, daStartHeight, daEpochSize uint64, - asyncFetcher AsyncEpochFetcher, + asyncFetcher AsyncBlockFetcher, ) *ForcedInclusionRetriever { return &ForcedInclusionRetriever{ client: client, @@ -49,7 +49,8 @@ func NewForcedInclusionRetriever( } // RetrieveForcedIncludedTxs retrieves forced inclusion transactions at the given DA height. -// It respects epoch boundaries and only fetches at epoch start. +// It respects epoch boundaries and only fetches at epoch end. +// It tries to get blocks from the async fetcher cache first, then falls back to sync fetching. func (r *ForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Context, daHeight uint64) (*ForcedInclusionEvent, error) { // when daStartHeight is not set or no namespace is configured, we retrieve nothing. if !r.client.HasForcedInclusionNamespace() { @@ -62,6 +63,9 @@ func (r *ForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Context epochStart, epochEnd, currentEpochNumber := types.CalculateEpochBoundaries(daHeight, r.daStartHeight, r.daEpochSize) + // Update the async fetcher's current height so it knows what to prefetch + r.asyncFetcher.UpdateCurrentHeight(daHeight) + if daHeight != epochEnd { r.logger.Debug(). Uint64("da_height", daHeight). @@ -75,24 +79,12 @@ func (r *ForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Context }, nil } - // Try to get from async fetcher cache first - cachedEvent, err := r.asyncFetcher.GetCachedEpoch(ctx, daHeight) - if err == nil && cachedEvent != nil { - r.logger.Debug(). - Uint64("da_height", daHeight). - Uint64("epoch_start", epochStart). - Uint64("epoch_end", epochEnd). - Int("tx_count", len(cachedEvent.Txs)). - Msg("using cached epoch data from async fetcher") - return cachedEvent, nil - } - // Cache miss or error, fall through to sync fetch - if err != nil { - r.logger.Debug(). - Err(err). - Uint64("da_height", daHeight). - Msg("failed to get cached epoch, falling back to sync fetch") - } + r.logger.Debug(). + Uint64("da_height", daHeight). + Uint64("epoch_start", epochStart). + Uint64("epoch_end", epochEnd). + Uint64("epoch_num", currentEpochNumber). + Msg("retrieving forced included transactions from DA epoch") event := &ForcedInclusionEvent{ StartDaHeight: epochStart, @@ -100,48 +92,75 @@ func (r *ForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Context Txs: [][]byte{}, } - epochEndResult := r.client.Retrieve(ctx, epochEnd, r.client.GetForcedInclusionNamespace()) - if epochEndResult.Code == datypes.StatusHeightFromFuture { - r.logger.Debug(). - Uint64("epoch_end", epochEnd). - Msg("epoch end height not yet available on DA - backoff required") - return nil, fmt.Errorf("%w: epoch end height %d not yet available", datypes.ErrHeightFromFuture, epochEnd) + // Collect all heights in this epoch + var heights []uint64 + for h := epochStart; h <= epochEnd; h++ { + heights = append(heights, h) } - epochStartResult := epochEndResult - if epochStart != epochEnd { - epochStartResult = r.client.Retrieve(ctx, epochStart, r.client.GetForcedInclusionNamespace()) - if epochStartResult.Code == datypes.StatusHeightFromFuture { + // Try to get blocks from cache first + cachedBlocks := make(map[uint64]*BlockData) + var missingHeights []uint64 + + for _, h := range heights { + block, err := r.asyncFetcher.GetCachedBlock(ctx, h) + if err != nil { + r.logger.Debug(). + Err(err). + Uint64("height", h). + Msg("error getting cached block, will fetch synchronously") + missingHeights = append(missingHeights, h) + continue + } + if block == nil { + // Cache miss + missingHeights = append(missingHeights, h) + } else { + // Cache hit + cachedBlocks[h] = block r.logger.Debug(). - Uint64("epoch_start", epochStart). - Msg("epoch start height not yet available on DA - backoff required") - return nil, fmt.Errorf("%w: epoch start height %d not yet available", datypes.ErrHeightFromFuture, epochStart) + Uint64("height", h). + Int("blob_count", len(block.Blobs)). + Msg("using cached block from async fetcher") } } - r.logger.Debug(). - Uint64("da_height", daHeight). - Uint64("epoch_start", epochStart). - Uint64("epoch_end", epochEnd). - Uint64("epoch_num", currentEpochNumber). - Msg("retrieving forced included transactions from DA") - + // Fetch missing heights synchronously var processErrs error - err = r.processForcedInclusionBlobs(event, epochStartResult, epochStart) - processErrs = errors.Join(processErrs, err) + for _, h := range missingHeights { + result := r.client.Retrieve(ctx, h, r.client.GetForcedInclusionNamespace()) - // Process heights between start and end (exclusive) - for epochHeight := epochStart + 1; epochHeight < epochEnd; epochHeight++ { - result := r.client.Retrieve(ctx, epochHeight, r.client.GetForcedInclusionNamespace()) + if result.Code == datypes.StatusHeightFromFuture { + r.logger.Debug(). + Uint64("height", h). + Msg("height not yet available on DA - backoff required") + return nil, fmt.Errorf("%w: height %d not yet available", datypes.ErrHeightFromFuture, h) + } - err = r.processForcedInclusionBlobs(event, result, epochHeight) + err := r.processRetrieveResult(event, result, h) processErrs = errors.Join(processErrs, err) } - // Process epoch end (only if different from start) - if epochEnd != epochStart { - err = r.processForcedInclusionBlobs(event, epochEndResult, epochEnd) - processErrs = errors.Join(processErrs, err) + // Process cached blocks in order + for _, h := range heights { + if block, ok := cachedBlocks[h]; ok { + // Add blobs from cached block + for _, blob := range block.Blobs { + if len(blob) > 0 { + event.Txs = append(event.Txs, blob) + } + } + + // Update timestamp if newer + if block.Timestamp.After(event.Timestamp) { + event.Timestamp = block.Timestamp + } + + r.logger.Debug(). + Uint64("height", h). + Int("blob_count", len(block.Blobs)). + Msg("added blobs from cached block") + } } // any error during process, need to retry at next call @@ -161,11 +180,20 @@ func (r *ForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Context }, nil } + r.logger.Info(). + Uint64("da_height", daHeight). + Uint64("epoch_start", epochStart). + Uint64("epoch_end", epochEnd). + Int("tx_count", len(event.Txs)). + Int("cached_blocks", len(cachedBlocks)). + Int("sync_fetched_blocks", len(missingHeights)). + Msg("successfully retrieved forced inclusion epoch") + return event, nil } -// processForcedInclusionBlobs processes blobs from a single DA height for forced inclusion. -func (r *ForcedInclusionRetriever) processForcedInclusionBlobs( +// processRetrieveResult processes the result from a DA retrieve operation. +func (r *ForcedInclusionRetriever) processRetrieveResult( event *ForcedInclusionEvent, result datypes.ResultRetrieve, height uint64, diff --git a/block/internal/da/forced_inclusion_retriever_test.go b/block/internal/da/forced_inclusion_retriever_test.go index eb8e6f7fd..34adfeb76 100644 --- a/block/internal/da/forced_inclusion_retriever_test.go +++ b/block/internal/da/forced_inclusion_retriever_test.go @@ -9,20 +9,20 @@ import ( "github.com/stretchr/testify/mock" "gotest.tools/v3/assert" + "github.com/evstack/ev-node/pkg/config" datypes "github.com/evstack/ev-node/pkg/da/types" "github.com/evstack/ev-node/pkg/genesis" "github.com/evstack/ev-node/test/mocks" ) // createTestAsyncFetcher creates a minimal async fetcher for tests -func createTestAsyncFetcher(client Client, gen genesis.Genesis) AsyncEpochFetcher { - return NewAsyncEpochFetcher( +func createTestAsyncFetcher(client Client, gen genesis.Genesis) AsyncBlockFetcher { + return NewAsyncBlockFetcher( client, zerolog.Nop(), + config.DefaultConfig(), gen.DAStartHeight, - gen.DAEpochForcedInclusion, - 1, // prefetch 1 epoch - 1*time.Second, // poll interval (doesn't matter for tests) + 10, // prefetch 10 blocks ) } @@ -123,7 +123,9 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_EpochStartNotAvailab fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() client.On("HasForcedInclusionNamespace").Return(true).Maybe() client.On("GetForcedInclusionNamespace").Return(fiNs).Maybe() - client.On("Retrieve", mock.Anything, uint64(109), fiNs).Return(datypes.ResultRetrieve{ + + // Mock the first height in epoch as not available + client.On("Retrieve", mock.Anything, uint64(100), fiNs).Return(datypes.ResultRetrieve{ BaseResult: datypes.BaseResult{Code: datypes.StatusHeightFromFuture}, }).Once() @@ -136,7 +138,7 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_EpochStartNotAvailab retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) ctx := context.Background() - // Epoch boundaries: [100, 109] - retrieval happens at epoch end (109) + // Epoch boundaries: [100, 109] - now tries to fetch all blocks in epoch _, err := retriever.RetrieveForcedIncludedTxs(ctx, 109) assert.Assert(t, err != nil) assert.ErrorContains(t, err, "not yet available") @@ -284,7 +286,7 @@ func TestForcedInclusionRetriever_processForcedInclusionBlobs(t *testing.T) { Txs: [][]byte{}, } - err := retriever.processForcedInclusionBlobs(event, tt.result, tt.height) + err := retriever.processRetrieveResult(event, tt.result, tt.height) if tt.expectError { assert.Assert(t, err != nil) diff --git a/block/internal/syncing/syncer.go b/block/internal/syncing/syncer.go index 4ff5c54ab..454da774b 100644 --- a/block/internal/syncing/syncer.go +++ b/block/internal/syncing/syncer.go @@ -188,14 +188,13 @@ func (s *Syncer) Start(ctx context.Context) error { // Initialize handlers s.daRetriever = NewDARetriever(s.daClient, s.cache, s.genesis, s.logger) - // Create async epoch fetcher for background prefetching - asyncFetcher := da.NewAsyncEpochFetcher( + // Create async block fetcher for background prefetching + asyncFetcher := da.NewAsyncBlockFetcher( s.daClient, s.logger, + s.config, s.genesis.DAStartHeight, - s.genesis.DAEpochForcedInclusion, - 1, // prefetch 1 epoch ahead - s.config.DA.BlockTime.Duration, + s.genesis.DAEpochForcedInclusion, // prefetch window: blocks ahead to prefetch ) asyncFetcher.Start() diff --git a/block/internal/syncing/syncer_forced_inclusion_test.go b/block/internal/syncing/syncer_forced_inclusion_test.go index 4c1967b61..d0a97d7a5 100644 --- a/block/internal/syncing/syncer_forced_inclusion_test.go +++ b/block/internal/syncing/syncer_forced_inclusion_test.go @@ -24,14 +24,13 @@ import ( ) // createTestAsyncFetcher creates a minimal async fetcher for tests (without starting it) -func createTestAsyncFetcherForSyncer(client da.Client, gen genesis.Genesis) da.AsyncEpochFetcher { - return da.NewAsyncEpochFetcher( +func createTestAsyncFetcherForSyncer(client da.Client, gen genesis.Genesis) da.AsyncBlockFetcher { + return da.NewAsyncBlockFetcher( client, zerolog.Nop(), + config.DefaultConfig(), gen.DAStartHeight, - gen.DAEpochForcedInclusion, - 1, // prefetch 1 epoch - 1*time.Second, // poll interval (doesn't matter for tests) + 10, // prefetch 10 blocks ) } diff --git a/block/public.go b/block/public.go index 67b85d482..2d59e2461 100644 --- a/block/public.go +++ b/block/public.go @@ -65,11 +65,12 @@ var ErrForceInclusionNotConfigured = da.ErrForceInclusionNotConfigured // ForcedInclusionEvent represents forced inclusion transactions retrieved from DA type ForcedInclusionEvent = da.ForcedInclusionEvent -// AsyncEpochFetcher provides background prefetching of DA epoch data -type AsyncEpochFetcher interface { +// AsyncBlockFetcher provides background prefetching of individual DA blocks +type AsyncBlockFetcher interface { Start() Stop() - GetCachedEpoch(ctx context.Context, daHeight uint64) (*da.ForcedInclusionEvent, error) + GetCachedBlock(ctx context.Context, daHeight uint64) (*da.BlockData, error) + UpdateCurrentHeight(height uint64) } // ForcedInclusionRetriever defines the interface for retrieving forced inclusion transactions from DA @@ -77,31 +78,31 @@ type ForcedInclusionRetriever interface { RetrieveForcedIncludedTxs(ctx context.Context, daHeight uint64) (*da.ForcedInclusionEvent, error) } -// NewAsyncEpochFetcher creates a new async epoch fetcher for background prefetching. +// NewAsyncBlockFetcher creates a new async block fetcher for background prefetching. // Parameters: // - client: DA client for fetching data // - config: Ev-node config // - logger: structured logger // - daStartHeight: genesis DA start height -// - daEpochSize: number of DA blocks per epoch -// - prefetchWindow: how many epochs ahead to prefetch (1-2 recommended) -func NewAsyncEpochFetcher( +// - prefetchWindow: how many blocks ahead to prefetch (10-20 recommended) +func NewAsyncBlockFetcher( client DAClient, cfg config.Config, logger zerolog.Logger, - daStartHeight, daEpochSize uint64, + daStartHeight uint64, prefetchWindow uint64, -) AsyncEpochFetcher { - return da.NewAsyncEpochFetcher(client, logger, daStartHeight, daEpochSize, prefetchWindow, cfg.DA.BlockTime.Duration) +) AsyncBlockFetcher { + return da.NewAsyncBlockFetcher(client, logger, cfg, daStartHeight, prefetchWindow) } // NewForcedInclusionRetriever creates a new forced inclusion retriever. -// The asyncFetcher parameter is required for background prefetching of DA epoch data. +// The asyncFetcher parameter is required for background prefetching of DA block data. +// It accepts either AsyncBlockFetcher (recommended) or AsyncEpochFetcher (deprecated) for backward compatibility. func NewForcedInclusionRetriever( client DAClient, logger zerolog.Logger, daStartHeight, daEpochSize uint64, - asyncFetcher AsyncEpochFetcher, + asyncFetcher AsyncBlockFetcher, ) ForcedInclusionRetriever { return da.NewForcedInclusionRetriever(client, logger, daStartHeight, daEpochSize, asyncFetcher) } diff --git a/pkg/sequencers/based/sequencer.go b/pkg/sequencers/based/sequencer.go index 64e514f62..9f947589f 100644 --- a/pkg/sequencers/based/sequencer.go +++ b/pkg/sequencers/based/sequencer.go @@ -26,7 +26,7 @@ var _ coresequencer.Sequencer = (*BasedSequencer)(nil) type BasedSequencer struct { logger zerolog.Logger - asyncFetcher block.AsyncEpochFetcher + asyncFetcher block.AsyncBlockFetcher fiRetriever block.ForcedInclusionRetriever daHeight atomic.Uint64 checkpointStore *seqcommon.CheckpointStore @@ -81,13 +81,12 @@ func NewBasedSequencer( } // Create async epoch fetcher for background prefetching (created once) - bs.asyncFetcher = block.NewAsyncEpochFetcher( + bs.asyncFetcher = block.NewAsyncBlockFetcher( daClient, cfg, logger, genesis.DAStartHeight, genesis.DAEpochForcedInclusion, - 2, // prefetch 2 epochs ahead for based sequencer ) bs.asyncFetcher.Start() diff --git a/pkg/sequencers/single/sequencer.go b/pkg/sequencers/single/sequencer.go index c0fafdc41..91499405b 100644 --- a/pkg/sequencers/single/sequencer.go +++ b/pkg/sequencers/single/sequencer.go @@ -41,7 +41,7 @@ type Sequencer struct { queue *BatchQueue // single queue for immediate availability // Forced inclusion support - asyncFetcher block.AsyncEpochFetcher + asyncFetcher block.AsyncBlockFetcher fiRetriever block.ForcedInclusionRetriever daHeight atomic.Uint64 daStartHeight atomic.Uint64 @@ -109,18 +109,12 @@ func NewSequencer( // Determine initial DA height for forced inclusion initialDAHeight := s.getInitialDAStartHeight(context.Background()) - if initialDAHeight == 0 { - initialDAHeight = s.GetDAHeight() - } - - // Create async epoch fetcher for background prefetching (created once) - s.asyncFetcher = block.NewAsyncEpochFetcher( + s.asyncFetcher = block.NewAsyncBlockFetcher( daClient, cfg, logger, initialDAHeight, genesis.DAEpochForcedInclusion, - 1, // prefetch 1 epoch ahead ) s.asyncFetcher.Start() @@ -193,6 +187,9 @@ func (c *Sequencer) GetNextBatch(ctx context.Context, req coresequencer.GetNextB } } + // override forced inclusion retriever, as the da start height could have been updated + c.fiRetriever = block.NewForcedInclusionRetriever(c.daClient, c.logger, c.getInitialDAStartHeight(ctx), c.genesis.DAEpochForcedInclusion, c.asyncFetcher) + // If we have no cached transactions or we've consumed all from the current cache, // fetch the next DA epoch if daHeight > 0 && (len(c.cachedForcedInclusionTxs) == 0 || c.checkpoint.TxIndex >= uint64(len(c.cachedForcedInclusionTxs))) { From c5f4e0faf5ece664a1890d7ebb7f63888bb73462 Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Thu, 8 Jan 2026 20:24:43 +0100 Subject: [PATCH 07/18] updates --- block/internal/da/async_block_fetcher.go | 155 +++--------------- block/internal/da/async_block_fetcher_test.go | 73 +++++++-- proto/evnode/v1/da.proto | 11 ++ types/pb/evnode/v1/da.pb.go | 141 ++++++++++++++++ 4 files changed, 235 insertions(+), 145 deletions(-) create mode 100644 proto/evnode/v1/da.proto create mode 100644 types/pb/evnode/v1/da.pb.go diff --git a/block/internal/da/async_block_fetcher.go b/block/internal/da/async_block_fetcher.go index 6f0aea3d5..f97c93bab 100644 --- a/block/internal/da/async_block_fetcher.go +++ b/block/internal/da/async_block_fetcher.go @@ -11,9 +11,11 @@ import ( dsq "github.com/ipfs/go-datastore/query" dsync "github.com/ipfs/go-datastore/sync" "github.com/rs/zerolog" + "google.golang.org/protobuf/proto" "github.com/evstack/ev-node/pkg/config" datypes "github.com/evstack/ev-node/pkg/da/types" + pb "github.com/evstack/ev-node/types/pb/evnode/v1" ) // AsyncBlockFetcher provides background prefetching of DA blocks @@ -143,9 +145,15 @@ func (f *asyncBlockFetcher) GetCachedBlock(ctx context.Context, daHeight uint64) } // Deserialize the cached block - block, err := deserializeBlockData(data) - if err != nil { - return nil, fmt.Errorf("failed to deserialize cached block: %w", err) + var pbBlock pb.BlockData + if err := proto.Unmarshal(data, &pbBlock); err != nil { + return nil, fmt.Errorf("failed to unmarshal cached block: %w", err) + } + + block := &BlockData{ + Height: pbBlock.Height, + Timestamp: time.Unix(pbBlock.Timestamp, 0).UTC(), + Blobs: pbBlock.Blobs, } f.logger.Debug(). @@ -214,25 +222,24 @@ func (f *asyncBlockFetcher) fetchAndCacheBlock(height uint64) { result := f.client.Retrieve(f.ctx, height, f.client.GetForcedInclusionNamespace()) - if result.Code == datypes.StatusHeightFromFuture { - f.logger.Debug(). - Uint64("height", height). - Msg("block height not yet available - will retry") - return - } - block := &BlockData{ Height: height, Timestamp: result.Timestamp, Blobs: [][]byte{}, } - if result.Code == datypes.StatusNotFound { + switch result.Code { + case datypes.StatusHeightFromFuture: + f.logger.Debug(). + Uint64("height", height). + Msg("block height not yet available - will retry") + return + case datypes.StatusNotFound: f.logger.Debug(). Uint64("height", height). Msg("no forced inclusion blobs at height") // Cache empty result to avoid re-fetching - } else if result.Code == datypes.StatusSuccess { + case datypes.StatusSuccess: // Process each blob for _, blob := range result.Data { if len(blob) > 0 { @@ -243,7 +250,7 @@ func (f *asyncBlockFetcher) fetchAndCacheBlock(height uint64) { Uint64("height", height). Int("blob_count", len(result.Data)). Msg("processed forced inclusion blobs for prefetch") - } else { + default: f.logger.Warn(). Uint64("height", height). Str("status", result.Message). @@ -252,12 +259,17 @@ func (f *asyncBlockFetcher) fetchAndCacheBlock(height uint64) { } // Serialize and cache the block - data, err := serializeBlockData(block) + pbBlock := &pb.BlockData{ + Height: block.Height, + Timestamp: block.Timestamp.Unix(), + Blobs: block.Blobs, + } + data, err := proto.Marshal(pbBlock) if err != nil { f.logger.Error(). Err(err). Uint64("height", height). - Msg("failed to serialize block for caching") + Msg("failed to marshal block for caching") return } @@ -329,116 +341,3 @@ func (f *asyncBlockFetcher) cleanupOldBlocks(currentHeight uint64) { } } } - -// serializeBlockData serializes block data to bytes. -// Format: timestamp (int64) + height (uint64) + blobCount (uint32) + blobs -func serializeBlockData(block *BlockData) ([]byte, error) { - // Calculate total size - size := 8 + 8 + 4 // timestamp + height + blobCount - for _, blob := range block.Blobs { - size += 4 + len(blob) // blobLen + blob - } - - buf := make([]byte, size) - offset := 0 - - // Timestamp - writeUint64(buf[offset:], uint64(block.Timestamp.Unix())) - offset += 8 - - // Height - writeUint64(buf[offset:], block.Height) - offset += 8 - - // BlobCount - writeUint32(buf[offset:], uint32(len(block.Blobs))) - offset += 4 - - // Blobs - for _, blob := range block.Blobs { - writeUint32(buf[offset:], uint32(len(blob))) - offset += 4 - copy(buf[offset:], blob) - offset += len(blob) - } - - return buf, nil -} - -// deserializeBlockData deserializes bytes to block data. -func deserializeBlockData(data []byte) (*BlockData, error) { - if len(data) < 20 { - return nil, errors.New("invalid data: too short") - } - - offset := 0 - block := &BlockData{} - - // Timestamp - timestamp := readUint64(data[offset:]) - block.Timestamp = time.Unix(int64(timestamp), 0).UTC() - offset += 8 - - // Height - block.Height = readUint64(data[offset:]) - offset += 8 - - // BlobCount - blobCount := readUint32(data[offset:]) - offset += 4 - - // Blobs - block.Blobs = make([][]byte, blobCount) - for i := uint32(0); i < blobCount; i++ { - if offset+4 > len(data) { - return nil, errors.New("invalid data: unexpected end while reading blob length") - } - blobLen := readUint32(data[offset:]) - offset += 4 - - if offset+int(blobLen) > len(data) { - return nil, errors.New("invalid data: unexpected end while reading blob") - } - block.Blobs[i] = make([]byte, blobLen) - copy(block.Blobs[i], data[offset:offset+int(blobLen)]) - offset += int(blobLen) - } - - return block, nil -} - -func writeUint64(buf []byte, val uint64) { - buf[0] = byte(val) - buf[1] = byte(val >> 8) - buf[2] = byte(val >> 16) - buf[3] = byte(val >> 24) - buf[4] = byte(val >> 32) - buf[5] = byte(val >> 40) - buf[6] = byte(val >> 48) - buf[7] = byte(val >> 56) -} - -func readUint64(buf []byte) uint64 { - return uint64(buf[0]) | - uint64(buf[1])<<8 | - uint64(buf[2])<<16 | - uint64(buf[3])<<24 | - uint64(buf[4])<<32 | - uint64(buf[5])<<40 | - uint64(buf[6])<<48 | - uint64(buf[7])<<56 -} - -func writeUint32(buf []byte, val uint32) { - buf[0] = byte(val) - buf[1] = byte(val >> 8) - buf[2] = byte(val >> 16) - buf[3] = byte(val >> 24) -} - -func readUint32(buf []byte) uint32 { - return uint32(buf[0]) | - uint32(buf[1])<<8 | - uint32(buf[2])<<16 | - uint32(buf[3])<<24 -} diff --git a/block/internal/da/async_block_fetcher_test.go b/block/internal/da/async_block_fetcher_test.go index 24ee24049..395a0eb7d 100644 --- a/block/internal/da/async_block_fetcher_test.go +++ b/block/internal/da/async_block_fetcher_test.go @@ -9,9 +9,11 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" + "google.golang.org/protobuf/proto" "github.com/evstack/ev-node/pkg/config" datypes "github.com/evstack/ev-node/pkg/da/types" + pb "github.com/evstack/ev-node/types/pb/evnode/v1" mocks "github.com/evstack/ev-node/test/mocks" ) @@ -75,7 +77,9 @@ func TestAsyncBlockFetcher_FetchAndCache(t *testing.T) { logger := zerolog.Nop() // Use a short poll interval for faster test execution - fetcher := NewAsyncBlockFetcher(client, logger, config.DefaultConfig(), 100, 10) + cfg := config.DefaultConfig() + cfg.DA.BlockTime.Duration = 100 * time.Millisecond + fetcher := NewAsyncBlockFetcher(client, logger, cfg, 100, 10) fetcher.Start() defer fetcher.Stop() @@ -134,7 +138,9 @@ func TestAsyncBlockFetcher_BackgroundPrefetch(t *testing.T) { } logger := zerolog.Nop() - fetcher := NewAsyncBlockFetcher(client, logger, config.DefaultConfig(), 100, 10) + cfg := config.DefaultConfig() + cfg.DA.BlockTime.Duration = 100 * time.Millisecond + fetcher := NewAsyncBlockFetcher(client, logger, cfg, 100, 10) fetcher.Start() defer fetcher.Stop() @@ -142,8 +148,8 @@ func TestAsyncBlockFetcher_BackgroundPrefetch(t *testing.T) { // Update current height to trigger prefetch fetcher.UpdateCurrentHeight(100) - // Wait for background prefetch to happen - time.Sleep(200 * time.Millisecond) + // Wait for background prefetch to happen (wait for at least one poll cycle) + time.Sleep(250 * time.Millisecond) // Check if block was prefetched ctx := context.Background() @@ -152,6 +158,7 @@ func TestAsyncBlockFetcher_BackgroundPrefetch(t *testing.T) { assert.NotNil(t, block) assert.Equal(t, uint64(105), block.Height) assert.Equal(t, 2, len(block.Blobs)) + } func TestAsyncBlockFetcher_HeightFromFuture(t *testing.T) { @@ -160,20 +167,24 @@ func TestAsyncBlockFetcher_HeightFromFuture(t *testing.T) { client.On("HasForcedInclusionNamespace").Return(true) client.On("GetForcedInclusionNamespace").Return(fiNs) - // Height not available yet - client.On("Retrieve", mock.Anything, uint64(100), fiNs).Return(datypes.ResultRetrieve{ - BaseResult: datypes.BaseResult{Code: datypes.StatusHeightFromFuture}, - }).Maybe() + // All heights in prefetch window not available yet + for height := uint64(100); height <= 109; height++ { + client.On("Retrieve", mock.Anything, height, fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{Code: datypes.StatusHeightFromFuture}, + }).Maybe() + } logger := zerolog.Nop() - fetcher := NewAsyncBlockFetcher(client, logger, config.DefaultConfig(), 100, 10) + cfg := config.DefaultConfig() + cfg.DA.BlockTime.Duration = 100 * time.Millisecond + fetcher := NewAsyncBlockFetcher(client, logger, cfg, 100, 10) fetcher.Start() defer fetcher.Stop() fetcher.UpdateCurrentHeight(100) - // Wait a bit - time.Sleep(100 * time.Millisecond) + // Wait for at least one poll cycle + time.Sleep(250 * time.Millisecond) // Cache should be empty ctx := context.Background() @@ -207,14 +218,27 @@ func TestBlockData_Serialization(t *testing.T) { }, } - // Serialize - data, err := serializeBlockData(block) + // Serialize using protobuf + pbBlock := &pb.BlockData{ + Height: block.Height, + Timestamp: block.Timestamp.Unix(), + Blobs: block.Blobs, + } + data, err := proto.Marshal(pbBlock) require.NoError(t, err) assert.Greater(t, len(data), 0) - // Deserialize - decoded, err := deserializeBlockData(data) + // Deserialize using protobuf + var decodedPb pb.BlockData + err = proto.Unmarshal(data, &decodedPb) require.NoError(t, err) + + decoded := &BlockData{ + Height: decodedPb.Height, + Timestamp: time.Unix(decodedPb.Timestamp, 0).UTC(), + Blobs: decodedPb.Blobs, + } + assert.Equal(t, block.Timestamp.Unix(), decoded.Timestamp.Unix()) assert.Equal(t, block.Height, decoded.Height) assert.Equal(t, len(block.Blobs), len(decoded.Blobs)) @@ -230,11 +254,26 @@ func TestBlockData_SerializationEmpty(t *testing.T) { Blobs: [][]byte{}, } - data, err := serializeBlockData(block) + // Serialize using protobuf + pbBlock := &pb.BlockData{ + Height: block.Height, + Timestamp: block.Timestamp.Unix(), + Blobs: block.Blobs, + } + data, err := proto.Marshal(pbBlock) require.NoError(t, err) - decoded, err := deserializeBlockData(data) + // Deserialize using protobuf + var decodedPb pb.BlockData + err = proto.Unmarshal(data, &decodedPb) require.NoError(t, err) + + decoded := &BlockData{ + Height: decodedPb.Height, + Timestamp: time.Unix(decodedPb.Timestamp, 0).UTC(), + Blobs: decodedPb.Blobs, + } + assert.Equal(t, uint64(100), decoded.Height) assert.Equal(t, 0, len(decoded.Blobs)) } diff --git a/proto/evnode/v1/da.proto b/proto/evnode/v1/da.proto new file mode 100644 index 000000000..1fcd0e112 --- /dev/null +++ b/proto/evnode/v1/da.proto @@ -0,0 +1,11 @@ +syntax = "proto3"; +package evnode.v1; + +option go_package = "github.com/evstack/ev-node/types/pb/evnode/v1"; + +// BlockData contains data retrieved from a single DA height. +message BlockData { + uint64 height = 1; + int64 timestamp = 2; // Unix timestamp in seconds + repeated bytes blobs = 3; +} diff --git a/types/pb/evnode/v1/da.pb.go b/types/pb/evnode/v1/da.pb.go new file mode 100644 index 000000000..a70d235f2 --- /dev/null +++ b/types/pb/evnode/v1/da.pb.go @@ -0,0 +1,141 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.36.11 +// protoc (unknown) +// source: evnode/v1/da.proto + +package v1 + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" + unsafe "unsafe" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// BlockData contains data retrieved from a single DA height. +type BlockData struct { + state protoimpl.MessageState `protogen:"open.v1"` + Height uint64 `protobuf:"varint,1,opt,name=height,proto3" json:"height,omitempty"` + Timestamp int64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` // Unix timestamp in seconds + Blobs [][]byte `protobuf:"bytes,3,rep,name=blobs,proto3" json:"blobs,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *BlockData) Reset() { + *x = BlockData{} + mi := &file_evnode_v1_da_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *BlockData) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*BlockData) ProtoMessage() {} + +func (x *BlockData) ProtoReflect() protoreflect.Message { + mi := &file_evnode_v1_da_proto_msgTypes[0] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use BlockData.ProtoReflect.Descriptor instead. +func (*BlockData) Descriptor() ([]byte, []int) { + return file_evnode_v1_da_proto_rawDescGZIP(), []int{0} +} + +func (x *BlockData) GetHeight() uint64 { + if x != nil { + return x.Height + } + return 0 +} + +func (x *BlockData) GetTimestamp() int64 { + if x != nil { + return x.Timestamp + } + return 0 +} + +func (x *BlockData) GetBlobs() [][]byte { + if x != nil { + return x.Blobs + } + return nil +} + +var File_evnode_v1_da_proto protoreflect.FileDescriptor + +const file_evnode_v1_da_proto_rawDesc = "" + + "\n" + + "\x12evnode/v1/da.proto\x12\tevnode.v1\"W\n" + + "\tBlockData\x12\x16\n" + + "\x06height\x18\x01 \x01(\x04R\x06height\x12\x1c\n" + + "\ttimestamp\x18\x02 \x01(\x03R\ttimestamp\x12\x14\n" + + "\x05blobs\x18\x03 \x03(\fR\x05blobsB/Z-github.com/evstack/ev-node/types/pb/evnode/v1b\x06proto3" + +var ( + file_evnode_v1_da_proto_rawDescOnce sync.Once + file_evnode_v1_da_proto_rawDescData []byte +) + +func file_evnode_v1_da_proto_rawDescGZIP() []byte { + file_evnode_v1_da_proto_rawDescOnce.Do(func() { + file_evnode_v1_da_proto_rawDescData = protoimpl.X.CompressGZIP(unsafe.Slice(unsafe.StringData(file_evnode_v1_da_proto_rawDesc), len(file_evnode_v1_da_proto_rawDesc))) + }) + return file_evnode_v1_da_proto_rawDescData +} + +var file_evnode_v1_da_proto_msgTypes = make([]protoimpl.MessageInfo, 1) +var file_evnode_v1_da_proto_goTypes = []any{ + (*BlockData)(nil), // 0: evnode.v1.BlockData +} +var file_evnode_v1_da_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_evnode_v1_da_proto_init() } +func file_evnode_v1_da_proto_init() { + if File_evnode_v1_da_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: unsafe.Slice(unsafe.StringData(file_evnode_v1_da_proto_rawDesc), len(file_evnode_v1_da_proto_rawDesc)), + NumEnums: 0, + NumMessages: 1, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_evnode_v1_da_proto_goTypes, + DependencyIndexes: file_evnode_v1_da_proto_depIdxs, + MessageInfos: file_evnode_v1_da_proto_msgTypes, + }.Build() + File_evnode_v1_da_proto = out.File + file_evnode_v1_da_proto_goTypes = nil + file_evnode_v1_da_proto_depIdxs = nil +} From a02247890ca19aaada0a6423b10beb52bc58be6e Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Thu, 8 Jan 2026 20:47:10 +0100 Subject: [PATCH 08/18] remove mutexes and use atomic --- block/internal/da/async_block_fetcher.go | 63 ++++++++++-------------- block/internal/syncing/syncer.go | 2 +- pkg/sequencers/based/sequencer.go | 4 +- pkg/sequencers/single/sequencer.go | 2 +- 4 files changed, 31 insertions(+), 40 deletions(-) diff --git a/block/internal/da/async_block_fetcher.go b/block/internal/da/async_block_fetcher.go index f97c93bab..0e27529fa 100644 --- a/block/internal/da/async_block_fetcher.go +++ b/block/internal/da/async_block_fetcher.go @@ -5,6 +5,7 @@ import ( "errors" "fmt" "sync" + "sync/atomic" "time" ds "github.com/ipfs/go-datastore" @@ -42,16 +43,14 @@ type asyncBlockFetcher struct { // In-memory cache for prefetched block data cache ds.Batching - mu sync.RWMutex // Background fetcher control ctx context.Context cancel context.CancelFunc wg sync.WaitGroup - // Current DA height tracking - currentDAHeight uint64 - heightMu sync.RWMutex + // Current DA height tracking (accessed atomically) + currentDAHeight atomic.Uint64 // Prefetch window - how many blocks ahead to prefetch prefetchWindow uint64 @@ -74,17 +73,18 @@ func NewAsyncBlockFetcher( ctx, cancel := context.WithCancel(context.Background()) - return &asyncBlockFetcher{ - client: client, - logger: logger.With().Str("component", "async_block_fetcher").Logger(), - daStartHeight: daStartHeight, - cache: dsync.MutexWrap(ds.NewMapDatastore()), - ctx: ctx, - cancel: cancel, - currentDAHeight: daStartHeight, - prefetchWindow: prefetchWindow, - pollInterval: config.DA.BlockTime.Duration, + fetcher := &asyncBlockFetcher{ + client: client, + logger: logger.With().Str("component", "async_block_fetcher").Logger(), + daStartHeight: daStartHeight, + cache: dsync.MutexWrap(ds.NewMapDatastore()), + ctx: ctx, + cancel: cancel, + prefetchWindow: prefetchWindow, + pollInterval: config.DA.BlockTime.Duration, } + fetcher.currentDAHeight.Store(daStartHeight) + return fetcher } // Start begins the background prefetching process. @@ -108,14 +108,18 @@ func (f *asyncBlockFetcher) Stop() { // UpdateCurrentHeight updates the current DA height for prefetching. func (f *asyncBlockFetcher) UpdateCurrentHeight(height uint64) { - f.heightMu.Lock() - defer f.heightMu.Unlock() - - if height > f.currentDAHeight { - f.currentDAHeight = height - f.logger.Debug(). - Uint64("new_height", height). - Msg("updated current DA height") + // Use atomic compare-and-swap to update only if the new height is greater + for { + current := f.currentDAHeight.Load() + if height <= current { + return + } + if f.currentDAHeight.CompareAndSwap(current, height) { + f.logger.Debug(). + Uint64("new_height", height). + Msg("updated current DA height") + return + } } } @@ -133,10 +137,7 @@ func (f *asyncBlockFetcher) GetCachedBlock(ctx context.Context, daHeight uint64) // Try to get from cache key := ds.NewKey(fmt.Sprintf("/block/%d", daHeight)) - f.mu.RLock() data, err := f.cache.Get(ctx, key) - f.mu.RUnlock() - if err != nil { if errors.Is(err, ds.ErrNotFound) { return nil, nil // Not cached yet @@ -187,9 +188,7 @@ func (f *asyncBlockFetcher) prefetchBlocks() { return } - f.heightMu.RLock() - currentHeight := f.currentDAHeight - f.heightMu.RUnlock() + currentHeight := f.currentDAHeight.Load() // Prefetch upcoming blocks for i := uint64(0); i < f.prefetchWindow; i++ { @@ -197,10 +196,7 @@ func (f *asyncBlockFetcher) prefetchBlocks() { // Check if already cached key := ds.NewKey(fmt.Sprintf("/block/%d", targetHeight)) - f.mu.RLock() _, err := f.cache.Get(f.ctx, key) - f.mu.RUnlock() - if err == nil { // Already cached continue @@ -274,9 +270,7 @@ func (f *asyncBlockFetcher) fetchAndCacheBlock(height uint64) { } key := ds.NewKey(fmt.Sprintf("/block/%d", height)) - f.mu.Lock() err = f.cache.Put(f.ctx, key, data) - f.mu.Unlock() if err != nil { f.logger.Error(). @@ -302,9 +296,6 @@ func (f *asyncBlockFetcher) cleanupOldBlocks(currentHeight uint64) { cleanupThreshold := currentHeight - f.prefetchWindow - f.mu.Lock() - defer f.mu.Unlock() - // Query all keys query := dsq.Query{Prefix: "/block/"} results, err := f.cache.Query(f.ctx, query) diff --git a/block/internal/syncing/syncer.go b/block/internal/syncing/syncer.go index 454da774b..4bd35e32d 100644 --- a/block/internal/syncing/syncer.go +++ b/block/internal/syncing/syncer.go @@ -194,7 +194,7 @@ func (s *Syncer) Start(ctx context.Context) error { s.logger, s.config, s.genesis.DAStartHeight, - s.genesis.DAEpochForcedInclusion, // prefetch window: blocks ahead to prefetch + s.genesis.DAEpochForcedInclusion*2, // prefetch window: blocks ahead to prefetch ) asyncFetcher.Start() diff --git a/pkg/sequencers/based/sequencer.go b/pkg/sequencers/based/sequencer.go index 9f947589f..a31829b18 100644 --- a/pkg/sequencers/based/sequencer.go +++ b/pkg/sequencers/based/sequencer.go @@ -80,13 +80,13 @@ func NewBasedSequencer( } } - // Create async epoch fetcher for background prefetching (created once) + // Create async epoch fetcher for background prefetching bs.asyncFetcher = block.NewAsyncBlockFetcher( daClient, cfg, logger, genesis.DAStartHeight, - genesis.DAEpochForcedInclusion, + genesis.DAEpochForcedInclusion*2, ) bs.asyncFetcher.Start() diff --git a/pkg/sequencers/single/sequencer.go b/pkg/sequencers/single/sequencer.go index 91499405b..da34f8644 100644 --- a/pkg/sequencers/single/sequencer.go +++ b/pkg/sequencers/single/sequencer.go @@ -114,7 +114,7 @@ func NewSequencer( cfg, logger, initialDAHeight, - genesis.DAEpochForcedInclusion, + genesis.DAEpochForcedInclusion*2, ) s.asyncFetcher.Start() From ca523a5705e4f992044081829d380eeeacbab593 Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Thu, 8 Jan 2026 20:48:40 +0100 Subject: [PATCH 09/18] lint --- block/internal/da/async_block_fetcher_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/block/internal/da/async_block_fetcher_test.go b/block/internal/da/async_block_fetcher_test.go index 395a0eb7d..154bb3ac2 100644 --- a/block/internal/da/async_block_fetcher_test.go +++ b/block/internal/da/async_block_fetcher_test.go @@ -104,9 +104,9 @@ func TestAsyncBlockFetcher_FetchAndCache(t *testing.T) { require.NotNil(t, block, "block should be cached after background fetch") assert.Equal(t, uint64(100), block.Height) assert.Equal(t, 3, len(block.Blobs)) - assert.Equal(t, testBlobs[0], block.Blobs[0]) - assert.Equal(t, testBlobs[1], block.Blobs[1]) - assert.Equal(t, testBlobs[2], block.Blobs[2]) + for i, tb := range testBlobs { + assert.Equal(t, tb, block.Blobs[i]) + } } func TestAsyncBlockFetcher_BackgroundPrefetch(t *testing.T) { From 5608e0ef09e21ae302aab263026fdef7a3a43c7f Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Fri, 9 Jan 2026 00:10:20 +0100 Subject: [PATCH 10/18] rename to retriever --- ...ck_fetcher.go => async_block_retriever.go} | 34 +++++++++---------- ..._test.go => async_block_retriever_test.go} | 24 ++++++------- .../internal/da/forced_inclusion_retriever.go | 4 +-- .../da/forced_inclusion_retriever_test.go | 4 +-- block/internal/syncing/syncer.go | 4 +-- .../syncing/syncer_forced_inclusion_test.go | 4 +-- block/public.go | 16 ++++----- pkg/sequencers/based/sequencer.go | 6 ++-- pkg/sequencers/single/sequencer.go | 4 +-- 9 files changed, 50 insertions(+), 50 deletions(-) rename block/internal/da/{async_block_fetcher.go => async_block_retriever.go} (88%) rename block/internal/da/{async_block_fetcher_test.go => async_block_retriever_test.go} (89%) diff --git a/block/internal/da/async_block_fetcher.go b/block/internal/da/async_block_retriever.go similarity index 88% rename from block/internal/da/async_block_fetcher.go rename to block/internal/da/async_block_retriever.go index 0e27529fa..983f30991 100644 --- a/block/internal/da/async_block_fetcher.go +++ b/block/internal/da/async_block_retriever.go @@ -19,8 +19,8 @@ import ( pb "github.com/evstack/ev-node/types/pb/evnode/v1" ) -// AsyncBlockFetcher provides background prefetching of DA blocks -type AsyncBlockFetcher interface { +// AsyncBlockRetriever provides background prefetching of DA blocks +type AsyncBlockRetriever interface { Start() Stop() GetCachedBlock(ctx context.Context, daHeight uint64) (*BlockData, error) @@ -34,9 +34,9 @@ type BlockData struct { Blobs [][]byte } -// asyncBlockFetcher handles background prefetching of individual DA blocks +// asyncBlockRetriever handles background prefetching of individual DA blocks // to speed up forced inclusion processing. -type asyncBlockFetcher struct { +type asyncBlockRetriever struct { client Client logger zerolog.Logger daStartHeight uint64 @@ -59,23 +59,23 @@ type asyncBlockFetcher struct { pollInterval time.Duration } -// NewAsyncBlockFetcher creates a new async block fetcher with in-memory cache. -func NewAsyncBlockFetcher( +// NewAsyncBlockRetriever creates a new async block retriever with in-memory cache. +func NewAsyncBlockRetriever( client Client, logger zerolog.Logger, config config.Config, daStartHeight uint64, prefetchWindow uint64, -) AsyncBlockFetcher { +) AsyncBlockRetriever { if prefetchWindow == 0 { prefetchWindow = 10 // Default: prefetch next 10 blocks } ctx, cancel := context.WithCancel(context.Background()) - fetcher := &asyncBlockFetcher{ + fetcher := &asyncBlockRetriever{ client: client, - logger: logger.With().Str("component", "async_block_fetcher").Logger(), + logger: logger.With().Str("component", "async_block_retriever").Logger(), daStartHeight: daStartHeight, cache: dsync.MutexWrap(ds.NewMapDatastore()), ctx: ctx, @@ -88,7 +88,7 @@ func NewAsyncBlockFetcher( } // Start begins the background prefetching process. -func (f *asyncBlockFetcher) Start() { +func (f *asyncBlockRetriever) Start() { f.wg.Add(1) go f.backgroundFetchLoop() f.logger.Info(). @@ -99,7 +99,7 @@ func (f *asyncBlockFetcher) Start() { } // Stop gracefully stops the background prefetching process. -func (f *asyncBlockFetcher) Stop() { +func (f *asyncBlockRetriever) Stop() { f.logger.Info().Msg("stopping async block fetcher") f.cancel() f.wg.Wait() @@ -107,7 +107,7 @@ func (f *asyncBlockFetcher) Stop() { } // UpdateCurrentHeight updates the current DA height for prefetching. -func (f *asyncBlockFetcher) UpdateCurrentHeight(height uint64) { +func (f *asyncBlockRetriever) UpdateCurrentHeight(height uint64) { // Use atomic compare-and-swap to update only if the new height is greater for { current := f.currentDAHeight.Load() @@ -125,7 +125,7 @@ func (f *asyncBlockFetcher) UpdateCurrentHeight(height uint64) { // GetCachedBlock retrieves a cached block from memory. // Returns nil if the block is not cached. -func (f *asyncBlockFetcher) GetCachedBlock(ctx context.Context, daHeight uint64) (*BlockData, error) { +func (f *asyncBlockRetriever) GetCachedBlock(ctx context.Context, daHeight uint64) (*BlockData, error) { if !f.client.HasForcedInclusionNamespace() { return nil, ErrForceInclusionNotConfigured } @@ -166,7 +166,7 @@ func (f *asyncBlockFetcher) GetCachedBlock(ctx context.Context, daHeight uint64) } // backgroundFetchLoop runs in the background and prefetches blocks ahead of time. -func (f *asyncBlockFetcher) backgroundFetchLoop() { +func (f *asyncBlockRetriever) backgroundFetchLoop() { defer f.wg.Done() ticker := time.NewTicker(f.pollInterval) @@ -183,7 +183,7 @@ func (f *asyncBlockFetcher) backgroundFetchLoop() { } // prefetchBlocks prefetches blocks within the prefetch window. -func (f *asyncBlockFetcher) prefetchBlocks() { +func (f *asyncBlockRetriever) prefetchBlocks() { if !f.client.HasForcedInclusionNamespace() { return } @@ -211,7 +211,7 @@ func (f *asyncBlockFetcher) prefetchBlocks() { } // fetchAndCacheBlock fetches a block and stores it in the cache. -func (f *asyncBlockFetcher) fetchAndCacheBlock(height uint64) { +func (f *asyncBlockRetriever) fetchAndCacheBlock(height uint64) { f.logger.Debug(). Uint64("height", height). Msg("prefetching block") @@ -287,7 +287,7 @@ func (f *asyncBlockFetcher) fetchAndCacheBlock(height uint64) { } // cleanupOldBlocks removes blocks older than a threshold from cache. -func (f *asyncBlockFetcher) cleanupOldBlocks(currentHeight uint64) { +func (f *asyncBlockRetriever) cleanupOldBlocks(currentHeight uint64) { // Remove blocks older than current - prefetchWindow // Keep some history in case of reorgs or restarts if currentHeight < f.prefetchWindow { diff --git a/block/internal/da/async_block_fetcher_test.go b/block/internal/da/async_block_retriever_test.go similarity index 89% rename from block/internal/da/async_block_fetcher_test.go rename to block/internal/da/async_block_retriever_test.go index 154bb3ac2..f6262b7f9 100644 --- a/block/internal/da/async_block_fetcher_test.go +++ b/block/internal/da/async_block_retriever_test.go @@ -18,26 +18,26 @@ import ( mocks "github.com/evstack/ev-node/test/mocks" ) -func TestAsyncBlockFetcher_GetCachedBlock_NoNamespace(t *testing.T) { +func TestAsyncBlockRetriever_GetCachedBlock_NoNamespace(t *testing.T) { client := &mocks.MockClient{} client.On("HasForcedInclusionNamespace").Return(false) logger := zerolog.Nop() - fetcher := NewAsyncBlockFetcher(client, logger, config.DefaultConfig(), 100, 10) + fetcher := NewAsyncBlockRetriever(client, logger, config.DefaultConfig(), 100, 10) ctx := context.Background() _, err := fetcher.GetCachedBlock(ctx, 100) assert.ErrorIs(t, err, ErrForceInclusionNotConfigured) } -func TestAsyncBlockFetcher_GetCachedBlock_CacheMiss(t *testing.T) { +func TestAsyncBlockRetriever_GetCachedBlock_CacheMiss(t *testing.T) { client := &mocks.MockClient{} fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() client.On("HasForcedInclusionNamespace").Return(true) client.On("GetForcedInclusionNamespace").Return(fiNs) logger := zerolog.Nop() - fetcher := NewAsyncBlockFetcher(client, logger, config.DefaultConfig(), 100, 10) + fetcher := NewAsyncBlockRetriever(client, logger, config.DefaultConfig(), 100, 10) ctx := context.Background() @@ -47,7 +47,7 @@ func TestAsyncBlockFetcher_GetCachedBlock_CacheMiss(t *testing.T) { assert.Nil(t, block) // Cache miss } -func TestAsyncBlockFetcher_FetchAndCache(t *testing.T) { +func TestAsyncBlockRetriever_FetchAndCache(t *testing.T) { testBlobs := [][]byte{ []byte("tx1"), []byte("tx2"), @@ -79,7 +79,7 @@ func TestAsyncBlockFetcher_FetchAndCache(t *testing.T) { // Use a short poll interval for faster test execution cfg := config.DefaultConfig() cfg.DA.BlockTime.Duration = 100 * time.Millisecond - fetcher := NewAsyncBlockFetcher(client, logger, cfg, 100, 10) + fetcher := NewAsyncBlockRetriever(client, logger, cfg, 100, 10) fetcher.Start() defer fetcher.Stop() @@ -109,7 +109,7 @@ func TestAsyncBlockFetcher_FetchAndCache(t *testing.T) { } } -func TestAsyncBlockFetcher_BackgroundPrefetch(t *testing.T) { +func TestAsyncBlockRetriever_BackgroundPrefetch(t *testing.T) { testBlobs := [][]byte{ []byte("tx1"), []byte("tx2"), @@ -140,7 +140,7 @@ func TestAsyncBlockFetcher_BackgroundPrefetch(t *testing.T) { logger := zerolog.Nop() cfg := config.DefaultConfig() cfg.DA.BlockTime.Duration = 100 * time.Millisecond - fetcher := NewAsyncBlockFetcher(client, logger, cfg, 100, 10) + fetcher := NewAsyncBlockRetriever(client, logger, cfg, 100, 10) fetcher.Start() defer fetcher.Stop() @@ -161,7 +161,7 @@ func TestAsyncBlockFetcher_BackgroundPrefetch(t *testing.T) { } -func TestAsyncBlockFetcher_HeightFromFuture(t *testing.T) { +func TestAsyncBlockRetriever_HeightFromFuture(t *testing.T) { client := &mocks.MockClient{} fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() client.On("HasForcedInclusionNamespace").Return(true) @@ -177,7 +177,7 @@ func TestAsyncBlockFetcher_HeightFromFuture(t *testing.T) { logger := zerolog.Nop() cfg := config.DefaultConfig() cfg.DA.BlockTime.Duration = 100 * time.Millisecond - fetcher := NewAsyncBlockFetcher(client, logger, cfg, 100, 10) + fetcher := NewAsyncBlockRetriever(client, logger, cfg, 100, 10) fetcher.Start() defer fetcher.Stop() @@ -193,12 +193,12 @@ func TestAsyncBlockFetcher_HeightFromFuture(t *testing.T) { assert.Nil(t, block) } -func TestAsyncBlockFetcher_StopGracefully(t *testing.T) { +func TestAsyncBlockRetriever_StopGracefully(t *testing.T) { client := &mocks.MockClient{} client.On("HasForcedInclusionNamespace").Return(false) logger := zerolog.Nop() - fetcher := NewAsyncBlockFetcher(client, logger, config.DefaultConfig(), 100, 10) + fetcher := NewAsyncBlockRetriever(client, logger, config.DefaultConfig(), 100, 10) fetcher.Start() time.Sleep(100 * time.Millisecond) diff --git a/block/internal/da/forced_inclusion_retriever.go b/block/internal/da/forced_inclusion_retriever.go index cd477aa48..ebd7a98e2 100644 --- a/block/internal/da/forced_inclusion_retriever.go +++ b/block/internal/da/forced_inclusion_retriever.go @@ -21,7 +21,7 @@ type ForcedInclusionRetriever struct { logger zerolog.Logger daEpochSize uint64 daStartHeight uint64 - asyncFetcher AsyncBlockFetcher // Fetches individual blocks in background + asyncFetcher AsyncBlockRetriever // Fetches individual blocks in background } // ForcedInclusionEvent contains forced inclusion transactions retrieved from DA. @@ -37,7 +37,7 @@ func NewForcedInclusionRetriever( client Client, logger zerolog.Logger, daStartHeight, daEpochSize uint64, - asyncFetcher AsyncBlockFetcher, + asyncFetcher AsyncBlockRetriever, ) *ForcedInclusionRetriever { return &ForcedInclusionRetriever{ client: client, diff --git a/block/internal/da/forced_inclusion_retriever_test.go b/block/internal/da/forced_inclusion_retriever_test.go index 34adfeb76..f597a4c09 100644 --- a/block/internal/da/forced_inclusion_retriever_test.go +++ b/block/internal/da/forced_inclusion_retriever_test.go @@ -16,8 +16,8 @@ import ( ) // createTestAsyncFetcher creates a minimal async fetcher for tests -func createTestAsyncFetcher(client Client, gen genesis.Genesis) AsyncBlockFetcher { - return NewAsyncBlockFetcher( +func createTestAsyncFetcher(client Client, gen genesis.Genesis) AsyncBlockRetriever { + return NewAsyncBlockRetriever( client, zerolog.Nop(), config.DefaultConfig(), diff --git a/block/internal/syncing/syncer.go b/block/internal/syncing/syncer.go index 4bd35e32d..15a577264 100644 --- a/block/internal/syncing/syncer.go +++ b/block/internal/syncing/syncer.go @@ -188,8 +188,8 @@ func (s *Syncer) Start(ctx context.Context) error { // Initialize handlers s.daRetriever = NewDARetriever(s.daClient, s.cache, s.genesis, s.logger) - // Create async block fetcher for background prefetching - asyncFetcher := da.NewAsyncBlockFetcher( + // Create async block retriever for background prefetching + asyncFetcher := da.NewAsyncBlockRetriever( s.daClient, s.logger, s.config, diff --git a/block/internal/syncing/syncer_forced_inclusion_test.go b/block/internal/syncing/syncer_forced_inclusion_test.go index d0a97d7a5..e3219fd33 100644 --- a/block/internal/syncing/syncer_forced_inclusion_test.go +++ b/block/internal/syncing/syncer_forced_inclusion_test.go @@ -24,8 +24,8 @@ import ( ) // createTestAsyncFetcher creates a minimal async fetcher for tests (without starting it) -func createTestAsyncFetcherForSyncer(client da.Client, gen genesis.Genesis) da.AsyncBlockFetcher { - return da.NewAsyncBlockFetcher( +func createTestAsyncFetcherForSyncer(client da.Client, gen genesis.Genesis) da.AsyncBlockRetriever { + return da.NewAsyncBlockRetriever( client, zerolog.Nop(), config.DefaultConfig(), diff --git a/block/public.go b/block/public.go index 2d59e2461..1c73ac6fa 100644 --- a/block/public.go +++ b/block/public.go @@ -65,8 +65,8 @@ var ErrForceInclusionNotConfigured = da.ErrForceInclusionNotConfigured // ForcedInclusionEvent represents forced inclusion transactions retrieved from DA type ForcedInclusionEvent = da.ForcedInclusionEvent -// AsyncBlockFetcher provides background prefetching of individual DA blocks -type AsyncBlockFetcher interface { +// AsyncBlockRetriever provides background prefetching of individual DA blocks +type AsyncBlockRetriever interface { Start() Stop() GetCachedBlock(ctx context.Context, daHeight uint64) (*da.BlockData, error) @@ -78,31 +78,31 @@ type ForcedInclusionRetriever interface { RetrieveForcedIncludedTxs(ctx context.Context, daHeight uint64) (*da.ForcedInclusionEvent, error) } -// NewAsyncBlockFetcher creates a new async block fetcher for background prefetching. +// NewAsyncBlockRetriever creates a new async block retriever for background prefetching. // Parameters: // - client: DA client for fetching data // - config: Ev-node config // - logger: structured logger // - daStartHeight: genesis DA start height // - prefetchWindow: how many blocks ahead to prefetch (10-20 recommended) -func NewAsyncBlockFetcher( +func NewAsyncBlockRetriever( client DAClient, cfg config.Config, logger zerolog.Logger, daStartHeight uint64, prefetchWindow uint64, -) AsyncBlockFetcher { - return da.NewAsyncBlockFetcher(client, logger, cfg, daStartHeight, prefetchWindow) +) AsyncBlockRetriever { + return da.NewAsyncBlockRetriever(client, logger, cfg, daStartHeight, prefetchWindow) } // NewForcedInclusionRetriever creates a new forced inclusion retriever. // The asyncFetcher parameter is required for background prefetching of DA block data. -// It accepts either AsyncBlockFetcher (recommended) or AsyncEpochFetcher (deprecated) for backward compatibility. +// It accepts either AsyncBlockRetriever (recommended) or AsyncEpochFetcher (deprecated) for backward compatibility. func NewForcedInclusionRetriever( client DAClient, logger zerolog.Logger, daStartHeight, daEpochSize uint64, - asyncFetcher AsyncBlockFetcher, + asyncFetcher AsyncBlockRetriever, ) ForcedInclusionRetriever { return da.NewForcedInclusionRetriever(client, logger, daStartHeight, daEpochSize, asyncFetcher) } diff --git a/pkg/sequencers/based/sequencer.go b/pkg/sequencers/based/sequencer.go index a31829b18..9437fe4a8 100644 --- a/pkg/sequencers/based/sequencer.go +++ b/pkg/sequencers/based/sequencer.go @@ -26,7 +26,7 @@ var _ coresequencer.Sequencer = (*BasedSequencer)(nil) type BasedSequencer struct { logger zerolog.Logger - asyncFetcher block.AsyncBlockFetcher + asyncFetcher block.AsyncBlockRetriever fiRetriever block.ForcedInclusionRetriever daHeight atomic.Uint64 checkpointStore *seqcommon.CheckpointStore @@ -80,8 +80,8 @@ func NewBasedSequencer( } } - // Create async epoch fetcher for background prefetching - bs.asyncFetcher = block.NewAsyncBlockFetcher( + // Create async block retriever for background prefetching + bs.asyncFetcher = block.NewAsyncBlockRetriever( daClient, cfg, logger, diff --git a/pkg/sequencers/single/sequencer.go b/pkg/sequencers/single/sequencer.go index da34f8644..0745dabbe 100644 --- a/pkg/sequencers/single/sequencer.go +++ b/pkg/sequencers/single/sequencer.go @@ -41,7 +41,7 @@ type Sequencer struct { queue *BatchQueue // single queue for immediate availability // Forced inclusion support - asyncFetcher block.AsyncBlockFetcher + asyncFetcher block.AsyncBlockRetriever fiRetriever block.ForcedInclusionRetriever daHeight atomic.Uint64 daStartHeight atomic.Uint64 @@ -109,7 +109,7 @@ func NewSequencer( // Determine initial DA height for forced inclusion initialDAHeight := s.getInitialDAStartHeight(context.Background()) - s.asyncFetcher = block.NewAsyncBlockFetcher( + s.asyncFetcher = block.NewAsyncBlockRetriever( daClient, cfg, logger, From d6d9af1c3bde8dd7e24beceb0a5a5b1c930ef77a Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Fri, 9 Jan 2026 00:11:32 +0100 Subject: [PATCH 11/18] fix --- pkg/sequencers/single/sequencer.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/sequencers/single/sequencer.go b/pkg/sequencers/single/sequencer.go index 0745dabbe..a23e3fa16 100644 --- a/pkg/sequencers/single/sequencer.go +++ b/pkg/sequencers/single/sequencer.go @@ -185,10 +185,10 @@ func (c *Sequencer) GetNextBatch(ctx context.Context, req coresequencer.GetNextB DAHeight: daHeight, TxIndex: 0, } - } - // override forced inclusion retriever, as the da start height could have been updated - c.fiRetriever = block.NewForcedInclusionRetriever(c.daClient, c.logger, c.getInitialDAStartHeight(ctx), c.genesis.DAEpochForcedInclusion, c.asyncFetcher) + // override forced inclusion retriever, as the da start height have been updated + c.fiRetriever = block.NewForcedInclusionRetriever(c.daClient, c.logger, c.getInitialDAStartHeight(ctx), c.genesis.DAEpochForcedInclusion, c.asyncFetcher) + } // If we have no cached transactions or we've consumed all from the current cache, // fetch the next DA epoch From 655d1e6401480a11113d0f4d9a7f3d3b0b509738 Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Fri, 9 Jan 2026 00:15:21 +0100 Subject: [PATCH 12/18] nits --- block/internal/da/async_block_retriever.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/block/internal/da/async_block_retriever.go b/block/internal/da/async_block_retriever.go index 983f30991..923a06851 100644 --- a/block/internal/da/async_block_retriever.go +++ b/block/internal/da/async_block_retriever.go @@ -95,15 +95,14 @@ func (f *asyncBlockRetriever) Start() { Uint64("da_start_height", f.daStartHeight). Uint64("prefetch_window", f.prefetchWindow). Dur("poll_interval", f.pollInterval). - Msg("async block fetcher started") + Msg("async block retriever started") } // Stop gracefully stops the background prefetching process. func (f *asyncBlockRetriever) Stop() { - f.logger.Info().Msg("stopping async block fetcher") + f.logger.Info().Msg("stopping async block retriever") f.cancel() f.wg.Wait() - f.logger.Info().Msg("async block fetcher stopped") } // UpdateCurrentHeight updates the current DA height for prefetching. @@ -289,7 +288,6 @@ func (f *asyncBlockRetriever) fetchAndCacheBlock(height uint64) { // cleanupOldBlocks removes blocks older than a threshold from cache. func (f *asyncBlockRetriever) cleanupOldBlocks(currentHeight uint64) { // Remove blocks older than current - prefetchWindow - // Keep some history in case of reorgs or restarts if currentHeight < f.prefetchWindow { return } From c81876677a314a2b6267b5e9ce9cf91231118c59 Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Fri, 9 Jan 2026 00:31:23 +0100 Subject: [PATCH 13/18] keep async retriever internal --- block/internal/da/async_block_retriever.go | 13 +++-- .../internal/da/async_block_retriever_test.go | 28 ++++------- .../internal/da/forced_inclusion_retriever.go | 29 +++++++++-- .../da/forced_inclusion_retriever_test.go | 50 ++++++++----------- block/internal/syncing/syncer.go | 12 +---- .../syncing/syncer_forced_inclusion_test.go | 44 +++++++--------- block/public.go | 33 ++---------- pkg/sequencers/based/sequencer.go | 13 +---- pkg/sequencers/based/sequencer_test.go | 11 ++++ pkg/sequencers/single/sequencer.go | 19 +++---- 10 files changed, 105 insertions(+), 147 deletions(-) diff --git a/block/internal/da/async_block_retriever.go b/block/internal/da/async_block_retriever.go index 923a06851..c34873999 100644 --- a/block/internal/da/async_block_retriever.go +++ b/block/internal/da/async_block_retriever.go @@ -35,10 +35,11 @@ type BlockData struct { } // asyncBlockRetriever handles background prefetching of individual DA blocks -// to speed up forced inclusion processing. +// from a specific namespace. type asyncBlockRetriever struct { client Client logger zerolog.Logger + namespace []byte daStartHeight uint64 // In-memory cache for prefetched block data @@ -63,6 +64,7 @@ type asyncBlockRetriever struct { func NewAsyncBlockRetriever( client Client, logger zerolog.Logger, + namespace []byte, config config.Config, daStartHeight uint64, prefetchWindow uint64, @@ -76,6 +78,7 @@ func NewAsyncBlockRetriever( fetcher := &asyncBlockRetriever{ client: client, logger: logger.With().Str("component", "async_block_retriever").Logger(), + namespace: namespace, daStartHeight: daStartHeight, cache: dsync.MutexWrap(ds.NewMapDatastore()), ctx: ctx, @@ -125,8 +128,8 @@ func (f *asyncBlockRetriever) UpdateCurrentHeight(height uint64) { // GetCachedBlock retrieves a cached block from memory. // Returns nil if the block is not cached. func (f *asyncBlockRetriever) GetCachedBlock(ctx context.Context, daHeight uint64) (*BlockData, error) { - if !f.client.HasForcedInclusionNamespace() { - return nil, ErrForceInclusionNotConfigured + if len(f.namespace) == 0 { + return nil, nil } if daHeight < f.daStartHeight { @@ -183,7 +186,7 @@ func (f *asyncBlockRetriever) backgroundFetchLoop() { // prefetchBlocks prefetches blocks within the prefetch window. func (f *asyncBlockRetriever) prefetchBlocks() { - if !f.client.HasForcedInclusionNamespace() { + if len(f.namespace) == 0 { return } @@ -215,7 +218,7 @@ func (f *asyncBlockRetriever) fetchAndCacheBlock(height uint64) { Uint64("height", height). Msg("prefetching block") - result := f.client.Retrieve(f.ctx, height, f.client.GetForcedInclusionNamespace()) + result := f.client.Retrieve(f.ctx, height, f.namespace) block := &BlockData{ Height: height, diff --git a/block/internal/da/async_block_retriever_test.go b/block/internal/da/async_block_retriever_test.go index f6262b7f9..82525ea31 100644 --- a/block/internal/da/async_block_retriever_test.go +++ b/block/internal/da/async_block_retriever_test.go @@ -20,24 +20,22 @@ import ( func TestAsyncBlockRetriever_GetCachedBlock_NoNamespace(t *testing.T) { client := &mocks.MockClient{} - client.On("HasForcedInclusionNamespace").Return(false) logger := zerolog.Nop() - fetcher := NewAsyncBlockRetriever(client, logger, config.DefaultConfig(), 100, 10) + fetcher := NewAsyncBlockRetriever(client, logger, nil, config.DefaultConfig(), 100, 10) ctx := context.Background() - _, err := fetcher.GetCachedBlock(ctx, 100) - assert.ErrorIs(t, err, ErrForceInclusionNotConfigured) + block, err := fetcher.GetCachedBlock(ctx, 100) + assert.NoError(t, err) + assert.Nil(t, block) } func TestAsyncBlockRetriever_GetCachedBlock_CacheMiss(t *testing.T) { client := &mocks.MockClient{} fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() - client.On("HasForcedInclusionNamespace").Return(true) - client.On("GetForcedInclusionNamespace").Return(fiNs) logger := zerolog.Nop() - fetcher := NewAsyncBlockRetriever(client, logger, config.DefaultConfig(), 100, 10) + fetcher := NewAsyncBlockRetriever(client, logger, fiNs, config.DefaultConfig(), 100, 10) ctx := context.Background() @@ -56,8 +54,6 @@ func TestAsyncBlockRetriever_FetchAndCache(t *testing.T) { client := &mocks.MockClient{} fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() - client.On("HasForcedInclusionNamespace").Return(true) - client.On("GetForcedInclusionNamespace").Return(fiNs) // Mock Retrieve call for height 100 client.On("Retrieve", mock.Anything, uint64(100), fiNs).Return(datypes.ResultRetrieve{ @@ -79,7 +75,7 @@ func TestAsyncBlockRetriever_FetchAndCache(t *testing.T) { // Use a short poll interval for faster test execution cfg := config.DefaultConfig() cfg.DA.BlockTime.Duration = 100 * time.Millisecond - fetcher := NewAsyncBlockRetriever(client, logger, cfg, 100, 10) + fetcher := NewAsyncBlockRetriever(client, logger, fiNs, cfg, 100, 10) fetcher.Start() defer fetcher.Stop() @@ -117,8 +113,6 @@ func TestAsyncBlockRetriever_BackgroundPrefetch(t *testing.T) { client := &mocks.MockClient{} fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() - client.On("HasForcedInclusionNamespace").Return(true) - client.On("GetForcedInclusionNamespace").Return(fiNs) // Mock for heights 100-110 (current + prefetch window) for height := uint64(100); height <= 110; height++ { @@ -140,7 +134,7 @@ func TestAsyncBlockRetriever_BackgroundPrefetch(t *testing.T) { logger := zerolog.Nop() cfg := config.DefaultConfig() cfg.DA.BlockTime.Duration = 100 * time.Millisecond - fetcher := NewAsyncBlockRetriever(client, logger, cfg, 100, 10) + fetcher := NewAsyncBlockRetriever(client, logger, fiNs, cfg, 100, 10) fetcher.Start() defer fetcher.Stop() @@ -164,8 +158,6 @@ func TestAsyncBlockRetriever_BackgroundPrefetch(t *testing.T) { func TestAsyncBlockRetriever_HeightFromFuture(t *testing.T) { client := &mocks.MockClient{} fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() - client.On("HasForcedInclusionNamespace").Return(true) - client.On("GetForcedInclusionNamespace").Return(fiNs) // All heights in prefetch window not available yet for height := uint64(100); height <= 109; height++ { @@ -177,7 +169,7 @@ func TestAsyncBlockRetriever_HeightFromFuture(t *testing.T) { logger := zerolog.Nop() cfg := config.DefaultConfig() cfg.DA.BlockTime.Duration = 100 * time.Millisecond - fetcher := NewAsyncBlockRetriever(client, logger, cfg, 100, 10) + fetcher := NewAsyncBlockRetriever(client, logger, fiNs, cfg, 100, 10) fetcher.Start() defer fetcher.Stop() @@ -195,10 +187,10 @@ func TestAsyncBlockRetriever_HeightFromFuture(t *testing.T) { func TestAsyncBlockRetriever_StopGracefully(t *testing.T) { client := &mocks.MockClient{} - client.On("HasForcedInclusionNamespace").Return(false) + fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() logger := zerolog.Nop() - fetcher := NewAsyncBlockRetriever(client, logger, config.DefaultConfig(), 100, 10) + fetcher := NewAsyncBlockRetriever(client, logger, fiNs, config.DefaultConfig(), 100, 10) fetcher.Start() time.Sleep(100 * time.Millisecond) diff --git a/block/internal/da/forced_inclusion_retriever.go b/block/internal/da/forced_inclusion_retriever.go index ebd7a98e2..b5e28a7a3 100644 --- a/block/internal/da/forced_inclusion_retriever.go +++ b/block/internal/da/forced_inclusion_retriever.go @@ -6,10 +6,10 @@ import ( "fmt" "time" - "github.com/rs/zerolog" - + "github.com/evstack/ev-node/pkg/config" datypes "github.com/evstack/ev-node/pkg/da/types" "github.com/evstack/ev-node/types" + "github.com/rs/zerolog" ) // ErrForceInclusionNotConfigured is returned when the forced inclusion namespace is not configured. @@ -33,21 +33,42 @@ type ForcedInclusionEvent struct { } // NewForcedInclusionRetriever creates a new forced inclusion retriever. +// It internally creates and manages an AsyncBlockRetriever for background prefetching. func NewForcedInclusionRetriever( client Client, logger zerolog.Logger, + cfg config.Config, daStartHeight, daEpochSize uint64, - asyncFetcher AsyncBlockRetriever, ) *ForcedInclusionRetriever { + retrieverLogger := logger.With().Str("component", "forced_inclusion_retriever").Logger() + + // Create async block retriever for background prefetching + asyncFetcher := NewAsyncBlockRetriever( + client, + logger, + client.GetForcedInclusionNamespace(), + cfg, + daStartHeight, + daEpochSize*2, // prefetch window: 2x epoch size + ) + asyncFetcher.Start() + return &ForcedInclusionRetriever{ client: client, - logger: logger.With().Str("component", "forced_inclusion_retriever").Logger(), + logger: retrieverLogger, daStartHeight: daStartHeight, daEpochSize: daEpochSize, asyncFetcher: asyncFetcher, } } +// Stop stops the background prefetcher. +func (r *ForcedInclusionRetriever) Stop() { + if r.asyncFetcher != nil { + r.asyncFetcher.Stop() + } +} + // RetrieveForcedIncludedTxs retrieves forced inclusion transactions at the given DA height. // It respects epoch boundaries and only fetches at epoch end. // It tries to get blocks from the async fetcher cache first, then falls back to sync fetching. diff --git a/block/internal/da/forced_inclusion_retriever_test.go b/block/internal/da/forced_inclusion_retriever_test.go index f597a4c09..4961d74d4 100644 --- a/block/internal/da/forced_inclusion_retriever_test.go +++ b/block/internal/da/forced_inclusion_retriever_test.go @@ -15,20 +15,8 @@ import ( "github.com/evstack/ev-node/test/mocks" ) -// createTestAsyncFetcher creates a minimal async fetcher for tests -func createTestAsyncFetcher(client Client, gen genesis.Genesis) AsyncBlockRetriever { - return NewAsyncBlockRetriever( - client, - zerolog.Nop(), - config.DefaultConfig(), - gen.DAStartHeight, - 10, // prefetch 10 blocks - ) -} - func TestNewForcedInclusionRetriever(t *testing.T) { client := mocks.NewMockClient(t) - client.On("HasForcedInclusionNamespace").Return(true).Maybe() client.On("GetForcedInclusionNamespace").Return(datypes.NamespaceFromString("test-fi-ns").Bytes()).Maybe() gen := genesis.Genesis{ @@ -36,22 +24,23 @@ func TestNewForcedInclusionRetriever(t *testing.T) { DAEpochForcedInclusion: 10, } - asyncFetcher := createTestAsyncFetcher(client, gen) - retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) assert.Assert(t, retriever != nil) + retriever.Stop() } func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_NoNamespace(t *testing.T) { client := mocks.NewMockClient(t) - client.On("HasForcedInclusionNamespace").Return(false).Once() + client.On("HasForcedInclusionNamespace").Return(false).Maybe() + client.On("GetForcedInclusionNamespace").Return([]byte(nil)).Maybe() gen := genesis.Genesis{ DAStartHeight: 100, DAEpochForcedInclusion: 10, } - asyncFetcher := createTestAsyncFetcher(client, gen) - retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer retriever.Stop() ctx := context.Background() _, err := retriever.RetrieveForcedIncludedTxs(ctx, 100) @@ -62,7 +51,7 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_NoNamespace(t *testi func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_NotAtEpochStart(t *testing.T) { client := mocks.NewMockClient(t) fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() - client.On("HasForcedInclusionNamespace").Return(true).Once() + client.On("HasForcedInclusionNamespace").Return(true).Maybe() client.On("GetForcedInclusionNamespace").Return(fiNs).Maybe() gen := genesis.Genesis{ @@ -70,8 +59,8 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_NotAtEpochStart(t *t DAEpochForcedInclusion: 10, } - asyncFetcher := createTestAsyncFetcher(client, gen) - retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer retriever.Stop() ctx := context.Background() // Height 105 is not an epoch start (100, 110, 120, etc. are epoch starts) @@ -104,8 +93,8 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_EpochStartSuccess(t DAEpochForcedInclusion: 1, // Single height epoch } - asyncFetcher := createTestAsyncFetcher(client, gen) - retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer retriever.Stop() ctx := context.Background() // Height 100 is an epoch start @@ -134,8 +123,8 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_EpochStartNotAvailab DAEpochForcedInclusion: 10, } - asyncFetcher := createTestAsyncFetcher(client, gen) - retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer retriever.Stop() ctx := context.Background() // Epoch boundaries: [100, 109] - now tries to fetch all blocks in epoch @@ -158,8 +147,8 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_NoBlobsAtHeight(t *t DAEpochForcedInclusion: 1, // Single height epoch } - asyncFetcher := createTestAsyncFetcher(client, gen) - retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer retriever.Stop() ctx := context.Background() event, err := retriever.RetrieveForcedIncludedTxs(ctx, 100) @@ -197,8 +186,8 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_MultiHeightEpoch(t * DAEpochForcedInclusion: 3, // Epoch: 100-102 } - asyncFetcher := createTestAsyncFetcher(client, gen) - retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer retriever.Stop() ctx := context.Background() // Epoch boundaries: [100, 102] - retrieval happens at epoch end (102) @@ -216,14 +205,15 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_MultiHeightEpoch(t * func TestForcedInclusionRetriever_processForcedInclusionBlobs(t *testing.T) { client := mocks.NewMockClient(t) + client.On("GetForcedInclusionNamespace").Return(datypes.NamespaceFromString("test-fi-ns").Bytes()).Maybe() gen := genesis.Genesis{ DAStartHeight: 100, DAEpochForcedInclusion: 10, } - asyncFetcher := createTestAsyncFetcher(client, gen) - retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer retriever.Stop() tests := []struct { name string diff --git a/block/internal/syncing/syncer.go b/block/internal/syncing/syncer.go index 15a577264..4295714ea 100644 --- a/block/internal/syncing/syncer.go +++ b/block/internal/syncing/syncer.go @@ -188,17 +188,7 @@ func (s *Syncer) Start(ctx context.Context) error { // Initialize handlers s.daRetriever = NewDARetriever(s.daClient, s.cache, s.genesis, s.logger) - // Create async block retriever for background prefetching - asyncFetcher := da.NewAsyncBlockRetriever( - s.daClient, - s.logger, - s.config, - s.genesis.DAStartHeight, - s.genesis.DAEpochForcedInclusion*2, // prefetch window: blocks ahead to prefetch - ) - asyncFetcher.Start() - - s.fiRetriever = da.NewForcedInclusionRetriever(s.daClient, s.logger, s.genesis.DAStartHeight, s.genesis.DAEpochForcedInclusion, asyncFetcher) + s.fiRetriever = da.NewForcedInclusionRetriever(s.daClient, s.logger, s.config, s.genesis.DAStartHeight, s.genesis.DAEpochForcedInclusion) s.p2pHandler = NewP2PHandler(s.headerStore.Store(), s.dataStore.Store(), s.cache, s.genesis, s.logger) if currentHeight, err := s.store.Height(s.ctx); err != nil { s.logger.Error().Err(err).Msg("failed to set initial processed height for p2p handler") diff --git a/block/internal/syncing/syncer_forced_inclusion_test.go b/block/internal/syncing/syncer_forced_inclusion_test.go index e3219fd33..61c556e83 100644 --- a/block/internal/syncing/syncer_forced_inclusion_test.go +++ b/block/internal/syncing/syncer_forced_inclusion_test.go @@ -23,17 +23,6 @@ import ( "github.com/evstack/ev-node/types" ) -// createTestAsyncFetcher creates a minimal async fetcher for tests (without starting it) -func createTestAsyncFetcherForSyncer(client da.Client, gen genesis.Genesis) da.AsyncBlockRetriever { - return da.NewAsyncBlockRetriever( - client, - zerolog.Nop(), - config.DefaultConfig(), - gen.DAStartHeight, - 10, // prefetch 10 blocks - ) -} - func TestCalculateBlockFullness_HalfFull(t *testing.T) { s := &Syncer{} @@ -381,8 +370,8 @@ func TestVerifyForcedInclusionTxs_AllTransactionsIncluded(t *testing.T) { client.On("GetForcedInclusionNamespace").Return([]byte(cfg.DA.ForcedInclusionNamespace)).Maybe() client.On("HasForcedInclusionNamespace").Return(true).Maybe() daRetriever := NewDARetriever(client, cm, gen, zerolog.Nop()) - asyncFetcher := createTestAsyncFetcherForSyncer(client, gen) - fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) + fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer fiRetriever.Stop() s := NewSyncer( st, @@ -455,8 +444,8 @@ func TestVerifyForcedInclusionTxs_MissingTransactions(t *testing.T) { client.On("GetForcedInclusionNamespace").Return([]byte(cfg.DA.ForcedInclusionNamespace)).Maybe() client.On("HasForcedInclusionNamespace").Return(true).Maybe() daRetriever := NewDARetriever(client, cm, gen, zerolog.Nop()) - asyncFetcher := createTestAsyncFetcherForSyncer(client, gen) - fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) + fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer fiRetriever.Stop() s := NewSyncer( st, @@ -559,8 +548,8 @@ func TestVerifyForcedInclusionTxs_PartiallyIncluded(t *testing.T) { client.On("GetForcedInclusionNamespace").Return([]byte(cfg.DA.ForcedInclusionNamespace)).Maybe() client.On("HasForcedInclusionNamespace").Return(true).Maybe() daRetriever := NewDARetriever(client, cm, gen, zerolog.Nop()) - asyncFetcher := createTestAsyncFetcherForSyncer(client, gen) - fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) + fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer fiRetriever.Stop() s := NewSyncer( st, @@ -667,8 +656,8 @@ func TestVerifyForcedInclusionTxs_NoForcedTransactions(t *testing.T) { client.On("GetForcedInclusionNamespace").Return([]byte(cfg.DA.ForcedInclusionNamespace)).Maybe() client.On("HasForcedInclusionNamespace").Return(true).Maybe() daRetriever := NewDARetriever(client, cm, gen, zerolog.Nop()) - asyncFetcher := createTestAsyncFetcherForSyncer(client, gen) - fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) + fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer fiRetriever.Stop() s := NewSyncer( st, @@ -733,9 +722,10 @@ func TestVerifyForcedInclusionTxs_NamespaceNotConfigured(t *testing.T) { client.On("GetDataNamespace").Return([]byte(cfg.DA.DataNamespace)).Maybe() client.On("GetForcedInclusionNamespace").Return([]byte(nil)).Maybe() client.On("HasForcedInclusionNamespace").Return(false).Maybe() + client.On("GetForcedInclusionNamespace").Return([]byte(nil)).Maybe() daRetriever := NewDARetriever(client, cm, gen, zerolog.Nop()) - asyncFetcher := createTestAsyncFetcherForSyncer(client, gen) - fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) + fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer fiRetriever.Stop() s := NewSyncer( st, @@ -800,8 +790,8 @@ func TestVerifyForcedInclusionTxs_DeferralWithinEpoch(t *testing.T) { client.On("GetForcedInclusionNamespace").Return([]byte(cfg.DA.ForcedInclusionNamespace)).Maybe() client.On("HasForcedInclusionNamespace").Return(true).Maybe() daRetriever := NewDARetriever(client, cm, gen, zerolog.Nop()) - asyncFetcher := createTestAsyncFetcherForSyncer(client, gen) - fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) + fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer fiRetriever.Stop() s := NewSyncer( st, @@ -924,8 +914,8 @@ func TestVerifyForcedInclusionTxs_MaliciousAfterEpochEnd(t *testing.T) { client.On("GetForcedInclusionNamespace").Return([]byte(cfg.DA.ForcedInclusionNamespace)).Maybe() client.On("HasForcedInclusionNamespace").Return(true).Maybe() daRetriever := NewDARetriever(client, cm, gen, zerolog.Nop()) - asyncFetcher := createTestAsyncFetcherForSyncer(client, gen) - fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) + fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer fiRetriever.Stop() s := NewSyncer( st, @@ -1014,8 +1004,8 @@ func TestVerifyForcedInclusionTxs_SmoothingExceedsEpoch(t *testing.T) { client.On("HasForcedInclusionNamespace").Return(true).Maybe() daRetriever := NewDARetriever(client, cm, gen, zerolog.Nop()) - asyncFetcher := createTestAsyncFetcherForSyncer(client, gen) - fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), gen.DAStartHeight, gen.DAEpochForcedInclusion, asyncFetcher) + fiRetriever := da.NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer fiRetriever.Stop() s := NewSyncer( st, diff --git a/block/public.go b/block/public.go index 1c73ac6fa..a43ea9694 100644 --- a/block/public.go +++ b/block/public.go @@ -65,44 +65,19 @@ var ErrForceInclusionNotConfigured = da.ErrForceInclusionNotConfigured // ForcedInclusionEvent represents forced inclusion transactions retrieved from DA type ForcedInclusionEvent = da.ForcedInclusionEvent -// AsyncBlockRetriever provides background prefetching of individual DA blocks -type AsyncBlockRetriever interface { - Start() - Stop() - GetCachedBlock(ctx context.Context, daHeight uint64) (*da.BlockData, error) - UpdateCurrentHeight(height uint64) -} - // ForcedInclusionRetriever defines the interface for retrieving forced inclusion transactions from DA type ForcedInclusionRetriever interface { RetrieveForcedIncludedTxs(ctx context.Context, daHeight uint64) (*da.ForcedInclusionEvent, error) -} - -// NewAsyncBlockRetriever creates a new async block retriever for background prefetching. -// Parameters: -// - client: DA client for fetching data -// - config: Ev-node config -// - logger: structured logger -// - daStartHeight: genesis DA start height -// - prefetchWindow: how many blocks ahead to prefetch (10-20 recommended) -func NewAsyncBlockRetriever( - client DAClient, - cfg config.Config, - logger zerolog.Logger, - daStartHeight uint64, - prefetchWindow uint64, -) AsyncBlockRetriever { - return da.NewAsyncBlockRetriever(client, logger, cfg, daStartHeight, prefetchWindow) + Stop() } // NewForcedInclusionRetriever creates a new forced inclusion retriever. -// The asyncFetcher parameter is required for background prefetching of DA block data. -// It accepts either AsyncBlockRetriever (recommended) or AsyncEpochFetcher (deprecated) for backward compatibility. +// It internally creates and manages an AsyncBlockRetriever for background prefetching. func NewForcedInclusionRetriever( client DAClient, + cfg config.Config, logger zerolog.Logger, daStartHeight, daEpochSize uint64, - asyncFetcher AsyncBlockRetriever, ) ForcedInclusionRetriever { - return da.NewForcedInclusionRetriever(client, logger, daStartHeight, daEpochSize, asyncFetcher) + return da.NewForcedInclusionRetriever(client, logger, cfg, daStartHeight, daEpochSize) } diff --git a/pkg/sequencers/based/sequencer.go b/pkg/sequencers/based/sequencer.go index 9437fe4a8..1b6cd519e 100644 --- a/pkg/sequencers/based/sequencer.go +++ b/pkg/sequencers/based/sequencer.go @@ -26,7 +26,6 @@ var _ coresequencer.Sequencer = (*BasedSequencer)(nil) type BasedSequencer struct { logger zerolog.Logger - asyncFetcher block.AsyncBlockRetriever fiRetriever block.ForcedInclusionRetriever daHeight atomic.Uint64 checkpointStore *seqcommon.CheckpointStore @@ -80,17 +79,7 @@ func NewBasedSequencer( } } - // Create async block retriever for background prefetching - bs.asyncFetcher = block.NewAsyncBlockRetriever( - daClient, - cfg, - logger, - genesis.DAStartHeight, - genesis.DAEpochForcedInclusion*2, - ) - bs.asyncFetcher.Start() - - bs.fiRetriever = block.NewForcedInclusionRetriever(daClient, logger, genesis.DAStartHeight, genesis.DAEpochForcedInclusion, bs.asyncFetcher) + bs.fiRetriever = block.NewForcedInclusionRetriever(daClient, cfg, logger, genesis.DAStartHeight, genesis.DAEpochForcedInclusion) return bs, nil } diff --git a/pkg/sequencers/based/sequencer_test.go b/pkg/sequencers/based/sequencer_test.go index 0e274edd8..44b054535 100644 --- a/pkg/sequencers/based/sequencer_test.go +++ b/pkg/sequencers/based/sequencer_test.go @@ -32,6 +32,10 @@ func (m *MockForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Con return args.Get(0).(*block.ForcedInclusionEvent), args.Error(1) } +func (m *MockForcedInclusionRetriever) Stop() { + // No-op for mock +} + // MockDAClient is a mock implementation of block.FullDAClient for testing type MockDAClient struct { mock.Mock @@ -106,6 +110,9 @@ func createTestSequencer(t *testing.T, mockRetriever *MockForcedInclusionRetriev // Create mock DA client that wraps the retriever mockDAClient := &MockDAClient{retriever: mockRetriever} + // Mock the forced inclusion namespace call + mockDAClient.On("GetForcedInclusionNamespace").Return([]byte("test-forced-inclusion-ns")).Maybe() + mockDAClient.On("HasForcedInclusionNamespace").Return(true).Maybe() seq, err := NewBasedSequencer(mockDAClient, config.DefaultConfig(), db, gen, zerolog.Nop()) require.NoError(t, err) @@ -529,6 +536,8 @@ func TestBasedSequencer_CheckpointPersistence(t *testing.T) { // Create mock DA client mockDAClient := &MockDAClient{retriever: mockRetriever} + mockDAClient.On("GetForcedInclusionNamespace").Return([]byte("test-forced-inclusion-ns")).Maybe() + mockDAClient.On("HasForcedInclusionNamespace").Return(true).Maybe() // Create first sequencer seq1, err := NewBasedSequencer(mockDAClient, config.DefaultConfig(), db, gen, zerolog.Nop()) @@ -550,6 +559,8 @@ func TestBasedSequencer_CheckpointPersistence(t *testing.T) { // Create a new sequencer with the same datastore (simulating restart) mockDAClient2 := &MockDAClient{retriever: mockRetriever} + mockDAClient2.On("GetForcedInclusionNamespace").Return([]byte("test-forced-inclusion-ns")).Maybe() + mockDAClient2.On("HasForcedInclusionNamespace").Return(true).Maybe() seq2, err := NewBasedSequencer(mockDAClient2, config.DefaultConfig(), db, gen, zerolog.Nop()) require.NoError(t, err) diff --git a/pkg/sequencers/single/sequencer.go b/pkg/sequencers/single/sequencer.go index a23e3fa16..07c32c835 100644 --- a/pkg/sequencers/single/sequencer.go +++ b/pkg/sequencers/single/sequencer.go @@ -33,6 +33,7 @@ type Sequencer struct { logger zerolog.Logger genesis genesis.Genesis db ds.Batching + cfg config.Config Id []byte daClient block.FullDAClient @@ -41,7 +42,6 @@ type Sequencer struct { queue *BatchQueue // single queue for immediate availability // Forced inclusion support - asyncFetcher block.AsyncBlockRetriever fiRetriever block.ForcedInclusionRetriever daHeight atomic.Uint64 daStartHeight atomic.Uint64 @@ -66,6 +66,7 @@ func NewSequencer( db: db, logger: logger, daClient: daClient, + cfg: cfg, batchTime: cfg.Node.BlockTime.Duration, Id: id, queue: NewBatchQueue(db, "batches", maxQueueSize), @@ -109,16 +110,8 @@ func NewSequencer( // Determine initial DA height for forced inclusion initialDAHeight := s.getInitialDAStartHeight(context.Background()) - s.asyncFetcher = block.NewAsyncBlockRetriever( - daClient, - cfg, - logger, - initialDAHeight, - genesis.DAEpochForcedInclusion*2, - ) - s.asyncFetcher.Start() - s.fiRetriever = block.NewForcedInclusionRetriever(daClient, logger, initialDAHeight, genesis.DAEpochForcedInclusion, s.asyncFetcher) + s.fiRetriever = block.NewForcedInclusionRetriever(daClient, cfg, logger, initialDAHeight, genesis.DAEpochForcedInclusion) return s, nil } @@ -187,7 +180,11 @@ func (c *Sequencer) GetNextBatch(ctx context.Context, req coresequencer.GetNextB } // override forced inclusion retriever, as the da start height have been updated - c.fiRetriever = block.NewForcedInclusionRetriever(c.daClient, c.logger, c.getInitialDAStartHeight(ctx), c.genesis.DAEpochForcedInclusion, c.asyncFetcher) + // Stop the old retriever first + if c.fiRetriever != nil { + c.fiRetriever.Stop() + } + c.fiRetriever = block.NewForcedInclusionRetriever(c.daClient, c.cfg, c.logger, c.getInitialDAStartHeight(ctx), c.genesis.DAEpochForcedInclusion) } // If we have no cached transactions or we've consumed all from the current cache, From 02d843af2698e04c79de2b7a3096b5b9bf8afd80 Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Fri, 9 Jan 2026 00:32:42 +0100 Subject: [PATCH 14/18] updates --- block/internal/syncing/syncer.go | 1 - 1 file changed, 1 deletion(-) diff --git a/block/internal/syncing/syncer.go b/block/internal/syncing/syncer.go index 4295714ea..0d6d56f0d 100644 --- a/block/internal/syncing/syncer.go +++ b/block/internal/syncing/syncer.go @@ -187,7 +187,6 @@ func (s *Syncer) Start(ctx context.Context) error { // Initialize handlers s.daRetriever = NewDARetriever(s.daClient, s.cache, s.genesis, s.logger) - s.fiRetriever = da.NewForcedInclusionRetriever(s.daClient, s.logger, s.config, s.genesis.DAStartHeight, s.genesis.DAEpochForcedInclusion) s.p2pHandler = NewP2PHandler(s.headerStore.Store(), s.dataStore.Store(), s.cache, s.genesis, s.logger) if currentHeight, err := s.store.Height(s.ctx); err != nil { From a3ab53558c6ff1c76d0d7efefa98763615a0ba9d Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Fri, 9 Jan 2026 09:13:39 +0100 Subject: [PATCH 15/18] update logs --- block/internal/da/async_block_retriever.go | 33 +++++++++---------- .../internal/da/forced_inclusion_retriever.go | 6 ++-- 2 files changed, 17 insertions(+), 22 deletions(-) diff --git a/block/internal/da/async_block_retriever.go b/block/internal/da/async_block_retriever.go index c34873999..5c147de30 100644 --- a/block/internal/da/async_block_retriever.go +++ b/block/internal/da/async_block_retriever.go @@ -94,7 +94,7 @@ func NewAsyncBlockRetriever( func (f *asyncBlockRetriever) Start() { f.wg.Add(1) go f.backgroundFetchLoop() - f.logger.Info(). + f.logger.Debug(). Uint64("da_start_height", f.daStartHeight). Uint64("prefetch_window", f.prefetchWindow). Dur("poll_interval", f.pollInterval). @@ -103,7 +103,7 @@ func (f *asyncBlockRetriever) Start() { // Stop gracefully stops the background prefetching process. func (f *asyncBlockRetriever) Stop() { - f.logger.Info().Msg("stopping async block retriever") + f.logger.Debug().Msg("stopping async block retriever") f.cancel() f.wg.Wait() } @@ -125,6 +125,10 @@ func (f *asyncBlockRetriever) UpdateCurrentHeight(height uint64) { } } +func newBlockDataKey(height uint64) ds.Key { + return ds.NewKey(fmt.Sprintf("/block/%d", height)) +} + // GetCachedBlock retrieves a cached block from memory. // Returns nil if the block is not cached. func (f *asyncBlockRetriever) GetCachedBlock(ctx context.Context, daHeight uint64) (*BlockData, error) { @@ -136,9 +140,7 @@ func (f *asyncBlockRetriever) GetCachedBlock(ctx context.Context, daHeight uint6 return nil, fmt.Errorf("DA height %d is before the configured start height %d", daHeight, f.daStartHeight) } - // Try to get from cache - key := ds.NewKey(fmt.Sprintf("/block/%d", daHeight)) - + key := newBlockDataKey(daHeight) data, err := f.cache.Get(ctx, key) if err != nil { if errors.Is(err, ds.ErrNotFound) { @@ -197,7 +199,7 @@ func (f *asyncBlockRetriever) prefetchBlocks() { targetHeight := currentHeight + i // Check if already cached - key := ds.NewKey(fmt.Sprintf("/block/%d", targetHeight)) + key := newBlockDataKey(targetHeight) _, err := f.cache.Get(f.ctx, key) if err == nil { // Already cached @@ -235,7 +237,7 @@ func (f *asyncBlockRetriever) fetchAndCacheBlock(height uint64) { case datypes.StatusNotFound: f.logger.Debug(). Uint64("height", height). - Msg("no forced inclusion blobs at height") + Msg("no blobs at height") // Cache empty result to avoid re-fetching case datypes.StatusSuccess: // Process each blob @@ -247,9 +249,9 @@ func (f *asyncBlockRetriever) fetchAndCacheBlock(height uint64) { f.logger.Debug(). Uint64("height", height). Int("blob_count", len(result.Data)). - Msg("processed forced inclusion blobs for prefetch") + Msg("processed blobs for prefetch") default: - f.logger.Warn(). + f.logger.Debug(). Uint64("height", height). Str("status", result.Message). Msg("failed to retrieve block - will retry") @@ -271,9 +273,8 @@ func (f *asyncBlockRetriever) fetchAndCacheBlock(height uint64) { return } - key := ds.NewKey(fmt.Sprintf("/block/%d", height)) + key := newBlockDataKey(height) err = f.cache.Put(f.ctx, key, data) - if err != nil { f.logger.Error(). Err(err). @@ -282,7 +283,7 @@ func (f *asyncBlockRetriever) fetchAndCacheBlock(height uint64) { return } - f.logger.Info(). + f.logger.Debug(). Uint64("height", height). Int("blob_count", len(block.Blobs)). Msg("successfully prefetched and cached block") @@ -301,7 +302,7 @@ func (f *asyncBlockRetriever) cleanupOldBlocks(currentHeight uint64) { query := dsq.Query{Prefix: "/block/"} results, err := f.cache.Query(f.ctx, query) if err != nil { - f.logger.Warn().Err(err).Msg("failed to query cache for cleanup") + f.logger.Debug().Err(err).Msg("failed to query cache for cleanup") return } defer results.Close() @@ -321,14 +322,10 @@ func (f *asyncBlockRetriever) cleanupOldBlocks(currentHeight uint64) { if height < cleanupThreshold { if err := f.cache.Delete(f.ctx, key); err != nil { - f.logger.Warn(). + f.logger.Debug(). Err(err). Uint64("height", height). Msg("failed to delete old block from cache") - } else { - f.logger.Debug(). - Uint64("height", height). - Msg("cleaned up old block from cache") } } } diff --git a/block/internal/da/forced_inclusion_retriever.go b/block/internal/da/forced_inclusion_retriever.go index b5e28a7a3..133a83493 100644 --- a/block/internal/da/forced_inclusion_retriever.go +++ b/block/internal/da/forced_inclusion_retriever.go @@ -21,7 +21,7 @@ type ForcedInclusionRetriever struct { logger zerolog.Logger daEpochSize uint64 daStartHeight uint64 - asyncFetcher AsyncBlockRetriever // Fetches individual blocks in background + asyncFetcher AsyncBlockRetriever } // ForcedInclusionEvent contains forced inclusion transactions retrieved from DA. @@ -64,9 +64,7 @@ func NewForcedInclusionRetriever( // Stop stops the background prefetcher. func (r *ForcedInclusionRetriever) Stop() { - if r.asyncFetcher != nil { - r.asyncFetcher.Stop() - } + r.asyncFetcher.Stop() } // RetrieveForcedIncludedTxs retrieves forced inclusion transactions at the given DA height. From 59b33b118721ae864db9e45d9857e7332d15c4c1 Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Fri, 9 Jan 2026 15:53:26 +0100 Subject: [PATCH 16/18] test: use proper mocks --- .mockery.yaml | 7 + .../internal/da/forced_inclusion_retriever.go | 3 +- block/public.go | 2 +- pkg/sequencers/based/sequencer_test.go | 152 +++++------------- .../common/forced_inclusion_retriever_mock.go | 140 ++++++++++++++++ pkg/sequencers/single/sequencer_test.go | 23 +-- 6 files changed, 198 insertions(+), 129 deletions(-) create mode 100644 pkg/sequencers/common/forced_inclusion_retriever_mock.go diff --git a/.mockery.yaml b/.mockery.yaml index f1d971730..b5e092cd9 100644 --- a/.mockery.yaml +++ b/.mockery.yaml @@ -80,3 +80,10 @@ packages: dir: ./pkg/da/jsonrpc/mocks pkgname: mocks filename: header_module_mock.go + github.com/evstack/ev-node/block: + interfaces: + ForcedInclusionRetriever: + config: + dir: ./pkg/sequencers/common + pkgname: common + filename: forced_inclusion_retriever_mock.go diff --git a/block/internal/da/forced_inclusion_retriever.go b/block/internal/da/forced_inclusion_retriever.go index 133a83493..f23746601 100644 --- a/block/internal/da/forced_inclusion_retriever.go +++ b/block/internal/da/forced_inclusion_retriever.go @@ -6,10 +6,11 @@ import ( "fmt" "time" + "github.com/rs/zerolog" + "github.com/evstack/ev-node/pkg/config" datypes "github.com/evstack/ev-node/pkg/da/types" "github.com/evstack/ev-node/types" - "github.com/rs/zerolog" ) // ErrForceInclusionNotConfigured is returned when the forced inclusion namespace is not configured. diff --git a/block/public.go b/block/public.go index a43ea9694..1e3e0de07 100644 --- a/block/public.go +++ b/block/public.go @@ -67,7 +67,7 @@ type ForcedInclusionEvent = da.ForcedInclusionEvent // ForcedInclusionRetriever defines the interface for retrieving forced inclusion transactions from DA type ForcedInclusionRetriever interface { - RetrieveForcedIncludedTxs(ctx context.Context, daHeight uint64) (*da.ForcedInclusionEvent, error) + RetrieveForcedIncludedTxs(ctx context.Context, daHeight uint64) (*ForcedInclusionEvent, error) Stop() } diff --git a/pkg/sequencers/based/sequencer_test.go b/pkg/sequencers/based/sequencer_test.go index 44b054535..493694227 100644 --- a/pkg/sequencers/based/sequencer_test.go +++ b/pkg/sequencers/based/sequencer_test.go @@ -17,102 +17,30 @@ import ( "github.com/evstack/ev-node/pkg/config" datypes "github.com/evstack/ev-node/pkg/da/types" "github.com/evstack/ev-node/pkg/genesis" + "github.com/evstack/ev-node/pkg/sequencers/common" + "github.com/evstack/ev-node/test/mocks" ) -// MockForcedInclusionRetriever is a mock implementation of ForcedInclusionRetriever for testing -type MockForcedInclusionRetriever struct { - mock.Mock +// MockFullDAClient combines MockClient and MockVerifier to implement FullDAClient +type MockFullDAClient struct { + *mocks.MockClient + *mocks.MockVerifier } -func (m *MockForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Context, daHeight uint64) (*block.ForcedInclusionEvent, error) { - args := m.Called(ctx, daHeight) - if args.Get(0) == nil { - return nil, args.Error(1) - } - return args.Get(0).(*block.ForcedInclusionEvent), args.Error(1) -} - -func (m *MockForcedInclusionRetriever) Stop() { - // No-op for mock -} - -// MockDAClient is a mock implementation of block.FullDAClient for testing -type MockDAClient struct { - mock.Mock - retriever *MockForcedInclusionRetriever -} - -// Client interface methods -func (m *MockDAClient) Submit(ctx context.Context, data [][]byte, gasPrice float64, namespace []byte, options []byte) datypes.ResultSubmit { - args := m.Called(ctx, data, gasPrice, namespace, options) - return args.Get(0).(datypes.ResultSubmit) -} - -func (m *MockDAClient) Retrieve(ctx context.Context, height uint64, namespace []byte) datypes.ResultRetrieve { - args := m.Called(ctx, height, namespace) - return args.Get(0).(datypes.ResultRetrieve) -} - -func (m *MockDAClient) Get(ctx context.Context, ids []datypes.ID, namespace []byte) ([]datypes.Blob, error) { - args := m.Called(ctx, ids, namespace) - if args.Get(0) == nil { - return nil, args.Error(1) - } - return args.Get(0).([]datypes.Blob), args.Error(1) -} - -func (m *MockDAClient) GetHeaderNamespace() []byte { - args := m.Called() - if args.Get(0) == nil { - return nil - } - return args.Get(0).([]byte) -} - -func (m *MockDAClient) GetDataNamespace() []byte { - args := m.Called() - if args.Get(0) == nil { - return nil - } - return args.Get(0).([]byte) -} - -func (m *MockDAClient) GetForcedInclusionNamespace() []byte { - args := m.Called() - if args.Get(0) == nil { - return nil - } - return args.Get(0).([]byte) -} - -func (m *MockDAClient) HasForcedInclusionNamespace() bool { - args := m.Called() - return args.Get(0).(bool) -} - -// Verifier interface methods -func (m *MockDAClient) GetProofs(ctx context.Context, ids []datypes.ID, namespace []byte) ([]datypes.Proof, error) { - args := m.Called(ctx, ids, namespace) - return args.Get(0).([]datypes.Proof), args.Error(1) -} - -func (m *MockDAClient) Validate(ctx context.Context, ids []datypes.ID, proofs []datypes.Proof, namespace []byte) ([]bool, error) { - args := m.Called(ctx, ids, proofs, namespace) - return args.Get(0).([]bool), args.Error(1) -} - -// createTestSequencer is a helper function to create a sequencer for testing -func createTestSequencer(t *testing.T, mockRetriever *MockForcedInclusionRetriever, gen genesis.Genesis) *BasedSequencer { +func createTestSequencer(t *testing.T, mockRetriever *common.MockForcedInclusionRetriever, gen genesis.Genesis) *BasedSequencer { t.Helper() // Create in-memory datastore db := syncds.MutexWrap(ds.NewMapDatastore()) - // Create mock DA client that wraps the retriever - mockDAClient := &MockDAClient{retriever: mockRetriever} + // Create mock DA client + mockDAClient := &MockFullDAClient{ + MockClient: mocks.NewMockClient(t), + MockVerifier: mocks.NewMockVerifier(t), + } // Mock the forced inclusion namespace call - mockDAClient.On("GetForcedInclusionNamespace").Return([]byte("test-forced-inclusion-ns")).Maybe() - mockDAClient.On("HasForcedInclusionNamespace").Return(true).Maybe() + mockDAClient.MockClient.On("GetForcedInclusionNamespace").Return([]byte("test-forced-inclusion-ns")).Maybe() + mockDAClient.MockClient.On("HasForcedInclusionNamespace").Return(true).Maybe() seq, err := NewBasedSequencer(mockDAClient, config.DefaultConfig(), db, gen, zerolog.Nop()) require.NoError(t, err) @@ -124,7 +52,7 @@ func createTestSequencer(t *testing.T, mockRetriever *MockForcedInclusionRetriev } func TestBasedSequencer_SubmitBatchTxs(t *testing.T) { - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) gen := genesis.Genesis{ ChainID: "test-chain", DAEpochForcedInclusion: 10, @@ -151,7 +79,7 @@ func TestBasedSequencer_SubmitBatchTxs(t *testing.T) { func TestBasedSequencer_GetNextBatch_WithForcedTxs(t *testing.T) { testBlobs := [][]byte{[]byte("tx1"), []byte("tx2")} - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) mockRetriever.On("RetrieveForcedIncludedTxs", mock.Anything, uint64(100)).Return(&block.ForcedInclusionEvent{ Txs: testBlobs, StartDaHeight: 100, @@ -187,7 +115,7 @@ func TestBasedSequencer_GetNextBatch_WithForcedTxs(t *testing.T) { } func TestBasedSequencer_GetNextBatch_EmptyDA(t *testing.T) { - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) mockRetriever.On("RetrieveForcedIncludedTxs", mock.Anything, uint64(100)).Return(&block.ForcedInclusionEvent{ Txs: [][]byte{}, StartDaHeight: 100, @@ -218,7 +146,7 @@ func TestBasedSequencer_GetNextBatch_EmptyDA(t *testing.T) { } func TestBasedSequencer_GetNextBatch_NotConfigured(t *testing.T) { - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) mockRetriever.On("RetrieveForcedIncludedTxs", mock.Anything, uint64(100)).Return(nil, block.ErrForceInclusionNotConfigured) gen := genesis.Genesis{ @@ -249,7 +177,7 @@ func TestBasedSequencer_GetNextBatch_WithMaxBytes(t *testing.T) { tx3 := make([]byte, 200) testBlobs := [][]byte{tx1, tx2, tx3} - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) mockRetriever.On("RetrieveForcedIncludedTxs", mock.Anything, uint64(100)).Return(&block.ForcedInclusionEvent{ Txs: testBlobs, StartDaHeight: 100, @@ -303,7 +231,7 @@ func TestBasedSequencer_GetNextBatch_MultipleDABlocks(t *testing.T) { testBlobs1 := [][]byte{[]byte("tx1"), []byte("tx2")} testBlobs2 := [][]byte{[]byte("tx3"), []byte("tx4")} - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) // First DA block mockRetriever.On("RetrieveForcedIncludedTxs", mock.Anything, uint64(100)).Return(&block.ForcedInclusionEvent{ Txs: testBlobs1, @@ -354,7 +282,7 @@ func TestBasedSequencer_GetNextBatch_MultipleDABlocks(t *testing.T) { func TestBasedSequencer_GetNextBatch_ResumesFromCheckpoint(t *testing.T) { testBlobs := [][]byte{[]byte("tx1"), []byte("tx2"), []byte("tx3")} - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) gen := genesis.Genesis{ ChainID: "test-chain", @@ -392,7 +320,7 @@ func TestBasedSequencer_GetNextBatch_ForcedInclusionExceedsMaxBytes(t *testing.T largeTx := make([]byte, 2000) testBlobs := [][]byte{largeTx} - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) mockRetriever.On("RetrieveForcedIncludedTxs", mock.Anything, uint64(100)).Return(&block.ForcedInclusionEvent{ Txs: testBlobs, StartDaHeight: 100, @@ -423,7 +351,7 @@ func TestBasedSequencer_GetNextBatch_ForcedInclusionExceedsMaxBytes(t *testing.T } func TestBasedSequencer_VerifyBatch(t *testing.T) { - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) gen := genesis.Genesis{ ChainID: "test-chain", DAEpochForcedInclusion: 10, @@ -444,7 +372,7 @@ func TestBasedSequencer_VerifyBatch(t *testing.T) { } func TestBasedSequencer_SetDAHeight(t *testing.T) { - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) gen := genesis.Genesis{ ChainID: "test-chain", DAStartHeight: 100, @@ -462,7 +390,7 @@ func TestBasedSequencer_SetDAHeight(t *testing.T) { } func TestBasedSequencer_GetNextBatch_ErrorHandling(t *testing.T) { - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) mockRetriever.On("RetrieveForcedIncludedTxs", mock.Anything, uint64(100)).Return(nil, block.ErrForceInclusionNotConfigured) gen := genesis.Genesis{ @@ -487,7 +415,7 @@ func TestBasedSequencer_GetNextBatch_ErrorHandling(t *testing.T) { } func TestBasedSequencer_GetNextBatch_HeightFromFuture(t *testing.T) { - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) mockRetriever.On("RetrieveForcedIncludedTxs", mock.Anything, uint64(100)).Return(nil, datypes.ErrHeightFromFuture) gen := genesis.Genesis{ @@ -518,7 +446,7 @@ func TestBasedSequencer_GetNextBatch_HeightFromFuture(t *testing.T) { func TestBasedSequencer_CheckpointPersistence(t *testing.T) { testBlobs := [][]byte{[]byte("tx1"), []byte("tx2")} - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) mockRetriever.On("RetrieveForcedIncludedTxs", mock.Anything, uint64(100)).Return(&block.ForcedInclusionEvent{ Txs: testBlobs, StartDaHeight: 100, @@ -535,9 +463,12 @@ func TestBasedSequencer_CheckpointPersistence(t *testing.T) { db := syncds.MutexWrap(ds.NewMapDatastore()) // Create mock DA client - mockDAClient := &MockDAClient{retriever: mockRetriever} - mockDAClient.On("GetForcedInclusionNamespace").Return([]byte("test-forced-inclusion-ns")).Maybe() - mockDAClient.On("HasForcedInclusionNamespace").Return(true).Maybe() + mockDAClient := &MockFullDAClient{ + MockClient: mocks.NewMockClient(t), + MockVerifier: mocks.NewMockVerifier(t), + } + mockDAClient.MockClient.On("GetForcedInclusionNamespace").Return([]byte("test-forced-inclusion-ns")).Maybe() + mockDAClient.MockClient.On("HasForcedInclusionNamespace").Return(true).Maybe() // Create first sequencer seq1, err := NewBasedSequencer(mockDAClient, config.DefaultConfig(), db, gen, zerolog.Nop()) @@ -558,9 +489,12 @@ func TestBasedSequencer_CheckpointPersistence(t *testing.T) { assert.Equal(t, 2, len(resp.Batch.Transactions)) // Create a new sequencer with the same datastore (simulating restart) - mockDAClient2 := &MockDAClient{retriever: mockRetriever} - mockDAClient2.On("GetForcedInclusionNamespace").Return([]byte("test-forced-inclusion-ns")).Maybe() - mockDAClient2.On("HasForcedInclusionNamespace").Return(true).Maybe() + mockDAClient2 := &MockFullDAClient{ + MockClient: mocks.NewMockClient(t), + MockVerifier: mocks.NewMockVerifier(t), + } + mockDAClient2.MockClient.On("GetForcedInclusionNamespace").Return([]byte("test-forced-inclusion-ns")).Maybe() + mockDAClient2.MockClient.On("HasForcedInclusionNamespace").Return(true).Maybe() seq2, err := NewBasedSequencer(mockDAClient2, config.DefaultConfig(), db, gen, zerolog.Nop()) require.NoError(t, err) @@ -575,7 +509,7 @@ func TestBasedSequencer_CheckpointPersistence(t *testing.T) { } func TestBasedSequencer_GetNextBatch_EmptyDABatch_IncreasesDAHeight(t *testing.T) { - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) // First DA block returns empty transactions mockRetriever.On("RetrieveForcedIncludedTxs", mock.Anything, uint64(100)).Return(&block.ForcedInclusionEvent{ @@ -642,7 +576,7 @@ func TestBasedSequencer_GetNextBatch_TimestampAdjustment(t *testing.T) { testBlobs := [][]byte{[]byte("tx1"), []byte("tx2"), []byte("tx3")} daEndTime := time.Date(2024, 1, 1, 12, 0, 0, 0, time.UTC) - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) mockRetriever.On("RetrieveForcedIncludedTxs", mock.Anything, uint64(100)).Return(&block.ForcedInclusionEvent{ Txs: testBlobs, StartDaHeight: 100, @@ -684,7 +618,7 @@ func TestBasedSequencer_GetNextBatch_TimestampAdjustment_PartialBatch(t *testing testBlobs := [][]byte{tx1, tx2, tx3} daEndTime := time.Date(2024, 1, 1, 12, 0, 0, 0, time.UTC) - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) mockRetriever.On("RetrieveForcedIncludedTxs", mock.Anything, uint64(100)).Return(&block.ForcedInclusionEvent{ Txs: testBlobs, StartDaHeight: 100, @@ -740,7 +674,7 @@ func TestBasedSequencer_GetNextBatch_TimestampAdjustment_EmptyBatch(t *testing.T // Test that timestamp is zero when batch is empty daEndTime := time.Date(2024, 1, 1, 12, 0, 0, 0, time.UTC) - mockRetriever := new(MockForcedInclusionRetriever) + mockRetriever := common.NewMockForcedInclusionRetriever(t) mockRetriever.On("RetrieveForcedIncludedTxs", mock.Anything, uint64(100)).Return(&block.ForcedInclusionEvent{ Txs: [][]byte{}, StartDaHeight: 100, diff --git a/pkg/sequencers/common/forced_inclusion_retriever_mock.go b/pkg/sequencers/common/forced_inclusion_retriever_mock.go new file mode 100644 index 000000000..c90b84f7f --- /dev/null +++ b/pkg/sequencers/common/forced_inclusion_retriever_mock.go @@ -0,0 +1,140 @@ +// Code generated by mockery; DO NOT EDIT. +// github.com/vektra/mockery +// template: testify + +package common + +import ( + "context" + + "github.com/evstack/ev-node/block" + mock "github.com/stretchr/testify/mock" +) + +// NewMockForcedInclusionRetriever creates a new instance of MockForcedInclusionRetriever. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewMockForcedInclusionRetriever(t interface { + mock.TestingT + Cleanup(func()) +}) *MockForcedInclusionRetriever { + mock := &MockForcedInclusionRetriever{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} + +// MockForcedInclusionRetriever is an autogenerated mock type for the ForcedInclusionRetriever type +type MockForcedInclusionRetriever struct { + mock.Mock +} + +type MockForcedInclusionRetriever_Expecter struct { + mock *mock.Mock +} + +func (_m *MockForcedInclusionRetriever) EXPECT() *MockForcedInclusionRetriever_Expecter { + return &MockForcedInclusionRetriever_Expecter{mock: &_m.Mock} +} + +// RetrieveForcedIncludedTxs provides a mock function for the type MockForcedInclusionRetriever +func (_mock *MockForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Context, daHeight uint64) (*block.ForcedInclusionEvent, error) { + ret := _mock.Called(ctx, daHeight) + + if len(ret) == 0 { + panic("no return value specified for RetrieveForcedIncludedTxs") + } + + var r0 *block.ForcedInclusionEvent + var r1 error + if returnFunc, ok := ret.Get(0).(func(context.Context, uint64) (*block.ForcedInclusionEvent, error)); ok { + return returnFunc(ctx, daHeight) + } + if returnFunc, ok := ret.Get(0).(func(context.Context, uint64) *block.ForcedInclusionEvent); ok { + r0 = returnFunc(ctx, daHeight) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*block.ForcedInclusionEvent) + } + } + if returnFunc, ok := ret.Get(1).(func(context.Context, uint64) error); ok { + r1 = returnFunc(ctx, daHeight) + } else { + r1 = ret.Error(1) + } + return r0, r1 +} + +// MockForcedInclusionRetriever_RetrieveForcedIncludedTxs_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RetrieveForcedIncludedTxs' +type MockForcedInclusionRetriever_RetrieveForcedIncludedTxs_Call struct { + *mock.Call +} + +// RetrieveForcedIncludedTxs is a helper method to define mock.On call +// - ctx context.Context +// - daHeight uint64 +func (_e *MockForcedInclusionRetriever_Expecter) RetrieveForcedIncludedTxs(ctx interface{}, daHeight interface{}) *MockForcedInclusionRetriever_RetrieveForcedIncludedTxs_Call { + return &MockForcedInclusionRetriever_RetrieveForcedIncludedTxs_Call{Call: _e.mock.On("RetrieveForcedIncludedTxs", ctx, daHeight)} +} + +func (_c *MockForcedInclusionRetriever_RetrieveForcedIncludedTxs_Call) Run(run func(ctx context.Context, daHeight uint64)) *MockForcedInclusionRetriever_RetrieveForcedIncludedTxs_Call { + _c.Call.Run(func(args mock.Arguments) { + var arg0 context.Context + if args[0] != nil { + arg0 = args[0].(context.Context) + } + var arg1 uint64 + if args[1] != nil { + arg1 = args[1].(uint64) + } + run( + arg0, + arg1, + ) + }) + return _c +} + +func (_c *MockForcedInclusionRetriever_RetrieveForcedIncludedTxs_Call) Return(v *block.ForcedInclusionEvent, err error) *MockForcedInclusionRetriever_RetrieveForcedIncludedTxs_Call { + _c.Call.Return(v, err) + return _c +} + +func (_c *MockForcedInclusionRetriever_RetrieveForcedIncludedTxs_Call) RunAndReturn(run func(ctx context.Context, daHeight uint64) (*block.ForcedInclusionEvent, error)) *MockForcedInclusionRetriever_RetrieveForcedIncludedTxs_Call { + _c.Call.Return(run) + return _c +} + +// Stop provides a mock function for the type MockForcedInclusionRetriever +func (_mock *MockForcedInclusionRetriever) Stop() { + _mock.Called() + return +} + +// MockForcedInclusionRetriever_Stop_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Stop' +type MockForcedInclusionRetriever_Stop_Call struct { + *mock.Call +} + +// Stop is a helper method to define mock.On call +func (_e *MockForcedInclusionRetriever_Expecter) Stop() *MockForcedInclusionRetriever_Stop_Call { + return &MockForcedInclusionRetriever_Stop_Call{Call: _e.mock.On("Stop")} +} + +func (_c *MockForcedInclusionRetriever_Stop_Call) Run(run func()) *MockForcedInclusionRetriever_Stop_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockForcedInclusionRetriever_Stop_Call) Return() *MockForcedInclusionRetriever_Stop_Call { + _c.Call.Return() + return _c +} + +func (_c *MockForcedInclusionRetriever_Stop_Call) RunAndReturn(run func()) *MockForcedInclusionRetriever_Stop_Call { + _c.Run(run) + return _c +} diff --git a/pkg/sequencers/single/sequencer_test.go b/pkg/sequencers/single/sequencer_test.go index cf145672d..b73d49bfd 100644 --- a/pkg/sequencers/single/sequencer_test.go +++ b/pkg/sequencers/single/sequencer_test.go @@ -18,33 +18,20 @@ import ( "github.com/evstack/ev-node/pkg/config" datypes "github.com/evstack/ev-node/pkg/da/types" "github.com/evstack/ev-node/pkg/genesis" - damocks "github.com/evstack/ev-node/test/mocks" + "github.com/evstack/ev-node/test/mocks" "github.com/evstack/ev-node/test/testda" ) -// MockForcedInclusionRetriever is a mock implementation of DARetriever for testing -type MockForcedInclusionRetriever struct { - mock.Mock -} - -func (m *MockForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Context, daHeight uint64) (*block.ForcedInclusionEvent, error) { - args := m.Called(ctx, daHeight) - if args.Get(0) == nil { - return nil, args.Error(1) - } - return args.Get(0).(*block.ForcedInclusionEvent), args.Error(1) -} - // MockFullDAClient combines MockClient and MockVerifier to implement FullDAClient type MockFullDAClient struct { - *damocks.MockClient - *damocks.MockVerifier + *mocks.MockClient + *mocks.MockVerifier } func newMockFullDAClient(t *testing.T) *MockFullDAClient { return &MockFullDAClient{ - MockClient: damocks.NewMockClient(t), - MockVerifier: damocks.NewMockVerifier(t), + MockClient: mocks.NewMockClient(t), + MockVerifier: mocks.NewMockVerifier(t), } } From ea9c055024390a1a823dc5cfb455e8607e0afe44 Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Fri, 9 Jan 2026 16:32:41 +0100 Subject: [PATCH 17/18] refactor: Combine cached and sync DA fetch results --- .../internal/da/forced_inclusion_retriever.go | 100 +++++++----------- 1 file changed, 41 insertions(+), 59 deletions(-) diff --git a/block/internal/da/forced_inclusion_retriever.go b/block/internal/da/forced_inclusion_retriever.go index f23746601..7b07b7d5d 100644 --- a/block/internal/da/forced_inclusion_retriever.go +++ b/block/internal/da/forced_inclusion_retriever.go @@ -132,24 +132,18 @@ func (r *ForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Context missingHeights = append(missingHeights, h) continue } - if block == nil { - // Cache miss + if block == nil { // Cache miss missingHeights = append(missingHeights, h) - } else { - // Cache hit + } else { // Cache hit cachedBlocks[h] = block - r.logger.Debug(). - Uint64("height", h). - Int("blob_count", len(block.Blobs)). - Msg("using cached block from async fetcher") } } - // Fetch missing heights synchronously + // Fetch missing heights synchronously and store in map + syncFetchedBlocks := make(map[uint64]*BlockData) var processErrs error for _, h := range missingHeights { result := r.client.Retrieve(ctx, h, r.client.GetForcedInclusionNamespace()) - if result.Code == datypes.StatusHeightFromFuture { r.logger.Debug(). Uint64("height", h). @@ -157,14 +151,40 @@ func (r *ForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Context return nil, fmt.Errorf("%w: height %d not yet available", datypes.ErrHeightFromFuture, h) } - err := r.processRetrieveResult(event, result, h) - processErrs = errors.Join(processErrs, err) + if result.Code == datypes.StatusNotFound { + r.logger.Debug().Uint64("height", h).Msg("no forced inclusion blobs at height") + continue + } + + if result.Code != datypes.StatusSuccess { + err := fmt.Errorf("failed to retrieve forced inclusion blobs at height %d: %s", h, result.Message) + processErrs = errors.Join(processErrs, err) + continue + } + + // Store the sync-fetched block data + syncFetchedBlocks[h] = &BlockData{ + Blobs: result.Data, + Timestamp: result.Timestamp, + } } - // Process cached blocks in order + // Process all blocks in height order for _, h := range heights { - if block, ok := cachedBlocks[h]; ok { - // Add blobs from cached block + var block *BlockData + var source string + + // Check cached blocks first, then sync-fetched + if cachedBlock, ok := cachedBlocks[h]; ok { + block = cachedBlock + source = "cache" + } else if syncBlock, ok := syncFetchedBlocks[h]; ok { + block = syncBlock + source = "sync" + } + + if block != nil { + // Add blobs from block for _, blob := range block.Blobs { if len(blob) > 0 { event.Txs = append(event.Txs, blob) @@ -179,8 +199,13 @@ func (r *ForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Context r.logger.Debug(). Uint64("height", h). Int("blob_count", len(block.Blobs)). - Msg("added blobs from cached block") + Str("source", source). + Msg("added blobs from block") } + + // Clean up maps to prevent unbounded memory growth + delete(cachedBlocks, h) + delete(syncFetchedBlocks, h) } // any error during process, need to retry at next call @@ -200,48 +225,5 @@ func (r *ForcedInclusionRetriever) RetrieveForcedIncludedTxs(ctx context.Context }, nil } - r.logger.Info(). - Uint64("da_height", daHeight). - Uint64("epoch_start", epochStart). - Uint64("epoch_end", epochEnd). - Int("tx_count", len(event.Txs)). - Int("cached_blocks", len(cachedBlocks)). - Int("sync_fetched_blocks", len(missingHeights)). - Msg("successfully retrieved forced inclusion epoch") - return event, nil } - -// processRetrieveResult processes the result from a DA retrieve operation. -func (r *ForcedInclusionRetriever) processRetrieveResult( - event *ForcedInclusionEvent, - result datypes.ResultRetrieve, - height uint64, -) error { - if result.Code == datypes.StatusNotFound { - r.logger.Debug().Uint64("height", height).Msg("no forced inclusion blobs at height") - return nil - } - - if result.Code != datypes.StatusSuccess { - return fmt.Errorf("failed to retrieve forced inclusion blobs at height %d: %s", height, result.Message) - } - - // Process each blob as a transaction - for _, blob := range result.Data { - if len(blob) > 0 { - event.Txs = append(event.Txs, blob) - } - } - - if result.Timestamp.After(event.Timestamp) { - event.Timestamp = result.Timestamp - } - - r.logger.Debug(). - Uint64("height", height). - Int("blob_count", len(result.Data)). - Msg("processed forced inclusion blobs") - - return nil -} From a7fdca7e404532a442e6c79e20dadffd6b3b0570 Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Fri, 9 Jan 2026 16:36:52 +0100 Subject: [PATCH 18/18] update tests --- .../da/forced_inclusion_retriever_test.go | 168 ++++++++++-------- 1 file changed, 96 insertions(+), 72 deletions(-) diff --git a/block/internal/da/forced_inclusion_retriever_test.go b/block/internal/da/forced_inclusion_retriever_test.go index 4961d74d4..446b655be 100644 --- a/block/internal/da/forced_inclusion_retriever_test.go +++ b/block/internal/da/forced_inclusion_retriever_test.go @@ -203,88 +203,112 @@ func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_MultiHeightEpoch(t * assert.Equal(t, len(event.Txs), expectedTxCount) } -func TestForcedInclusionRetriever_processForcedInclusionBlobs(t *testing.T) { +func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_ErrorHandling(t *testing.T) { client := mocks.NewMockClient(t) - client.On("GetForcedInclusionNamespace").Return(datypes.NamespaceFromString("test-fi-ns").Bytes()).Maybe() + fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() + client.On("HasForcedInclusionNamespace").Return(true).Maybe() + client.On("GetForcedInclusionNamespace").Return(fiNs).Maybe() + client.On("Retrieve", mock.Anything, uint64(100), fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{ + Code: datypes.StatusError, + Message: "test error", + }, + }).Once() gen := genesis.Genesis{ DAStartHeight: 100, - DAEpochForcedInclusion: 10, + DAEpochForcedInclusion: 1, // Single height epoch } retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) defer retriever.Stop() + ctx := context.Background() - tests := []struct { - name string - result datypes.ResultRetrieve - height uint64 - expectedTxCount int - expectError bool - }{ - { - name: "success with blobs", - result: datypes.ResultRetrieve{ - BaseResult: datypes.BaseResult{ - Code: datypes.StatusSuccess, - }, - Data: [][]byte{[]byte("tx1"), []byte("tx2")}, - }, - height: 100, - expectedTxCount: 2, - expectError: false, - }, - { - name: "not found", - result: datypes.ResultRetrieve{ - BaseResult: datypes.BaseResult{ - Code: datypes.StatusNotFound, - }, - }, - height: 100, - expectedTxCount: 0, - expectError: false, - }, - { - name: "error status", - result: datypes.ResultRetrieve{ - BaseResult: datypes.BaseResult{ - Code: datypes.StatusError, - Message: "test error", - }, - }, - height: 100, - expectError: true, - }, - { - name: "empty blobs are skipped", - result: datypes.ResultRetrieve{ - BaseResult: datypes.BaseResult{ - Code: datypes.StatusSuccess, - }, - Data: [][]byte{[]byte("tx1"), {}, []byte("tx2")}, - }, - height: 100, - expectedTxCount: 2, - expectError: false, - }, + // Should return empty event with no error (errors are logged and retried later) + event, err := retriever.RetrieveForcedIncludedTxs(ctx, 100) + assert.NilError(t, err) + assert.Assert(t, event != nil) + assert.Equal(t, len(event.Txs), 0) +} + +func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_EmptyBlobsSkipped(t *testing.T) { + client := mocks.NewMockClient(t) + fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() + client.On("HasForcedInclusionNamespace").Return(true).Maybe() + client.On("GetForcedInclusionNamespace").Return(fiNs).Maybe() + client.On("Retrieve", mock.Anything, uint64(100), fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{Code: datypes.StatusSuccess, Timestamp: time.Now()}, + Data: [][]byte{[]byte("tx1"), {}, []byte("tx2"), nil, []byte("tx3")}, + }).Once() + + gen := genesis.Genesis{ + DAStartHeight: 100, + DAEpochForcedInclusion: 1, // Single height epoch + } + + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer retriever.Stop() + ctx := context.Background() + + event, err := retriever.RetrieveForcedIncludedTxs(ctx, 100) + assert.NilError(t, err) + assert.Assert(t, event != nil) + // Should skip empty and nil blobs + assert.Equal(t, len(event.Txs), 3) + assert.DeepEqual(t, event.Txs[0], []byte("tx1")) + assert.DeepEqual(t, event.Txs[1], []byte("tx2")) + assert.DeepEqual(t, event.Txs[2], []byte("tx3")) +} + +func TestForcedInclusionRetriever_RetrieveForcedIncludedTxs_OrderPreserved(t *testing.T) { + // Test that transactions are returned in height order even when fetched out of order + testBlobsByHeight := map[uint64][][]byte{ + 100: {[]byte("tx-100-1"), []byte("tx-100-2")}, + 101: {[]byte("tx-101-1")}, + 102: {[]byte("tx-102-1"), []byte("tx-102-2")}, } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - event := &ForcedInclusionEvent{ - Txs: [][]byte{}, - } - - err := retriever.processRetrieveResult(event, tt.result, tt.height) - - if tt.expectError { - assert.Assert(t, err != nil) - } else { - assert.NilError(t, err) - assert.Equal(t, len(event.Txs), tt.expectedTxCount) - assert.Equal(t, event.Timestamp, time.Time{}) - } - }) + client := mocks.NewMockClient(t) + fiNs := datypes.NamespaceFromString("test-fi-ns").Bytes() + client.On("HasForcedInclusionNamespace").Return(true).Maybe() + client.On("GetForcedInclusionNamespace").Return(fiNs).Maybe() + // Return heights out of order to test ordering is preserved + client.On("Retrieve", mock.Anything, uint64(102), fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{Code: datypes.StatusSuccess, Timestamp: time.Now()}, + Data: testBlobsByHeight[102], + }).Once() + client.On("Retrieve", mock.Anything, uint64(100), fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{Code: datypes.StatusSuccess, Timestamp: time.Now()}, + Data: testBlobsByHeight[100], + }).Once() + client.On("Retrieve", mock.Anything, uint64(101), fiNs).Return(datypes.ResultRetrieve{ + BaseResult: datypes.BaseResult{Code: datypes.StatusSuccess, Timestamp: time.Now()}, + Data: testBlobsByHeight[101], + }).Once() + + gen := genesis.Genesis{ + DAStartHeight: 100, + DAEpochForcedInclusion: 3, // Epoch: 100-102 + } + + retriever := NewForcedInclusionRetriever(client, zerolog.Nop(), config.DefaultConfig(), gen.DAStartHeight, gen.DAEpochForcedInclusion) + defer retriever.Stop() + ctx := context.Background() + + event, err := retriever.RetrieveForcedIncludedTxs(ctx, 102) + assert.NilError(t, err) + assert.Assert(t, event != nil) + + // Verify transactions are in height order: 100, 100, 101, 102, 102 + expectedOrder := [][]byte{ + []byte("tx-100-1"), + []byte("tx-100-2"), + []byte("tx-101-1"), + []byte("tx-102-1"), + []byte("tx-102-2"), + } + assert.Equal(t, len(event.Txs), len(expectedOrder)) + for i, expected := range expectedOrder { + assert.DeepEqual(t, event.Txs[i], expected) } }