Compare commits

...

14 Commits

Author SHA1 Message Date
Manu NALEPA
405340f61c When saving data column sidecars in the storage, send event to the sidecars feed and update the cache before waiting these sidecars are actually saved into the filesystem.
The node will now consider the sidecars as available as soon as there is the INTENT to save them into the storage, and not any more after ensuring the sidecars were actually SAVED into the storage.
2025-12-12 12:12:49 +01:00
Manu NALEPA
4d9fee8560 saveFilesystem: Remove parameters. 2025-12-12 11:44:55 +01:00
Manu NALEPA
597f6c4df0 Add metric for data column prune latency 2025-12-12 10:54:23 +01:00
Manu NALEPA
7a4229a36a Add summary metric for unseen data column indices 2025-12-12 10:46:56 +01:00
Manu NALEPA
1d191987e8 Add stored count metric for SSZ encoded data column sidecars 2025-12-12 10:46:56 +01:00
Manu NALEPA
a577a7f5bd Add metrics for data column file sync latency 2025-12-12 10:46:56 +01:00
Manu NALEPA
f8b717dab4 data_column_storage_save_latency: Refactor buckets. 2025-12-12 10:46:56 +01:00
Manu NALEPA
5a178cd660 Add data_columns_recovered_from_el_attempts metrics 2025-12-12 10:46:56 +01:00
Manu NALEPA
5c24c37116 Add changelog 2025-12-12 10:46:56 +01:00
Manu NALEPA
d4510ef6a8 cellsAndProofsFromStructuredComputationTime: Move summary buckets 2025-12-12 10:46:56 +01:00
Manu NALEPA
bcb95cb9ef Run ComputeCellsAndProofsFromFlat in parallel 2025-12-12 10:46:56 +01:00
Manu NALEPA
c77a2f9b2b Run ComputeCellsAndProofsFromStructured in parallel 2025-12-12 10:46:56 +01:00
Manu NALEPA
819569892f Add cells_and_proofs_from_structured_computation_milliseconds metric 2025-12-12 10:46:56 +01:00
Manu NALEPA
e1f9615130 Add get_blobs_v2_latency_milliseconds metric. 2025-12-12 10:46:56 +01:00
9 changed files with 209 additions and 94 deletions

View File

@@ -5,10 +5,20 @@ import (
"github.com/prometheus/client_golang/prometheus/promauto"
)
var dataColumnComputationTime = promauto.NewHistogram(
prometheus.HistogramOpts{
Name: "beacon_data_column_sidecar_computation_milliseconds",
Help: "Captures the time taken to compute data column sidecars from blobs.",
Buckets: []float64{25, 50, 100, 250, 500, 750, 1000},
},
var (
dataColumnComputationTime = promauto.NewHistogram(
prometheus.HistogramOpts{
Name: "beacon_data_column_sidecar_computation_milliseconds",
Help: "Captures the time taken to compute data column sidecars from blobs.",
Buckets: []float64{25, 50, 100, 250, 500, 750, 1000},
},
)
cellsAndProofsFromStructuredComputationTime = promauto.NewHistogram(
prometheus.HistogramOpts{
Name: "cells_and_proofs_from_structured_computation_milliseconds",
Help: "Captures the time taken to compute cells and proofs from structured computation.",
Buckets: []float64{10, 20, 30, 40, 50, 100, 200},
},
)
)

View File

@@ -3,6 +3,7 @@ package peerdas
import (
"sort"
"sync"
"time"
"github.com/OffchainLabs/prysm/v7/beacon-chain/blockchain/kzg"
fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams"
@@ -296,32 +297,42 @@ func ComputeCellsAndProofsFromFlat(blobs [][]byte, cellProofs [][]byte) ([][]kzg
return nil, nil, ErrBlobsCellsProofsMismatch
}
cellsPerBlob := make([][]kzg.Cell, 0, blobCount)
proofsPerBlob := make([][]kzg.Proof, 0, blobCount)
var wg errgroup.Group
cellsPerBlob := make([][]kzg.Cell, blobCount)
proofsPerBlob := make([][]kzg.Proof, blobCount)
for i, blob := range blobs {
var kzgBlob kzg.Blob
if copy(kzgBlob[:], blob) != len(kzgBlob) {
return nil, nil, errors.New("wrong blob size - should never happen")
}
// Compute the extended cells from the (non-extended) blob.
cells, err := kzg.ComputeCells(&kzgBlob)
if err != nil {
return nil, nil, errors.Wrap(err, "compute cells")
}
var proofs []kzg.Proof
for idx := uint64(i) * numberOfColumns; idx < (uint64(i)+1)*numberOfColumns; idx++ {
var kzgProof kzg.Proof
if copy(kzgProof[:], cellProofs[idx]) != len(kzgProof) {
return nil, nil, errors.New("wrong KZG proof size - should never happen")
wg.Go(func() error {
var kzgBlob kzg.Blob
if copy(kzgBlob[:], blob) != len(kzgBlob) {
return errors.New("wrong blob size - should never happen")
}
proofs = append(proofs, kzgProof)
}
// Compute the extended cells from the (non-extended) blob.
cells, err := kzg.ComputeCells(&kzgBlob)
if err != nil {
return errors.Wrap(err, "compute cells")
}
cellsPerBlob = append(cellsPerBlob, cells)
proofsPerBlob = append(proofsPerBlob, proofs)
proofs := make([]kzg.Proof, 0, numberOfColumns)
for idx := uint64(i) * numberOfColumns; idx < (uint64(i)+1)*numberOfColumns; idx++ {
var kzgProof kzg.Proof
if copy(kzgProof[:], cellProofs[idx]) != len(kzgProof) {
return errors.New("wrong KZG proof size - should never happen")
}
proofs = append(proofs, kzgProof)
}
cellsPerBlob[i] = cells
proofsPerBlob[i] = proofs
return nil
})
}
if err := wg.Wait(); err != nil {
return nil, nil, err
}
return cellsPerBlob, proofsPerBlob, nil
@@ -329,40 +340,55 @@ func ComputeCellsAndProofsFromFlat(blobs [][]byte, cellProofs [][]byte) ([][]kzg
// ComputeCellsAndProofsFromStructured computes the cells and proofs from blobs and cell proofs.
func ComputeCellsAndProofsFromStructured(blobsAndProofs []*pb.BlobAndProofV2) ([][]kzg.Cell, [][]kzg.Proof, error) {
cellsPerBlob := make([][]kzg.Cell, 0, len(blobsAndProofs))
proofsPerBlob := make([][]kzg.Proof, 0, len(blobsAndProofs))
for _, blobAndProof := range blobsAndProofs {
start := time.Now()
defer func() {
cellsAndProofsFromStructuredComputationTime.Observe(float64(time.Since(start).Milliseconds()))
}()
var wg errgroup.Group
cellsPerBlob := make([][]kzg.Cell, len(blobsAndProofs))
proofsPerBlob := make([][]kzg.Proof, len(blobsAndProofs))
for i, blobAndProof := range blobsAndProofs {
if blobAndProof == nil {
return nil, nil, ErrNilBlobAndProof
}
var kzgBlob kzg.Blob
if copy(kzgBlob[:], blobAndProof.Blob) != len(kzgBlob) {
return nil, nil, errors.New("wrong blob size - should never happen")
}
// Compute the extended cells from the (non-extended) blob.
cells, err := kzg.ComputeCells(&kzgBlob)
if err != nil {
return nil, nil, errors.Wrap(err, "compute cells")
}
kzgProofs := make([]kzg.Proof, 0, fieldparams.NumberOfColumns)
for _, kzgProofBytes := range blobAndProof.KzgProofs {
if len(kzgProofBytes) != kzg.BytesPerProof {
return nil, nil, errors.New("wrong KZG proof size - should never happen")
wg.Go(func() error {
var kzgBlob kzg.Blob
if copy(kzgBlob[:], blobAndProof.Blob) != len(kzgBlob) {
return errors.New("wrong blob size - should never happen")
}
var kzgProof kzg.Proof
if copy(kzgProof[:], kzgProofBytes) != len(kzgProof) {
return nil, nil, errors.New("wrong copied KZG proof size - should never happen")
// Compute the extended cells from the (non-extended) blob.
cells, err := kzg.ComputeCells(&kzgBlob)
if err != nil {
return errors.Wrap(err, "compute cells")
}
kzgProofs = append(kzgProofs, kzgProof)
}
kzgProofs := make([]kzg.Proof, 0, fieldparams.NumberOfColumns)
for _, kzgProofBytes := range blobAndProof.KzgProofs {
if len(kzgProofBytes) != kzg.BytesPerProof {
return errors.New("wrong KZG proof size - should never happen")
}
cellsPerBlob = append(cellsPerBlob, cells)
proofsPerBlob = append(proofsPerBlob, kzgProofs)
var kzgProof kzg.Proof
if copy(kzgProof[:], kzgProofBytes) != len(kzgProof) {
return errors.New("wrong copied KZG proof size - should never happen")
}
kzgProofs = append(kzgProofs, kzgProof)
}
cellsPerBlob[i] = cells
proofsPerBlob[i] = kzgProofs
return nil
})
}
if err := wg.Wait(); err != nil {
return nil, nil, err
}
return cellsPerBlob, proofsPerBlob, nil

View File

@@ -288,7 +288,7 @@ func (dcs *DataColumnStorage) Save(dataColumnSidecars []blocks.VerifiedRODataCol
dataColumnSidecarsByRoot[root] = append(dataColumnSidecarsByRoot[root], dataColumnSidecar)
}
for root, dataColumnSidecars := range dataColumnSidecarsByRoot {
for _, dataColumnSidecars := range dataColumnSidecarsByRoot {
// Safety check all data column sidecars for this root are from the same slot.
slot := dataColumnSidecars[0].Slot()
for _, dataColumnSidecar := range dataColumnSidecars[1:] {
@@ -298,27 +298,9 @@ func (dcs *DataColumnStorage) Save(dataColumnSidecars []blocks.VerifiedRODataCol
}
// Save data columns in the filesystem.
epoch := slots.ToEpoch(slot)
if err := dcs.saveFilesystem(root, epoch, dataColumnSidecars); err != nil {
if err := dcs.saveFilesystem(dataColumnSidecars); err != nil {
return errors.Wrap(err, "save filesystem")
}
// Get all indices.
indices := make([]uint64, 0, len(dataColumnSidecars))
for _, dataColumnSidecar := range dataColumnSidecars {
indices = append(indices, dataColumnSidecar.Index)
}
// Compute the data columns ident.
dataColumnsIdent := DataColumnsIdent{Root: root, Epoch: epoch, Indices: indices}
// Set data columns in the cache.
if err := dcs.cache.set(dataColumnsIdent); err != nil {
return errors.Wrap(err, "cache set")
}
// Notify the data column feed.
dcs.dataColumnFeed.Send(dataColumnsIdent)
}
dataColumnSaveLatency.Observe(float64(time.Since(startTime).Milliseconds()))
@@ -344,8 +326,12 @@ func (dcs *DataColumnStorage) Subscribe() (event.Subscription, <-chan DataColumn
}
// saveFilesystem saves data column sidecars into the database.
// This function expects all data column sidecars to belong to the same block.
func (dcs *DataColumnStorage) saveFilesystem(root [fieldparams.RootLength]byte, epoch primitives.Epoch, dataColumnSidecars []blocks.VerifiedRODataColumn) error {
// This function expects all data column sidecars to belong to the same block, and assumes `dataColumnSidecars` is not empty.
func (dcs *DataColumnStorage) saveFilesystem(dataColumnSidecars []blocks.VerifiedRODataColumn) error {
root := dataColumnSidecars[0].BlockRoot()
slot := dataColumnSidecars[0].SignedBlockHeader.Header.Slot
epoch := slots.ToEpoch(slot)
// Compute the file path.
filePath := filePath(root, epoch)
@@ -358,6 +344,22 @@ func (dcs *DataColumnStorage) saveFilesystem(root [fieldparams.RootLength]byte,
fileMu.Lock()
defer fileMu.Unlock()
// Get all indices.
indices := make([]uint64, 0, len(dataColumnSidecars))
for _, dataColumnSidecar := range dataColumnSidecars {
indices = append(indices, dataColumnSidecar.Index)
}
dataColumnsIdent := DataColumnsIdent{Root: root, Epoch: epoch, Indices: indices}
// Set data columns in the cache.
if err := dcs.cache.set(dataColumnsIdent); err != nil {
return errors.Wrap(err, "cache set")
}
// Notify the data column feed.
dcs.dataColumnFeed.Send(dataColumnsIdent)
// Check if the file exists.
exists, err := afero.Exists(dcs.fs, filePath)
if err != nil {
@@ -515,6 +517,11 @@ func (dcs *DataColumnStorage) Clear() error {
// prune clean the cache, the filesystem and mutexes.
func (dcs *DataColumnStorage) prune() {
startTime := time.Now()
defer func() {
dataColumnPruneLatency.Observe(float64(time.Since(startTime).Milliseconds()))
}()
highestStoredEpoch := dcs.cache.HighestEpoch()
// Check if we need to prune.
@@ -622,6 +629,9 @@ func (dcs *DataColumnStorage) saveDataColumnSidecarsExistingFile(filePath string
// Create the SSZ encoded data column sidecars.
var sszEncodedDataColumnSidecars []byte
// Initialize the count of the saved SSZ encoded data column sidecar.
storedCount := uint8(0)
for {
dataColumnSidecars := pullChan(inputDataColumnSidecars)
if len(dataColumnSidecars) == 0 {
@@ -668,6 +678,9 @@ func (dcs *DataColumnStorage) saveDataColumnSidecarsExistingFile(filePath string
return errors.Wrap(err, "set index")
}
// Increment the count of the saved SSZ encoded data column sidecar.
storedCount++
// Append the SSZ encoded data column sidecar to the SSZ encoded data column sidecars.
sszEncodedDataColumnSidecars = append(sszEncodedDataColumnSidecars, sszEncodedDataColumnSidecar...)
}
@@ -692,9 +705,12 @@ func (dcs *DataColumnStorage) saveDataColumnSidecarsExistingFile(filePath string
return errWrongBytesWritten
}
syncStart := time.Now()
if err := file.Sync(); err != nil {
return errors.Wrap(err, "sync")
}
dataColumnFileSyncLatency.Observe(float64(time.Since(syncStart).Milliseconds()))
dataColumnBatchStoreCount.Observe(float64(storedCount))
return nil
}
@@ -808,10 +824,14 @@ func (dcs *DataColumnStorage) saveDataColumnSidecarsNewFile(filePath string, inp
return errWrongBytesWritten
}
syncStart := time.Now()
if err := file.Sync(); err != nil {
return errors.Wrap(err, "sync")
}
dataColumnFileSyncLatency.Observe(float64(time.Since(syncStart).Milliseconds()))
dataColumnBatchStoreCount.Observe(float64(storedCount))
return nil
}

View File

@@ -36,16 +36,15 @@ var (
})
// Data columns
dataColumnBuckets = []float64{3, 5, 7, 9, 11, 13}
dataColumnSaveLatency = promauto.NewHistogram(prometheus.HistogramOpts{
Name: "data_column_storage_save_latency",
Help: "Latency of DataColumnSidecar storage save operations in milliseconds",
Buckets: dataColumnBuckets,
Buckets: []float64{10, 20, 30, 50, 100, 200, 500},
})
dataColumnFetchLatency = promauto.NewHistogram(prometheus.HistogramOpts{
Name: "data_column_storage_get_latency",
Help: "Latency of DataColumnSidecar storage get operations in milliseconds",
Buckets: dataColumnBuckets,
Buckets: []float64{3, 5, 7, 9, 11, 13},
})
dataColumnPrunedCounter = promauto.NewCounter(prometheus.CounterOpts{
Name: "data_column_pruned",
@@ -59,4 +58,16 @@ var (
Name: "data_column_disk_count",
Help: "Approximate number of data columns in storage",
})
dataColumnFileSyncLatency = promauto.NewSummary(prometheus.SummaryOpts{
Name: "data_column_file_sync_latency",
Help: "Latency of sync operations when saving data columns in milliseconds",
})
dataColumnBatchStoreCount = promauto.NewSummary(prometheus.SummaryOpts{
Name: "data_column_batch_store_count",
Help: "Number of data columns stored in a batch",
})
dataColumnPruneLatency = promauto.NewSummary(prometheus.SummaryOpts{
Name: "data_column_prune_latency",
Help: "Latency of data column prune operations in milliseconds",
})
)

View File

@@ -532,12 +532,19 @@ func (s *Service) GetBlobsV2(ctx context.Context, versionedHashes []common.Hash)
ctx, span := trace.StartSpan(ctx, "powchain.engine-api-client.GetBlobsV2")
defer span.End()
start := time.Now()
if !s.capabilityCache.has(GetBlobsV2) {
return nil, errors.New(fmt.Sprintf("%s is not supported", GetBlobsV2))
}
result := make([]*pb.BlobAndProofV2, len(versionedHashes))
err := s.rpcClient.CallContext(ctx, &result, GetBlobsV2, versionedHashes)
if len(result) != 0 {
getBlobsV2Latency.Observe(float64(time.Since(start).Milliseconds()))
}
return result, handleRPCError(err)
}

View File

@@ -27,6 +27,13 @@ var (
Buckets: []float64{25, 50, 100, 200, 500, 1000, 2000, 4000},
},
)
getBlobsV2Latency = promauto.NewHistogram(
prometheus.HistogramOpts{
Name: "get_blobs_v2_latency_milliseconds",
Help: "Captures RPC latency for getBlobsV2 in milliseconds",
Buckets: []float64{25, 50, 100, 200, 500, 1000, 2000, 4000},
},
)
errParseCount = promauto.NewCounter(prometheus.CounterOpts{
Name: "execution_parse_error_count",
Help: "The number of errors that occurred while parsing execution payload",

View File

@@ -204,6 +204,13 @@ var (
},
)
dataColumnsRecoveredFromELAttempts = promauto.NewCounter(
prometheus.CounterOpts{
Name: "data_columns_recovered_from_el_attempts",
Help: "Count the number of data columns recovery attempts from the execution layer.",
},
)
dataColumnsRecoveredFromELTotal = promauto.NewCounter(
prometheus.CounterOpts{
Name: "data_columns_recovered_from_el_total",
@@ -242,6 +249,13 @@ var (
Buckets: []float64{100, 250, 500, 750, 1000, 1500, 2000, 4000, 8000, 12000, 16000},
},
)
dataColumnSidecarsObtainedViaELCount = promauto.NewSummary(
prometheus.SummaryOpts{
Name: "data_column_obtained_via_el_count",
Help: "Count the number of data column sidecars obtained via the execution layer.",
},
)
)
func (s *Service) updateMetrics() {

View File

@@ -189,12 +189,30 @@ func (s *Service) processDataColumnSidecarsFromExecution(ctx context.Context, so
ctx, cancel := context.WithTimeout(ctx, secondsPerHalfSlot)
defer cancel()
log := log.WithFields(logrus.Fields{
"root": fmt.Sprintf("%#x", source.Root()),
"slot": source.Slot(),
"proposerIndex": source.ProposerIndex(),
"type": source.Type(),
})
var constructedSidecarCount uint64
for iteration := uint64(0); ; /*no stop condition*/ iteration++ {
log = log.WithField("iteration", iteration)
// Exit early if all sidecars to sample have been seen.
if s.haveAllSidecarsBeenSeen(source.Slot(), source.ProposerIndex(), columnIndicesToSample) {
if iteration > 0 && constructedSidecarCount == 0 {
log.Debug("No data column sidecars constructed from the execution client")
}
return nil, nil
}
if iteration == 0 {
dataColumnsRecoveredFromELAttempts.Inc()
}
// Try to reconstruct data column constructedSidecars from the execution client.
constructedSidecars, err := s.cfg.executionReconstructor.ConstructDataColumnSidecars(ctx, source)
if err != nil {
@@ -202,8 +220,8 @@ func (s *Service) processDataColumnSidecarsFromExecution(ctx context.Context, so
}
// No sidecars are retrieved from the EL, retry later
sidecarCount := uint64(len(constructedSidecars))
if sidecarCount == 0 {
constructedSidecarCount = uint64(len(constructedSidecars))
if constructedSidecarCount == 0 {
if ctx.Err() != nil {
return nil, ctx.Err()
}
@@ -212,9 +230,11 @@ func (s *Service) processDataColumnSidecarsFromExecution(ctx context.Context, so
continue
}
dataColumnsRecoveredFromELTotal.Inc()
// Boundary check.
if sidecarCount != fieldparams.NumberOfColumns {
return nil, errors.Errorf("reconstruct data column sidecars returned %d sidecars, expected %d - should never happen", sidecarCount, fieldparams.NumberOfColumns)
if constructedSidecarCount != fieldparams.NumberOfColumns {
return nil, errors.Errorf("reconstruct data column sidecars returned %d sidecars, expected %d - should never happen", constructedSidecarCount, fieldparams.NumberOfColumns)
}
unseenIndices, err := s.broadcastAndReceiveUnseenDataColumnSidecars(ctx, source.Slot(), source.ProposerIndex(), columnIndicesToSample, constructedSidecars)
@@ -222,19 +242,12 @@ func (s *Service) processDataColumnSidecarsFromExecution(ctx context.Context, so
return nil, errors.Wrap(err, "broadcast and receive unseen data column sidecars")
}
if len(unseenIndices) > 0 {
dataColumnsRecoveredFromELTotal.Inc()
log.WithFields(logrus.Fields{
"count": len(unseenIndices),
"indices": helpers.SortedPrettySliceFromMap(unseenIndices),
}).Debug("Constructed data column sidecars from the execution client")
log.WithFields(logrus.Fields{
"root": fmt.Sprintf("%#x", source.Root()),
"slot": source.Slot(),
"proposerIndex": source.ProposerIndex(),
"iteration": iteration,
"type": source.Type(),
"count": len(unseenIndices),
"indices": helpers.SortedPrettySliceFromMap(unseenIndices),
}).Debug("Constructed data column sidecars from the execution client")
}
dataColumnSidecarsObtainedViaELCount.Observe(float64(len(unseenIndices)))
return nil, nil
}

View File

@@ -0,0 +1,7 @@
### Added
- prometheus histogram `cells_and_proofs_from_structured_computation_milliseconds` to track computation time for cells and proofs from structured blobs.
- prometheus histogram `get_blobs_v2_latency_milliseconds` to track RPC latency for `getBlobsV2` calls to the execution layer.
### Changed
- Run `ComputeCellsAndProofsFromFlat` in parallel to improve performance when computing cells and proofs.
- Run `ComputeCellsAndProofsFromStructured` in parallel to improve performance when computing cells and proofs.