From 362a390c5e69bce21d300075f7b9a049ac37788d Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Tue, 26 Nov 2024 15:28:47 -0600 Subject: [PATCH 01/30] Add metrics to relay. Signed-off-by: Cody Littley --- common/metrics/metrics.go | 8 ++++- common/metrics/metrics_server.go | 4 +++ common/metrics/mock_metrics.go | 5 ++++ relay/metrics/relay_metrics.go | 51 ++++++++++++++++++++++++++++++++ relay/server.go | 16 ++++++++-- 5 files changed, 80 insertions(+), 4 deletions(-) create mode 100644 relay/metrics/relay_metrics.go diff --git a/common/metrics/metrics.go b/common/metrics/metrics.go index ed63a1c6a..4fbf5d054 100644 --- a/common/metrics/metrics.go +++ b/common/metrics/metrics.go @@ -1,6 +1,9 @@ package metrics -import "time" +import ( + "github.com/prometheus/client_golang/prometheus" + "time" +) // Metrics provides a convenient interface for reporting metrics. type Metrics interface { @@ -62,6 +65,9 @@ type Metrics interface { pollPeriod time.Duration, source func() float64, label ...any) error + + // RegisterExternalMetrics registers prometheus collectors created outside the metrics framework. + RegisterExternalMetrics(collectors ...prometheus.Collector) } // Metric represents a metric that can be reported. diff --git a/common/metrics/metrics_server.go b/common/metrics/metrics_server.go index f18fd02ac..34362aace 100644 --- a/common/metrics/metrics_server.go +++ b/common/metrics/metrics_server.go @@ -428,3 +428,7 @@ func (m *metrics) WriteMetricsDocumentation(fileName string) error { return nil } + +func (m *metrics) RegisterExternalMetrics(collectors ...prometheus.Collector) { + m.registry.MustRegister(collectors...) +} diff --git a/common/metrics/mock_metrics.go b/common/metrics/mock_metrics.go index 695a2662e..244f268c9 100644 --- a/common/metrics/mock_metrics.go +++ b/common/metrics/mock_metrics.go @@ -1,6 +1,7 @@ package metrics import ( + "github.com/prometheus/client_golang/prometheus" "time" ) @@ -62,6 +63,10 @@ func (m *mockMetrics) NewAutoGauge( return nil } +func (m *mockMetrics) RegisterExternalMetrics(collectors ...prometheus.Collector) { + +} + var _ CountMetric = &mockCountMetric{} type mockCountMetric struct { diff --git a/relay/metrics/relay_metrics.go b/relay/metrics/relay_metrics.go new file mode 100644 index 000000000..1b62422a6 --- /dev/null +++ b/relay/metrics/relay_metrics.go @@ -0,0 +1,51 @@ +package metrics + +import ( + "github.com/Layr-Labs/eigenda/common/metrics" + "github.com/Layr-Labs/eigensdk-go/logging" + grpcprom "github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus" + "google.golang.org/grpc" +) + +type RelayMetrics struct { + metricsServer metrics.Metrics + grpcServerOption grpc.ServerOption +} + +func NewRelayMetrics( + logger logging.Logger, + config *metrics.Config) (*RelayMetrics, error) { + + server := metrics.NewMetrics(logger, config) + + grpcMetrics := grpcprom.NewServerMetrics() + server.RegisterExternalMetrics(grpcMetrics) + grpcServerOption := grpc.UnaryInterceptor( + grpcMetrics.UnaryServerInterceptor(), + ) + + return &RelayMetrics{ + metricsServer: server, + grpcServerOption: grpcServerOption, + }, nil +} + +// Start starts the metrics server. +func (m *RelayMetrics) Start() error { + return m.metricsServer.Start() +} + +// Stop stops the metrics server. +func (m *RelayMetrics) Stop() error { + return m.metricsServer.Stop() +} + +// WriteMetricsDocumentation writes documentation for all currently registered metrics to a file. +func (m *RelayMetrics) WriteMetricsDocumentation() error { + return m.metricsServer.WriteMetricsDocumentation("relay/metrics/relay-metrics.md") +} + +// GetGRPCServerOption returns the gRPC server option that enables automatic GRPC metrics collection. +func (m *RelayMetrics) GetGRPCServerOption() grpc.ServerOption { + return m.grpcServerOption +} diff --git a/relay/server.go b/relay/server.go index 540b46b0b..ac7efbc92 100644 --- a/relay/server.go +++ b/relay/server.go @@ -4,6 +4,7 @@ import ( "context" "errors" "fmt" + "github.com/Layr-Labs/eigenda/relay/metrics" "net" "time" @@ -57,6 +58,9 @@ type Server struct { // chainReader is the core.Reader used to fetch blob parameters. chainReader core.Reader + + // metrics encapsulates the metrics for the relay server. + relayMetrics *metrics.RelayMetrics } type Config struct { @@ -124,8 +128,8 @@ func NewServer( blobStore *blobstore.BlobStore, chunkReader chunkstore.ChunkReader, chainReader core.Reader, - ics core.IndexedChainState, -) (*Server, error) { + ics core.IndexedChainState) (*Server, error) { + if chainReader == nil { return nil, errors.New("chainReader is required") } @@ -184,6 +188,11 @@ func NewServer( } } + relayMetrics, err := metrics.NewRelayMetrics(logger, nil) // TODO config + if err != nil { + return nil, fmt.Errorf("error creating metrics: %w", err) + } + return &Server{ config: config, logger: logger, @@ -193,6 +202,7 @@ func NewServer( blobRateLimiter: limiter.NewBlobRateLimiter(&config.RateLimits), chunkRateLimiter: limiter.NewChunkRateLimiter(&config.RateLimits), authenticator: authenticator, + relayMetrics: relayMetrics, }, nil } @@ -445,7 +455,7 @@ func (s *Server) Start(ctx context.Context) error { opt := grpc.MaxRecvMsgSize(s.config.MaxGRPCMessageSize) - s.grpcServer = grpc.NewServer(opt) + s.grpcServer = grpc.NewServer(opt, s.relayMetrics.GetGRPCServerOption()) reflection.Register(s.grpcServer) pb.RegisterRelayServer(s.grpcServer, s) From 3b4637e12903ac89d0e854218e520338cb62d3ce Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Wed, 27 Nov 2024 08:02:45 -0600 Subject: [PATCH 02/30] Incremental progress. Signed-off-by: Cody Littley --- common/metrics/config.go | 10 --------- common/metrics/metrics_server.go | 22 ++++++++++---------- common/metrics/test/main.go | 8 +------- relay/cmd/config.go | 1 + relay/cmd/flags/flags.go | 8 ++++++++ relay/metrics/relay_metrics.go | 12 +++-------- relay/server.go | 19 +++++++++++++++-- relay/server_test.go | 35 +++++++++++++++++++++++++------- 8 files changed, 69 insertions(+), 46 deletions(-) delete mode 100644 common/metrics/config.go diff --git a/common/metrics/config.go b/common/metrics/config.go deleted file mode 100644 index 1bc8c0e4d..000000000 --- a/common/metrics/config.go +++ /dev/null @@ -1,10 +0,0 @@ -package metrics - -// Config provides configuration for a Metrics instance. -type Config struct { - // Namespace is the namespace for the metrics. - Namespace string - - // HTTPPort is the port to serve metrics on. - HTTPPort int -} diff --git a/common/metrics/metrics_server.go b/common/metrics/metrics_server.go index 34362aace..7a5868dd0 100644 --- a/common/metrics/metrics_server.go +++ b/common/metrics/metrics_server.go @@ -24,8 +24,8 @@ type metrics struct { // logger is the logger used to log messages. logger logging.Logger - // config is the configuration for the metrics. - config *Config + // namespace is prepended to all metric names. + namespace string // registry is the prometheus registry used to report metrics. registry *prometheus.Registry @@ -55,13 +55,13 @@ type metrics struct { } // NewMetrics creates a new Metrics instance. -func NewMetrics(logger logging.Logger, config *Config) Metrics { +func NewMetrics(logger logging.Logger, namespace string, port int) Metrics { reg := prometheus.NewRegistry() reg.MustRegister(collectors.NewProcessCollector(collectors.ProcessCollectorOpts{})) reg.MustRegister(collectors.NewGoCollector()) - logger.Infof("Starting metrics server at port %d", config.HTTPPort) - addr := fmt.Sprintf(":%d", config.HTTPPort) + logger.Infof("Starting metrics server at port %d", port) + addr := fmt.Sprintf(":%d", port) mux := http.NewServeMux() mux.Handle("/metrics", promhttp.HandlerFor( reg, @@ -74,7 +74,7 @@ func NewMetrics(logger logging.Logger, config *Config) Metrics { m := &metrics{ logger: logger, - config: config, + namespace: namespace, registry: reg, metricMap: make(map[metricID]Metric), isAlive: atomic.Bool{}, @@ -198,7 +198,7 @@ func (m *metrics) NewLatencyMetric( metric, err := newLatencyMetric( m.logger, m.registry, - m.config.Namespace, + m.namespace, name, description, objectives, @@ -238,7 +238,7 @@ func (m *metrics) NewCountMetric( metric, err := newCountMetric( m.logger, m.registry, - m.config.Namespace, + m.namespace, name, description, labelTemplate) @@ -287,7 +287,7 @@ func (m *metrics) newGaugeMetricUnsafe( metric, err := newGaugeMetric( m.logger, m.registry, - m.config.Namespace, + m.namespace, name, unit, description, @@ -368,7 +368,7 @@ func (m *metrics) GenerateMetricsDocumentation() string { } slices.SortFunc(metricIDs, sortFunc) - sb.Write([]byte(fmt.Sprintf("# Metrics Documentation for namespace '%s'\n\n", m.config.Namespace))) + sb.Write([]byte(fmt.Sprintf("# Metrics Documentation for namespace '%s'\n\n", m.namespace))) sb.Write([]byte(fmt.Sprintf("This documentation was automatically generated at time `%s`\n\n", time.Now().Format(time.RFC3339)))) @@ -402,7 +402,7 @@ func (m *metrics) GenerateMetricsDocumentation() string { sb.Write([]byte(fmt.Sprintf("| **Quantiles** | %s |\n", m.quantilesMap[*id]))) } sb.Write([]byte(fmt.Sprintf("| **Fully Qualified Name** | `%s_%s_%s` |\n", - m.config.Namespace, id.name, id.unit))) + m.namespace, id.name, id.unit))) } return sb.String() diff --git a/common/metrics/test/main.go b/common/metrics/test/main.go index bdd1050b2..430bfc75b 100644 --- a/common/metrics/test/main.go +++ b/common/metrics/test/main.go @@ -24,18 +24,12 @@ type LabelType2 struct { } func main() { - - metricsConfig := &metrics.Config{ - Namespace: "test", - HTTPPort: 9101, - } - logger, err := common.NewLogger(common.DefaultLoggerConfig()) if err != nil { panic(err) } - metricsServer := metrics.NewMetrics(logger, metricsConfig) + metricsServer := metrics.NewMetrics(logger, "test", 9101) l1, err := metricsServer.NewLatencyMetric( "l1", diff --git a/relay/cmd/config.go b/relay/cmd/config.go index 154c4c2bd..0eb9e22cc 100644 --- a/relay/cmd/config.go +++ b/relay/cmd/config.go @@ -93,6 +93,7 @@ func NewConfig(ctx *cli.Context) (Config, error) { InternalGetProofsTimeout: ctx.Duration(flags.InternalGetProofsTimeoutFlag.Name), InternalGetCoefficientsTimeout: ctx.Duration(flags.InternalGetCoefficientsTimeoutFlag.Name), }, + MetricsPort: ctx.Int(flags.MetricsPortFlag.Name), }, EthClientConfig: geth.ReadEthClientConfig(ctx), BLSOperatorStateRetrieverAddr: ctx.String(flags.BlsOperatorStateRetrieverAddrFlag.Name), diff --git a/relay/cmd/flags/flags.go b/relay/cmd/flags/flags.go index baed1fbcf..927ef13b7 100644 --- a/relay/cmd/flags/flags.go +++ b/relay/cmd/flags/flags.go @@ -280,6 +280,13 @@ var ( EnvVar: common.PrefixEnvVar(envVarPrefix, "ONCHAIN_STATE_REFRESH_INTERVAL"), Value: 1 * time.Hour, } + MetricsPortFlag = cli.IntFlag{ + Name: common.PrefixFlag(FlagPrefix, "metrics-port"), + Usage: "Port to listen on for metrics", + Required: false, + EnvVar: common.PrefixEnvVar(envVarPrefix, "METRICS_PORT"), + Value: 9101, + } ) var requiredFlags = []cli.Flag{ @@ -327,6 +334,7 @@ var optionalFlags = []cli.Flag{ InternalGetProofsTimeoutFlag, InternalGetCoefficientsTimeoutFlag, OnchainStateRefreshIntervalFlag, + MetricsPortFlag, } var Flags []cli.Flag diff --git a/relay/metrics/relay_metrics.go b/relay/metrics/relay_metrics.go index 1b62422a6..d7b8ffddd 100644 --- a/relay/metrics/relay_metrics.go +++ b/relay/metrics/relay_metrics.go @@ -12,11 +12,10 @@ type RelayMetrics struct { grpcServerOption grpc.ServerOption } -func NewRelayMetrics( - logger logging.Logger, - config *metrics.Config) (*RelayMetrics, error) { +// NewRelayMetrics creates a new RelayMetrics instance, which encapsulates all metrics related to the relay. +func NewRelayMetrics(logger logging.Logger, port int) (*RelayMetrics, error) { - server := metrics.NewMetrics(logger, config) + server := metrics.NewMetrics(logger, "relay", port) grpcMetrics := grpcprom.NewServerMetrics() server.RegisterExternalMetrics(grpcMetrics) @@ -40,11 +39,6 @@ func (m *RelayMetrics) Stop() error { return m.metricsServer.Stop() } -// WriteMetricsDocumentation writes documentation for all currently registered metrics to a file. -func (m *RelayMetrics) WriteMetricsDocumentation() error { - return m.metricsServer.WriteMetricsDocumentation("relay/metrics/relay-metrics.md") -} - // GetGRPCServerOption returns the gRPC server option that enables automatic GRPC metrics collection. func (m *RelayMetrics) GetGRPCServerOption() grpc.ServerOption { return m.grpcServerOption diff --git a/relay/server.go b/relay/server.go index ac7efbc92..63f608ff9 100644 --- a/relay/server.go +++ b/relay/server.go @@ -117,6 +117,9 @@ type Config struct { // OnchainStateRefreshInterval is the interval at which the onchain state is refreshed. OnchainStateRefreshInterval time.Duration + + // MetricsPort is the port that the relay metrics server listens on. + MetricsPort int } // NewServer creates a new relay Server. @@ -188,7 +191,7 @@ func NewServer( } } - relayMetrics, err := metrics.NewRelayMetrics(logger, nil) // TODO config + relayMetrics, err := metrics.NewRelayMetrics(logger, config.MetricsPort) if err != nil { return nil, fmt.Errorf("error creating metrics: %w", err) } @@ -469,6 +472,11 @@ func (s *Server) Start(ctx context.Context) error { return errors.New("could not start GRPC server") } + err = s.relayMetrics.Start() + if err != nil { + return fmt.Errorf("error starting metrics server: %w", err) + } + return nil } @@ -493,8 +501,15 @@ func (s *Server) RefreshOnchainState(ctx context.Context) error { } // Stop stops the server. -func (s *Server) Stop() { +func (s *Server) Stop() error { if s.grpcServer != nil { s.grpcServer.Stop() } + + err := s.relayMetrics.Stop() + if err != nil { + return fmt.Errorf("error stopping metrics server: %w", err) + } + + return nil } diff --git a/relay/server_test.go b/relay/server_test.go index 3e16c624c..3a7e31977 100644 --- a/relay/server_test.go +++ b/relay/server_test.go @@ -122,7 +122,10 @@ func TestReadWriteBlobs(t *testing.T) { err = server.Start(context.Background()) require.NoError(t, err) }() - defer server.Stop() + defer func() { + err = server.Stop() + require.NoError(t, err) + }() expectedData := make(map[v2.BlobKey][]byte) @@ -202,7 +205,10 @@ func TestReadNonExistentBlob(t *testing.T) { err = server.Start(context.Background()) require.NoError(t, err) }() - defer server.Stop() + defer func() { + err = server.Stop() + require.NoError(t, err) + }() for i := 0; i < 10; i++ { request := &pb.GetBlobRequest{ @@ -257,7 +263,10 @@ func TestReadWriteBlobsWithSharding(t *testing.T) { err = server.Start(context.Background()) require.NoError(t, err) }() - defer server.Stop() + defer func() { + err = server.Stop() + require.NoError(t, err) + }() expectedData := make(map[v2.BlobKey][]byte) shardMap := make(map[v2.BlobKey][]v2.RelayKey) @@ -377,7 +386,10 @@ func TestReadWriteChunks(t *testing.T) { err = server.Start(context.Background()) require.NoError(t, err) }() - defer server.Stop() + defer func() { + err = server.Stop() + require.NoError(t, err) + }() expectedData := make(map[v2.BlobKey][]*encoding.Frame) fragmentInfoMap := make(map[v2.BlobKey]*encoding.FragmentInfo) @@ -575,7 +587,10 @@ func TestBatchedReadWriteChunks(t *testing.T) { err = server.Start(context.Background()) require.NoError(t, err) }() - defer server.Stop() + defer func() { + err = server.Stop() + require.NoError(t, err) + }() expectedData := make(map[v2.BlobKey][]*encoding.Frame) fragmentInfoMap := make(map[v2.BlobKey]*encoding.FragmentInfo) @@ -703,7 +718,10 @@ func TestReadWriteChunksWithSharding(t *testing.T) { err = server.Start(context.Background()) require.NoError(t, err) }() - defer server.Stop() + defer func() { + err = server.Stop() + require.NoError(t, err) + }() expectedData := make(map[v2.BlobKey][]*encoding.Frame) fragmentInfoMap := make(map[v2.BlobKey]*encoding.FragmentInfo) @@ -980,7 +998,10 @@ func TestBatchedReadWriteChunksWithSharding(t *testing.T) { err = server.Start(context.Background()) require.NoError(t, err) }() - defer server.Stop() + defer func() { + err = server.Stop() + require.NoError(t, err) + }() expectedData := make(map[v2.BlobKey][]*encoding.Frame) fragmentInfoMap := make(map[v2.BlobKey]*encoding.FragmentInfo) From 60f015eb400a89ed67fc43310c86b740b84052d6 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Wed, 27 Nov 2024 08:44:35 -0600 Subject: [PATCH 03/30] Incremental progress. Signed-off-by: Cody Littley --- relay/limiter/chunk_rate_limiter.go | 31 +++++++++- relay/limiter/chunk_rate_limiter_test.go | 12 ++-- relay/metrics/relay_metrics.go | 76 +++++++++++++++++++++++- relay/server.go | 29 ++++++--- relay/server_test.go | 5 +- 5 files changed, 135 insertions(+), 18 deletions(-) diff --git a/relay/limiter/chunk_rate_limiter.go b/relay/limiter/chunk_rate_limiter.go index af71b317b..c5f244574 100644 --- a/relay/limiter/chunk_rate_limiter.go +++ b/relay/limiter/chunk_rate_limiter.go @@ -2,6 +2,7 @@ package limiter import ( "fmt" + "github.com/Layr-Labs/eigenda/common/metrics" "golang.org/x/time/rate" "sync" "time" @@ -36,12 +37,17 @@ type ChunkRateLimiter struct { // perClientOperationsInFlight is the number of GetChunk operations currently in flight for each client. perClientOperationsInFlight map[string]int + // limiterCounter is used to track rate limiting events, ignored if nil + limiterCounter metrics.CountMetric + // this lock is used to provide thread safety lock sync.Mutex } // NewChunkRateLimiter creates a new ChunkRateLimiter. -func NewChunkRateLimiter(config *Config) *ChunkRateLimiter { +func NewChunkRateLimiter( + config *Config, + limiterCounter metrics.CountMetric) *ChunkRateLimiter { globalOpLimiter := rate.NewLimiter(rate.Limit( config.MaxGetChunkOpsPerSecond), @@ -58,9 +64,14 @@ func NewChunkRateLimiter(config *Config) *ChunkRateLimiter { perClientOpLimiter: make(map[string]*rate.Limiter), perClientBandwidthLimiter: make(map[string]*rate.Limiter), perClientOperationsInFlight: make(map[string]int), + limiterCounter: limiterCounter, } } +type RateLimitLabel struct { + reason string +} + // BeginGetChunkOperation should be called when a GetChunk operation is about to begin. If it returns an error, // the operation should not be performed. If it does not return an error, FinishGetChunkOperation should be // called when the operation completes. @@ -90,19 +101,31 @@ func (l *ChunkRateLimiter) BeginGetChunkOperation( } if l.globalOperationsInFlight >= l.config.MaxConcurrentGetChunkOps { + if l.limiterCounter != nil { + l.limiterCounter.Increment(RateLimitLabel{"global concurrency"}) + } return fmt.Errorf( "global concurrent request limit %d exceeded for GetChunks operations, try again later", l.config.MaxConcurrentGetChunkOps) } if l.globalOpLimiter.TokensAt(now) < 1 { + if l.limiterCounter != nil { + l.limiterCounter.Increment(RateLimitLabel{"global rate"}) + } return fmt.Errorf("global rate limit %0.1fhz exceeded for GetChunks operations, try again later", l.config.MaxGetChunkOpsPerSecond) } if l.perClientOperationsInFlight[requesterID] >= l.config.MaxConcurrentGetChunkOpsClient { + if l.limiterCounter != nil { + l.limiterCounter.Increment(RateLimitLabel{"client concurrency"}) + } return fmt.Errorf("client concurrent request limit %d exceeded for GetChunks", l.config.MaxConcurrentGetChunkOpsClient) } if l.perClientOpLimiter[requesterID].TokensAt(now) < 1 { + if l.limiterCounter != nil { + l.limiterCounter.Increment(RateLimitLabel{"client rate"}) + } return fmt.Errorf("client rate limit %0.1fhz exceeded for GetChunks, try again later", l.config.MaxGetChunkOpsPerSecondClient) } @@ -139,6 +162,9 @@ func (l *ChunkRateLimiter) RequestGetChunkBandwidth(now time.Time, requesterID s allowed := l.globalBandwidthLimiter.AllowN(now, bytes) if !allowed { + if l.limiterCounter != nil { + l.limiterCounter.Increment(RateLimitLabel{"global bandwidth"}) + } return fmt.Errorf("global rate limit %dMiB exceeded for GetChunk bandwidth, try again later", int(l.config.MaxGetChunkBytesPerSecond/1024/1024)) } @@ -150,6 +176,9 @@ func (l *ChunkRateLimiter) RequestGetChunkBandwidth(now time.Time, requesterID s allowed = limiter.AllowN(now, bytes) if !allowed { l.globalBandwidthLimiter.AllowN(now, -bytes) + if l.limiterCounter != nil { + l.limiterCounter.Increment(RateLimitLabel{"client bandwidth"}) + } return fmt.Errorf("client rate limit %dMiB exceeded for GetChunk bandwidth, try again later", int(l.config.MaxGetChunkBytesPerSecondClient/1024/1024)) } diff --git a/relay/limiter/chunk_rate_limiter_test.go b/relay/limiter/chunk_rate_limiter_test.go index 59399ca17..98116e07c 100644 --- a/relay/limiter/chunk_rate_limiter_test.go +++ b/relay/limiter/chunk_rate_limiter_test.go @@ -22,7 +22,7 @@ func TestConcurrentGetChunksOperations(t *testing.T) { userID := tu.RandomString(64) - limiter := NewChunkRateLimiter(config) + limiter := NewChunkRateLimiter(config, nil) // time starts at current time, but advances manually afterward now := time.Now() @@ -56,7 +56,7 @@ func TestGetChunksRateLimit(t *testing.T) { userID := tu.RandomString(64) - limiter := NewChunkRateLimiter(config) + limiter := NewChunkRateLimiter(config, nil) // time starts at current time, but advances manually afterward now := time.Now() @@ -120,7 +120,7 @@ func TestGetChunksBandwidthLimit(t *testing.T) { userID := tu.RandomString(64) - limiter := NewChunkRateLimiter(config) + limiter := NewChunkRateLimiter(config, nil) // time starts at current time, but advances manually afterward now := time.Now() @@ -170,7 +170,7 @@ func TestPerClientConcurrencyLimit(t *testing.T) { userID1 := tu.RandomString(64) userID2 := tu.RandomString(64) - limiter := NewChunkRateLimiter(config) + limiter := NewChunkRateLimiter(config, nil) // time starts at current time, but advances manually afterward now := time.Now() @@ -218,7 +218,7 @@ func TestOpLimitPerClient(t *testing.T) { userID1 := tu.RandomString(64) userID2 := tu.RandomString(64) - limiter := NewChunkRateLimiter(config) + limiter := NewChunkRateLimiter(config, nil) // time starts at current time, but advances manually afterward now := time.Now() @@ -276,7 +276,7 @@ func TestBandwidthLimitPerClient(t *testing.T) { userID1 := tu.RandomString(64) userID2 := tu.RandomString(64) - limiter := NewChunkRateLimiter(config) + limiter := NewChunkRateLimiter(config, nil) // time starts at current time, but advances manually afterward now := time.Now() diff --git a/relay/metrics/relay_metrics.go b/relay/metrics/relay_metrics.go index d7b8ffddd..92c4bb4c5 100644 --- a/relay/metrics/relay_metrics.go +++ b/relay/metrics/relay_metrics.go @@ -2,6 +2,7 @@ package metrics import ( "github.com/Layr-Labs/eigenda/common/metrics" + "github.com/Layr-Labs/eigenda/relay/limiter" "github.com/Layr-Labs/eigensdk-go/logging" grpcprom "github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus" "google.golang.org/grpc" @@ -10,6 +11,13 @@ import ( type RelayMetrics struct { metricsServer metrics.Metrics grpcServerOption grpc.ServerOption + + GetChunksLatency metrics.LatencyMetric + GetChunksAuthenticationLatency metrics.LatencyMetric + GetChunksMetadataLatency metrics.LatencyMetric + GetChunksDataLatency metrics.LatencyMetric + GetChunksAuthFailures metrics.CountMetric + GetChunksRateLimited metrics.CountMetric } // NewRelayMetrics creates a new RelayMetrics instance, which encapsulates all metrics related to the relay. @@ -23,9 +31,73 @@ func NewRelayMetrics(logger logging.Logger, port int) (*RelayMetrics, error) { grpcMetrics.UnaryServerInterceptor(), ) + standardQuantiles := []*metrics.Quantile{ + metrics.NewQuantile(0.5), + metrics.NewQuantile(0.9), + metrics.NewQuantile(0.99), + } + + getChunksLatencyMetric, err := server.NewLatencyMetric( + "get_chunks_latency", + "Latency of the GetChunks RPC", + nil, + standardQuantiles...) + if err != nil { + return nil, err + } + + getChunksAuthenticationLatencyMetric, err := server.NewLatencyMetric( + "get_chunks_authentication_latency", + "Latency of the GetChunks RPC client authentication", + nil, + standardQuantiles...) + if err != nil { + return nil, err + } + + getChunksMetadataLatencyMetric, err := server.NewLatencyMetric( + "get_chunks_metadata_latency", + "Latency of the GetChunks RPC metadata retrieval", + nil, + standardQuantiles...) + if err != nil { + return nil, err + } + + getChunksDataLatencyMetric, err := server.NewLatencyMetric( + "get_chunks_data_latency", + "Latency of the GetChunks RPC data retrieval", + nil, + standardQuantiles...) + if err != nil { + return nil, err + } + + getChunksAuthFailures, err := server.NewCountMetric( + "get_chunks_auth_failure", + "Number of GetChunks RPC authentication failures", + nil) + if err != nil { + return nil, err + } + + getChunksRateLimited, err := server.NewCountMetric( + "get_chunks_rate_limited", + "Number of GetChunks RPC rate limited", + limiter.RateLimitLabel{}) + if err != nil { + return nil, err + } + return &RelayMetrics{ - metricsServer: server, - grpcServerOption: grpcServerOption, + metricsServer: server, + grpcServerOption: grpcServerOption, + GetChunksLatency: getChunksLatencyMetric, + GetChunksAuthenticationLatency: getChunksAuthenticationLatencyMetric, + GetChunksMetadataLatency: getChunksMetadataLatencyMetric, + GetChunksDataLatency: getChunksDataLatencyMetric, + GetChunksAuthFailures: getChunksAuthFailures, + GetChunksRateLimited: getChunksRateLimited, }, nil } diff --git a/relay/server.go b/relay/server.go index 63f608ff9..1d6956a7d 100644 --- a/relay/server.go +++ b/relay/server.go @@ -203,7 +203,7 @@ func NewServer( blobProvider: bp, chunkProvider: cp, blobRateLimiter: limiter.NewBlobRateLimiter(&config.RateLimits), - chunkRateLimiter: limiter.NewChunkRateLimiter(&config.RateLimits), + chunkRateLimiter: limiter.NewChunkRateLimiter(&config.RateLimits, relayMetrics.GetChunksRateLimited), authenticator: authenticator, relayMetrics: relayMetrics, }, nil @@ -258,6 +258,8 @@ func (s *Server) GetBlob(ctx context.Context, request *pb.GetBlobRequest) (*pb.G // GetChunks retrieves chunks from blobs stored by the relay. func (s *Server) GetChunks(ctx context.Context, request *pb.GetChunksRequest) (*pb.GetChunksReply, error) { + start := time.Now() + if s.config.Timeouts.GetChunksTimeout > 0 { var cancel context.CancelFunc ctx, cancel = context.WithTimeout(ctx, s.config.Timeouts.GetChunksTimeout) @@ -281,13 +283,18 @@ func (s *Server) GetChunks(ctx context.Context, request *pb.GetChunksRequest) (* err := s.authenticator.AuthenticateGetChunksRequest(ctx, clientAddress, request, time.Now()) if err != nil { + s.relayMetrics.GetChunksAuthFailures.Increment() return nil, fmt.Errorf("auth failed: %w", err) } } + finishedAuthenticating := time.Now() + if s.authenticator != nil { + s.relayMetrics.GetChunksAuthenticationLatency.ReportLatency(finishedAuthenticating.Sub(start)) + } + clientID := string(request.OperatorId) err := s.chunkRateLimiter.BeginGetChunkOperation(time.Now(), clientID) - if err != nil { return nil, err } @@ -305,6 +312,9 @@ func (s *Server) GetChunks(ctx context.Context, request *pb.GetChunksRequest) (* "error fetching metadata for blob, check if blob exists and is assigned to this relay: %w", err) } + finishedFetchingMetadata := time.Now() + s.relayMetrics.GetChunksMetadataLatency.ReportLatency(finishedFetchingMetadata.Sub(finishedAuthenticating)) + requiredBandwidth, err := computeChunkRequestRequiredBandwidth(request, mMap) if err != nil { return nil, fmt.Errorf("error computing required bandwidth: %w", err) @@ -324,6 +334,10 @@ func (s *Server) GetChunks(ctx context.Context, request *pb.GetChunksRequest) (* return nil, fmt.Errorf("error gathering chunk data: %w", err) } + finishedFetchingData := time.Now() + s.relayMetrics.GetChunksDataLatency.ReportLatency(finishedFetchingData.Sub(finishedFetchingMetadata)) + s.relayMetrics.GetChunksLatency.ReportLatency(time.Since(start)) + return &pb.GetChunksReply{ Data: bytesToSend, }, nil @@ -443,6 +457,11 @@ func computeChunkRequestRequiredBandwidth(request *pb.GetChunksRequest, mMap met // Start starts the server listening for requests. This method will block until the server is stopped. func (s *Server) Start(ctx context.Context) error { + err := s.relayMetrics.Start() + if err != nil { + return fmt.Errorf("error starting metrics server: %w", err) + } + if s.chainReader != nil && s.metadataProvider != nil { go func() { _ = s.RefreshOnchainState(ctx) @@ -467,16 +486,10 @@ func (s *Server) Start(ctx context.Context) error { healthcheck.RegisterHealthServer(name, s.grpcServer) s.logger.Info("GRPC Listening", "port", s.config.GRPCPort, "address", listener.Addr().String()) - if err = s.grpcServer.Serve(listener); err != nil { return errors.New("could not start GRPC server") } - err = s.relayMetrics.Start() - if err != nil { - return fmt.Errorf("error starting metrics server: %w", err) - } - return nil } diff --git a/relay/server_test.go b/relay/server_test.go index 3a7e31977..e5990cc84 100644 --- a/relay/server_test.go +++ b/relay/server_test.go @@ -2,6 +2,7 @@ package relay import ( "context" + "fmt" "math/rand" "testing" "time" @@ -56,6 +57,7 @@ func defaultConfig() *Config { InternalGetProofsTimeout: 10 * time.Second, InternalGetCoefficientsTimeout: 10 * time.Second, }, + MetricsPort: 9101, } } @@ -394,8 +396,9 @@ func TestReadWriteChunks(t *testing.T) { expectedData := make(map[v2.BlobKey][]*encoding.Frame) fragmentInfoMap := make(map[v2.BlobKey]*encoding.FragmentInfo) - blobCount := 10 + blobCount := 100 // TODO revert this to 10 for i := 0; i < blobCount; i++ { + fmt.Printf("blob %d\n", i) // TODO remove this header, _, chunks := randomBlobChunks(t) blobKey, err := header.BlobKey() From 2d7e9efa237d671e7ab2ceb12dc808626c9b57d8 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Wed, 27 Nov 2024 09:14:07 -0600 Subject: [PATCH 04/30] Incremental progress, need running averages. Signed-off-by: Cody Littley --- common/metrics/histogram_metric.go | 101 +++++++++++++++++++++++++++++ common/metrics/metrics.go | 23 +++++++ common/metrics/metrics_server.go | 38 +++++++++++ common/metrics/mock_metrics.go | 33 ++++++++++ relay/metrics/relay_metrics.go | 24 +++++++ relay/server.go | 2 + 6 files changed, 221 insertions(+) create mode 100644 common/metrics/histogram_metric.go diff --git a/common/metrics/histogram_metric.go b/common/metrics/histogram_metric.go new file mode 100644 index 000000000..03226ce14 --- /dev/null +++ b/common/metrics/histogram_metric.go @@ -0,0 +1,101 @@ +package metrics + +import ( + "fmt" + "github.com/Layr-Labs/eigensdk-go/logging" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" +) + +var _ HistogramMetric = &histogramMetric{} + +type histogramMetric struct { + + // logger is the logger used to log errors. + logger logging.Logger + + // name is the name of the metric. + name string + + // unit is the unit of the metric. + unit string + + // description is the description of the metric. + description string + + // vec is the prometheus histogram vector used to report this metric. + vec *prometheus.HistogramVec + + // lm is the label maker used to create labels for this metric. + labeler *labelMaker +} + +// newHistogramMetric creates a new HistogramMetric instance. +func newHistogramMetric( + logger logging.Logger, + registry *prometheus.Registry, + namespace string, + name string, + unit string, + description string, + bucketFactor float64, + labelTemplate any) (HistogramMetric, error) { + + labeler, err := newLabelMaker(labelTemplate) + if err != nil { + return nil, err + } + + vec := promauto.With(registry).NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: namespace, + Name: fmt.Sprintf("%s_%s", name, unit), + Help: description, + NativeHistogramBucketFactor: bucketFactor, + }, + labeler.getKeys(), + ) + + return &histogramMetric{ + logger: logger, + name: name, + unit: unit, + description: description, + vec: vec, + labeler: labeler, + }, nil +} + +func (m *histogramMetric) Name() string { + return m.name +} + +func (m *histogramMetric) Unit() string { + return m.unit +} + +func (m *histogramMetric) Description() string { + return m.description +} + +func (m *histogramMetric) Type() string { + return "histogram" +} + +func (m *histogramMetric) LabelFields() []string { + return m.labeler.getKeys() +} + +func (m *histogramMetric) Observe(value float64, label ...any) { + var l any + if len(label) > 0 { + l = label[0] + } + + values, err := m.labeler.extractValues(l) + if err != nil { + m.logger.Errorf("error extracting values from label: %v", err) + } + + m.vec.WithLabelValues(values...).Observe(value) +} diff --git a/common/metrics/metrics.go b/common/metrics/metrics.go index 4fbf5d054..da6be3498 100644 --- a/common/metrics/metrics.go +++ b/common/metrics/metrics.go @@ -66,6 +66,21 @@ type Metrics interface { source func() float64, label ...any) error + // NewHistogramMetric creates a new HistogramMetric instance. Useful for tracking the distribution of values. + // Metric name and label may only contain alphanumeric characters and underscores. + // + // Suggested bucket factor is 1.1. For additional documentation on bucket factor, see the prometheus documentation: + // https://github.com/prometheus/client_golang/blob/v1.20.5/prometheus/histogram.go#L430 + // + // The labelTemplate parameter is the label type that will be used for this metric. Each field becomes a label for + // the metric. Each field type must be a string. If no labels are needed, pass nil. + NewHistogramMetric( + name string, + unit string, + description string, + bucketFactor float64, + labelTemplate any) (HistogramMetric, error) + // RegisterExternalMetrics registers prometheus collectors created outside the metrics framework. RegisterExternalMetrics(collectors ...prometheus.Collector) } @@ -146,3 +161,11 @@ type LatencyMetric interface { // when creating the metric, an error will be returned. ReportLatency(latency time.Duration, label ...any) } + +// HistogramMetric allows the distribution of values to be tracked. +type HistogramMetric interface { + Metric + + // Observe reports a value to the histogram. + Observe(value float64, label ...any) +} diff --git a/common/metrics/metrics_server.go b/common/metrics/metrics_server.go index 7a5868dd0..818606c71 100644 --- a/common/metrics/metrics_server.go +++ b/common/metrics/metrics_server.go @@ -350,6 +350,44 @@ func (m *metrics) NewAutoGauge( return nil } +func (m *metrics) NewHistogramMetric( + name string, + unit string, + description string, + bucketFactor float64, + labelTemplate any) (HistogramMetric, error) { + + if !m.isAlive.Load() { + return nil, errors.New("metrics server is not alive") + } + + id, err := newMetricID(name, unit) + if err != nil { + return nil, err + } + + preExistingMetric, ok := m.metricMap[id] + if ok { + return preExistingMetric.(HistogramMetric), nil + } + + metric, err := newHistogramMetric( + m.logger, + m.registry, + m.namespace, + name, + unit, + description, + bucketFactor, + labelTemplate) + if err != nil { + return nil, err + } + + m.metricMap[id] = metric + return metric, nil +} + func (m *metrics) GenerateMetricsDocumentation() string { sb := &strings.Builder{} diff --git a/common/metrics/mock_metrics.go b/common/metrics/mock_metrics.go index 244f268c9..994ab9fec 100644 --- a/common/metrics/mock_metrics.go +++ b/common/metrics/mock_metrics.go @@ -63,6 +63,10 @@ func (m *mockMetrics) NewAutoGauge( return nil } +func (m *mockMetrics) NewHistogramMetric(name string, unit string, description string, bucketFactor float64, labelTemplate any) (HistogramMetric, error) { + return &mockHistogramMetric{}, nil +} + func (m *mockMetrics) RegisterExternalMetrics(collectors ...prometheus.Collector) { } @@ -157,3 +161,32 @@ func (m *mockLatencyMetric) LabelFields() []string { func (m *mockLatencyMetric) ReportLatency(latency time.Duration, label ...any) { } + +var _ HistogramMetric = &mockHistogramMetric{} + +type mockHistogramMetric struct { +} + +func (m mockHistogramMetric) Name() string { + return "" +} + +func (m mockHistogramMetric) Unit() string { + return "" +} + +func (m mockHistogramMetric) Description() string { + return "" +} + +func (m mockHistogramMetric) Type() string { + return "" +} + +func (m mockHistogramMetric) LabelFields() []string { + return make([]string, 0) +} + +func (m mockHistogramMetric) Observe(value float64, label ...any) { + +} diff --git a/relay/metrics/relay_metrics.go b/relay/metrics/relay_metrics.go index 92c4bb4c5..c8c61151a 100644 --- a/relay/metrics/relay_metrics.go +++ b/relay/metrics/relay_metrics.go @@ -18,6 +18,8 @@ type RelayMetrics struct { GetChunksDataLatency metrics.LatencyMetric GetChunksAuthFailures metrics.CountMetric GetChunksRateLimited metrics.CountMetric + GetChunksKeyCountHistogram metrics.HistogramMetric + GetChunksDataSizeHistogram metrics.HistogramMetric } // NewRelayMetrics creates a new RelayMetrics instance, which encapsulates all metrics related to the relay. @@ -89,6 +91,26 @@ func NewRelayMetrics(logger logging.Logger, port int) (*RelayMetrics, error) { return nil, err } + getChunksKeyCountHistogram, err := server.NewHistogramMetric( + "get_chunks_key", + "count", + "Number of keys in a GetChunks request", + 1.1, + nil) + if err != nil { + return nil, err + } + + getChunksDataSizeHistogram, err := server.NewHistogramMetric( + "get_chunks_data", + "bytes", + "Size of data in a GetChunks request, in bytes", + 1.1, + nil) + if err != nil { + return nil, err + } + return &RelayMetrics{ metricsServer: server, grpcServerOption: grpcServerOption, @@ -98,6 +120,8 @@ func NewRelayMetrics(logger logging.Logger, port int) (*RelayMetrics, error) { GetChunksDataLatency: getChunksDataLatencyMetric, GetChunksAuthFailures: getChunksAuthFailures, GetChunksRateLimited: getChunksRateLimited, + GetChunksKeyCountHistogram: getChunksKeyCountHistogram, + GetChunksDataSizeHistogram: getChunksDataSizeHistogram, }, nil } diff --git a/relay/server.go b/relay/server.go index 1d6956a7d..6c4dccfb6 100644 --- a/relay/server.go +++ b/relay/server.go @@ -273,6 +273,7 @@ func (s *Server) GetChunks(ctx context.Context, request *pb.GetChunksRequest) (* return nil, fmt.Errorf( "too many chunk requests provided, max is %d", s.config.MaxKeysPerGetChunksRequest) } + s.relayMetrics.GetChunksKeyCountHistogram.Observe(float64(len(request.ChunkRequests))) if s.authenticator != nil { client, ok := peer.FromContext(ctx) @@ -323,6 +324,7 @@ func (s *Server) GetChunks(ctx context.Context, request *pb.GetChunksRequest) (* if err != nil { return nil, err } + s.relayMetrics.GetChunksDataSizeHistogram.Observe(float64(requiredBandwidth)) frames, err := s.chunkProvider.GetFrames(ctx, mMap) if err != nil { From b8c7d35502ea2f58f9df8f25e04a2d36625b49e4 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Wed, 27 Nov 2024 09:57:17 -0600 Subject: [PATCH 05/30] Added running average metrics for GetChunks Signed-off-by: Cody Littley --- common/metrics/histogram_metric.go | 101 ---------------------- common/metrics/metrics.go | 24 ++---- common/metrics/metrics_server.go | 12 +-- common/metrics/mock_metrics.go | 25 +++--- common/metrics/running_average.go | 70 +++++++++++++++ common/metrics/running_average_metric.go | 105 +++++++++++++++++++++++ go.mod | 1 + go.sum | 2 + relay/metrics/relay_metrics.go | 25 +++--- relay/server.go | 4 +- 10 files changed, 223 insertions(+), 146 deletions(-) delete mode 100644 common/metrics/histogram_metric.go create mode 100644 common/metrics/running_average.go create mode 100644 common/metrics/running_average_metric.go diff --git a/common/metrics/histogram_metric.go b/common/metrics/histogram_metric.go deleted file mode 100644 index 03226ce14..000000000 --- a/common/metrics/histogram_metric.go +++ /dev/null @@ -1,101 +0,0 @@ -package metrics - -import ( - "fmt" - "github.com/Layr-Labs/eigensdk-go/logging" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" -) - -var _ HistogramMetric = &histogramMetric{} - -type histogramMetric struct { - - // logger is the logger used to log errors. - logger logging.Logger - - // name is the name of the metric. - name string - - // unit is the unit of the metric. - unit string - - // description is the description of the metric. - description string - - // vec is the prometheus histogram vector used to report this metric. - vec *prometheus.HistogramVec - - // lm is the label maker used to create labels for this metric. - labeler *labelMaker -} - -// newHistogramMetric creates a new HistogramMetric instance. -func newHistogramMetric( - logger logging.Logger, - registry *prometheus.Registry, - namespace string, - name string, - unit string, - description string, - bucketFactor float64, - labelTemplate any) (HistogramMetric, error) { - - labeler, err := newLabelMaker(labelTemplate) - if err != nil { - return nil, err - } - - vec := promauto.With(registry).NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: namespace, - Name: fmt.Sprintf("%s_%s", name, unit), - Help: description, - NativeHistogramBucketFactor: bucketFactor, - }, - labeler.getKeys(), - ) - - return &histogramMetric{ - logger: logger, - name: name, - unit: unit, - description: description, - vec: vec, - labeler: labeler, - }, nil -} - -func (m *histogramMetric) Name() string { - return m.name -} - -func (m *histogramMetric) Unit() string { - return m.unit -} - -func (m *histogramMetric) Description() string { - return m.description -} - -func (m *histogramMetric) Type() string { - return "histogram" -} - -func (m *histogramMetric) LabelFields() []string { - return m.labeler.getKeys() -} - -func (m *histogramMetric) Observe(value float64, label ...any) { - var l any - if len(label) > 0 { - l = label[0] - } - - values, err := m.labeler.extractValues(l) - if err != nil { - m.logger.Errorf("error extracting values from label: %v", err) - } - - m.vec.WithLabelValues(values...).Observe(value) -} diff --git a/common/metrics/metrics.go b/common/metrics/metrics.go index da6be3498..b52f576b4 100644 --- a/common/metrics/metrics.go +++ b/common/metrics/metrics.go @@ -66,20 +66,14 @@ type Metrics interface { source func() float64, label ...any) error - // NewHistogramMetric creates a new HistogramMetric instance. Useful for tracking the distribution of values. - // Metric name and label may only contain alphanumeric characters and underscores. - // - // Suggested bucket factor is 1.1. For additional documentation on bucket factor, see the prometheus documentation: - // https://github.com/prometheus/client_golang/blob/v1.20.5/prometheus/histogram.go#L430 - // - // The labelTemplate parameter is the label type that will be used for this metric. Each field becomes a label for - // the metric. Each field type must be a string. If no labels are needed, pass nil. - NewHistogramMetric( + // NewRunningAverageMetric creates a new GaugeMetric instance that keeps track of the average of a series of values + // over a given time window. Each value within the window is given equal weight. + NewRunningAverageMetric( name string, unit string, description string, - bucketFactor float64, - labelTemplate any) (HistogramMetric, error) + timeWindow time.Duration, + labelTemplate any) (RunningAverageMetric, error) // RegisterExternalMetrics registers prometheus collectors created outside the metrics framework. RegisterExternalMetrics(collectors ...prometheus.Collector) @@ -162,10 +156,10 @@ type LatencyMetric interface { ReportLatency(latency time.Duration, label ...any) } -// HistogramMetric allows the distribution of values to be tracked. -type HistogramMetric interface { +// RunningAverageMetric tracks the average of a series of values over a given time window. +type RunningAverageMetric interface { Metric - // Observe reports a value to the histogram. - Observe(value float64, label ...any) + // Update adds a new value to the RunningAverage. + Update(value float64, label ...any) } diff --git a/common/metrics/metrics_server.go b/common/metrics/metrics_server.go index 818606c71..2fba427aa 100644 --- a/common/metrics/metrics_server.go +++ b/common/metrics/metrics_server.go @@ -350,12 +350,12 @@ func (m *metrics) NewAutoGauge( return nil } -func (m *metrics) NewHistogramMetric( +func (m *metrics) NewRunningAverageMetric( name string, unit string, description string, - bucketFactor float64, - labelTemplate any) (HistogramMetric, error) { + timeWindow time.Duration, + labelTemplate any) (RunningAverageMetric, error) { if !m.isAlive.Load() { return nil, errors.New("metrics server is not alive") @@ -368,17 +368,17 @@ func (m *metrics) NewHistogramMetric( preExistingMetric, ok := m.metricMap[id] if ok { - return preExistingMetric.(HistogramMetric), nil + return preExistingMetric.(RunningAverageMetric), nil } - metric, err := newHistogramMetric( + metric, err := newRunningAverageMetric( m.logger, m.registry, m.namespace, name, unit, description, - bucketFactor, + timeWindow, labelTemplate) if err != nil { return nil, err diff --git a/common/metrics/mock_metrics.go b/common/metrics/mock_metrics.go index 994ab9fec..2f5deaf09 100644 --- a/common/metrics/mock_metrics.go +++ b/common/metrics/mock_metrics.go @@ -63,8 +63,13 @@ func (m *mockMetrics) NewAutoGauge( return nil } -func (m *mockMetrics) NewHistogramMetric(name string, unit string, description string, bucketFactor float64, labelTemplate any) (HistogramMetric, error) { - return &mockHistogramMetric{}, nil +func (m *mockMetrics) NewRunningAverageMetric( + name string, + unit string, + description string, + timeWindow time.Duration, + labelTemplate any) (RunningAverageMetric, error) { + return &mockRunningAverageMetric{}, nil } func (m *mockMetrics) RegisterExternalMetrics(collectors ...prometheus.Collector) { @@ -162,31 +167,31 @@ func (m *mockLatencyMetric) ReportLatency(latency time.Duration, label ...any) { } -var _ HistogramMetric = &mockHistogramMetric{} +var _ RunningAverageMetric = &mockRunningAverageMetric{} -type mockHistogramMetric struct { +type mockRunningAverageMetric struct { } -func (m mockHistogramMetric) Name() string { +func (m *mockRunningAverageMetric) Name() string { return "" } -func (m mockHistogramMetric) Unit() string { +func (m *mockRunningAverageMetric) Unit() string { return "" } -func (m mockHistogramMetric) Description() string { +func (m *mockRunningAverageMetric) Description() string { return "" } -func (m mockHistogramMetric) Type() string { +func (m *mockRunningAverageMetric) Type() string { return "" } -func (m mockHistogramMetric) LabelFields() []string { +func (m *mockRunningAverageMetric) LabelFields() []string { return make([]string, 0) } -func (m mockHistogramMetric) Observe(value float64, label ...any) { +func (m *mockRunningAverageMetric) Update(value float64, label ...any) { } diff --git a/common/metrics/running_average.go b/common/metrics/running_average.go new file mode 100644 index 000000000..409a31a49 --- /dev/null +++ b/common/metrics/running_average.go @@ -0,0 +1,70 @@ +package metrics + +import ( + "github.com/emirpasic/gods/queues" + "github.com/emirpasic/gods/queues/linkedlistqueue" + "time" +) + +// RunningAverage keeps track of the average of a series of values over a given time window. +type RunningAverage struct { + maxAge time.Duration + sum float64 + count int + entries queues.Queue +} + +// NewRunningAverage creates a new RunningAverage with the given time window. +func NewRunningAverage(maxAge time.Duration) *RunningAverage { + return &RunningAverage{ + maxAge: maxAge, + entries: linkedlistqueue.New(), + } +} + +type runningAverageEntry struct { + value float64 + time time.Time +} + +// Update adds a new value to the RunningAverage and returns the new average. +func (a *RunningAverage) Update(now time.Time, value float64) float64 { + a.count++ + a.sum += value + a.entries.Enqueue(&runningAverageEntry{value: value, time: now}) + return a.GetAverage(now) +} + +// GetAverage returns the current average of the RunningAverage. +func (a *RunningAverage) GetAverage(now time.Time) float64 { + a.cleanup(now) + if a.count == 0 { + return 0 + } + return a.sum / float64(a.count) +} + +// cleanup removes old entries from the RunningAverage. +func (a *RunningAverage) cleanup(now time.Time) { + + for { + v, ok := a.entries.Peek() + if !ok { + break + } + entry := v.(*runningAverageEntry) + + if now.Sub(entry.time) <= a.maxAge { + break + } + + a.entries.Dequeue() + a.sum -= entry.value + a.count-- + } + + if a.count == 0 { + // clear away any cruft from accumulated floating point errors if we have no entries + a.sum = 0 + } +} diff --git a/common/metrics/running_average_metric.go b/common/metrics/running_average_metric.go new file mode 100644 index 000000000..f4aaaf113 --- /dev/null +++ b/common/metrics/running_average_metric.go @@ -0,0 +1,105 @@ +package metrics + +import ( + "fmt" + "github.com/Layr-Labs/eigensdk-go/logging" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "time" +) + +var _ RunningAverageMetric = &runningAverageMetric{} + +type runningAverageMetric struct { + + // logger is the logger used to log errors. + logger logging.Logger + + // name is the name of the metric. + name string + + // unit is the unit of the metric. + unit string + + // description is the description of the metric. + description string + + // vec is the prometheus gauge vector used to store the metric. + vec *prometheus.GaugeVec + + // lm is the label maker used to create labels for this metric. + labeler *labelMaker + + // runningAverage is the running average used to calculate the average of the metric. + runningAverage *RunningAverage +} + +// newRunningAverageMetric creates a new RunningAverageMetric instance. +func newRunningAverageMetric( + logger logging.Logger, + registry *prometheus.Registry, + namespace string, + name string, + unit string, + description string, + timeWindow time.Duration, + labelTemplate any) (RunningAverageMetric, error) { + + labeler, err := newLabelMaker(labelTemplate) + if err != nil { + return nil, err + } + + vec := promauto.With(registry).NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: namespace, + Name: fmt.Sprintf("%s_%s", name, unit), + }, + labeler.getKeys(), + ) + + return &runningAverageMetric{ + logger: logger, + name: name, + unit: unit, + description: description, + vec: vec, + labeler: labeler, + runningAverage: NewRunningAverage(timeWindow), + }, nil +} + +func (m *runningAverageMetric) Name() string { + return m.name +} + +func (m *runningAverageMetric) Unit() string { + return m.unit +} + +func (m *runningAverageMetric) Description() string { + return m.description +} + +func (m *runningAverageMetric) Type() string { + return "histogram" +} + +func (m *runningAverageMetric) LabelFields() []string { + return m.labeler.getKeys() +} + +func (m *runningAverageMetric) Update(value float64, label ...any) { + var l any + if len(label) > 0 { + l = label[0] + } + + values, err := m.labeler.extractValues(l) + if err != nil { + m.logger.Errorf("error extracting values from label: %v", err) + } + + average := m.runningAverage.Update(time.Now(), value) + m.vec.WithLabelValues(values...).Set(average) +} diff --git a/go.mod b/go.mod index 45e0bbe49..5703948b0 100644 --- a/go.mod +++ b/go.mod @@ -85,6 +85,7 @@ require ( github.com/docker/docker v25.0.5+incompatible // indirect github.com/docker/go-connections v0.5.0 // indirect github.com/docker/go-units v0.5.0 // indirect + github.com/emirpasic/gods v1.18.1 // indirect github.com/ethereum/c-kzg-4844 v1.0.0 // indirect github.com/ethereum/go-verkle v0.1.1-0.20240306133620-7d920df305f0 // indirect github.com/gabriel-vasile/mimetype v1.4.2 // indirect diff --git a/go.sum b/go.sum index 4762b276f..d3b4dde0b 100644 --- a/go.sum +++ b/go.sum @@ -165,6 +165,8 @@ github.com/docker/go-connections v0.5.0/go.mod h1:ov60Kzw0kKElRwhNs9UlUHAE/F9Fe6 github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= github.com/docker/go-units v0.5.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= +github.com/emirpasic/gods v1.18.1 h1:FXtiHYKDGKCW2KzwZKx0iC0PQmdlorYgdFG9jPXJ1Bc= +github.com/emirpasic/gods v1.18.1/go.mod h1:8tpGGwCnJ5H4r6BWwaV6OrWmMoPhUl5jm/FMNAnJvWQ= github.com/ethereum/c-kzg-4844 v1.0.0 h1:0X1LBXxaEtYD9xsyj9B9ctQEZIpnvVDeoBx8aHEwTNA= github.com/ethereum/c-kzg-4844 v1.0.0/go.mod h1:VewdlzQmpT5QSrVhbBuGoCdFJkpaJlO1aQputP83wc0= github.com/ethereum/go-ethereum v1.14.8 h1:NgOWvXS+lauK+zFukEvi85UmmsS/OkV0N23UZ1VTIig= diff --git a/relay/metrics/relay_metrics.go b/relay/metrics/relay_metrics.go index c8c61151a..81fca84ad 100644 --- a/relay/metrics/relay_metrics.go +++ b/relay/metrics/relay_metrics.go @@ -6,6 +6,7 @@ import ( "github.com/Layr-Labs/eigensdk-go/logging" grpcprom "github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus" "google.golang.org/grpc" + "time" ) type RelayMetrics struct { @@ -18,8 +19,8 @@ type RelayMetrics struct { GetChunksDataLatency metrics.LatencyMetric GetChunksAuthFailures metrics.CountMetric GetChunksRateLimited metrics.CountMetric - GetChunksKeyCountHistogram metrics.HistogramMetric - GetChunksDataSizeHistogram metrics.HistogramMetric + GetChunksAverageKeyCount metrics.RunningAverageMetric + GetChunksAverageDataSize metrics.RunningAverageMetric } // NewRelayMetrics creates a new RelayMetrics instance, which encapsulates all metrics related to the relay. @@ -91,21 +92,21 @@ func NewRelayMetrics(logger logging.Logger, port int) (*RelayMetrics, error) { return nil, err } - getChunksKeyCountHistogram, err := server.NewHistogramMetric( - "get_chunks_key", + getChunksAverageKeyCount, err := server.NewRunningAverageMetric( + "average_get_chunks_key", "count", - "Number of keys in a GetChunks request", - 1.1, + "Average number of keys in a GetChunks request", + time.Minute, nil) if err != nil { return nil, err } - getChunksDataSizeHistogram, err := server.NewHistogramMetric( - "get_chunks_data", + getChunksAverageDataSize, err := server.NewRunningAverageMetric( + "average_get_chunks_data", "bytes", - "Size of data in a GetChunks request, in bytes", - 1.1, + "Average data size in a GetChunks request", + time.Minute, nil) if err != nil { return nil, err @@ -120,8 +121,8 @@ func NewRelayMetrics(logger logging.Logger, port int) (*RelayMetrics, error) { GetChunksDataLatency: getChunksDataLatencyMetric, GetChunksAuthFailures: getChunksAuthFailures, GetChunksRateLimited: getChunksRateLimited, - GetChunksKeyCountHistogram: getChunksKeyCountHistogram, - GetChunksDataSizeHistogram: getChunksDataSizeHistogram, + GetChunksAverageKeyCount: getChunksAverageKeyCount, + GetChunksAverageDataSize: getChunksAverageDataSize, }, nil } diff --git a/relay/server.go b/relay/server.go index 6c4dccfb6..a47a5c7ed 100644 --- a/relay/server.go +++ b/relay/server.go @@ -273,7 +273,7 @@ func (s *Server) GetChunks(ctx context.Context, request *pb.GetChunksRequest) (* return nil, fmt.Errorf( "too many chunk requests provided, max is %d", s.config.MaxKeysPerGetChunksRequest) } - s.relayMetrics.GetChunksKeyCountHistogram.Observe(float64(len(request.ChunkRequests))) + s.relayMetrics.GetChunksAverageKeyCount.Update(float64(len(request.ChunkRequests))) if s.authenticator != nil { client, ok := peer.FromContext(ctx) @@ -324,7 +324,7 @@ func (s *Server) GetChunks(ctx context.Context, request *pb.GetChunksRequest) (* if err != nil { return nil, err } - s.relayMetrics.GetChunksDataSizeHistogram.Observe(float64(requiredBandwidth)) + s.relayMetrics.GetChunksAverageDataSize.Update(float64(requiredBandwidth)) frames, err := s.chunkProvider.GetFrames(ctx, mMap) if err != nil { From b9d71d675714021160c713a4f61a248940fabe28 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Wed, 27 Nov 2024 10:05:31 -0600 Subject: [PATCH 06/30] Documentation Signed-off-by: Cody Littley --- common/metrics/running_average_metric.go | 2 +- metrics.md | 3 +- .../churner/{ => mdoc}/churner-metrics.md | 0 operators/churner/metrics.go | 7 +- relay/mdoc/main.go | 24 ++++ relay/mdoc/relay-metrics.md | 107 ++++++++++++++++++ relay/metrics/relay_metrics.go | 5 + 7 files changed, 141 insertions(+), 7 deletions(-) rename operators/churner/{ => mdoc}/churner-metrics.md (100%) create mode 100644 relay/mdoc/main.go create mode 100644 relay/mdoc/relay-metrics.md diff --git a/common/metrics/running_average_metric.go b/common/metrics/running_average_metric.go index f4aaaf113..3a1dc4466 100644 --- a/common/metrics/running_average_metric.go +++ b/common/metrics/running_average_metric.go @@ -82,7 +82,7 @@ func (m *runningAverageMetric) Description() string { } func (m *runningAverageMetric) Type() string { - return "histogram" + return "running average" } func (m *runningAverageMetric) LabelFields() []string { diff --git a/metrics.md b/metrics.md index 72c61a314..7138d98ca 100644 --- a/metrics.md +++ b/metrics.md @@ -1,4 +1,5 @@ # EigenDA Metrics Documentation -- [churner](operators/churner/churner-metrics.md) +- [churner](operators/churner/mdoc/churner-metrics.md) +- [relay](relay/mdoc/relay-metrics.md) diff --git a/operators/churner/churner-metrics.md b/operators/churner/mdoc/churner-metrics.md similarity index 100% rename from operators/churner/churner-metrics.md rename to operators/churner/mdoc/churner-metrics.md diff --git a/operators/churner/metrics.go b/operators/churner/metrics.go index 1f586e7ef..6ca30a0ed 100644 --- a/operators/churner/metrics.go +++ b/operators/churner/metrics.go @@ -65,10 +65,7 @@ func NewMetrics(httpPort int, logger logging.Logger) (*Metrics, error) { reg.MustRegister(collectors.NewProcessCollector(collectors.ProcessCollectorOpts{})) reg.MustRegister(collectors.NewGoCollector()) - metricsServer := metrics.NewMetrics(logger, &metrics.Config{ - Namespace: "eigenda_churner", - HTTPPort: httpPort, - }) + metricsServer := metrics.NewMetrics(logger, "eigenda_churner", httpPort) numRequests, err := metricsServer.NewCountMetric( "request", @@ -100,7 +97,7 @@ func NewMetrics(httpPort int, logger logging.Logger) (*Metrics, error) { // WriteMetricsDocumentation writes the metrics for the churner to a markdown file. func (g *Metrics) WriteMetricsDocumentation() error { - return g.metricsServer.WriteMetricsDocumentation("operators/churner/churner-metrics.md") + return g.metricsServer.WriteMetricsDocumentation("operators/churner/mdoc/churner-metrics.md") } // ObserveLatency observes the latency of a stage diff --git a/relay/mdoc/main.go b/relay/mdoc/main.go new file mode 100644 index 000000000..56c68999f --- /dev/null +++ b/relay/mdoc/main.go @@ -0,0 +1,24 @@ +package main + +import ( + "github.com/Layr-Labs/eigenda/common" + "github.com/Layr-Labs/eigenda/relay/metrics" +) + +// main generates documentation for relay metrics. +func main() { + logger, err := common.NewLogger(common.DefaultLoggerConfig()) + if err != nil { + panic(err) + } + + metrics, err := metrics.NewRelayMetrics(logger, 0) + if err != nil { + panic(err) + } + + err = metrics.WriteMetricsDocumentation() + if err != nil { + panic(err) + } +} diff --git a/relay/mdoc/relay-metrics.md b/relay/mdoc/relay-metrics.md new file mode 100644 index 000000000..2e29a30c8 --- /dev/null +++ b/relay/mdoc/relay-metrics.md @@ -0,0 +1,107 @@ +# Metrics Documentation for namespace 'relay' + +This documentation was automatically generated at time `2024-11-27T10:04:53-06:00` + +There are a total of `8` registered metrics. + +--- + +## average_get_chunks_data_bytes + +Average data size in a GetChunks request + +| | | +|---|---| +| **Name** | `average_get_chunks_data` | +| **Unit** | `bytes` | +| **Type** | `running average` | +| **Fully Qualified Name** | `relay_average_get_chunks_data_bytes` | +--- + +## average_get_chunks_key_count + +Average number of keys in a GetChunks request + +| | | +|---|---| +| **Name** | `average_get_chunks_key` | +| **Unit** | `count` | +| **Type** | `running average` | +| **Fully Qualified Name** | `relay_average_get_chunks_key_count` | +--- + +## get_chunks_auth_failure_count + +Number of GetChunks RPC authentication failures + +| | | +|---|---| +| **Name** | `get_chunks_auth_failure` | +| **Unit** | `count` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_get_chunks_auth_failure_count` | +--- + +## get_chunks_authentication_latency_ms + +Latency of the GetChunks RPC client authentication + +| | | +|---|---| +| **Name** | `get_chunks_authentication_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_get_chunks_authentication_latency_ms` | +--- + +## get_chunks_data_latency_ms + +Latency of the GetChunks RPC data retrieval + +| | | +|---|---| +| **Name** | `get_chunks_data_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_get_chunks_data_latency_ms` | +--- + +## get_chunks_latency_ms + +Latency of the GetChunks RPC + +| | | +|---|---| +| **Name** | `get_chunks_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_get_chunks_latency_ms` | +--- + +## get_chunks_metadata_latency_ms + +Latency of the GetChunks RPC metadata retrieval + +| | | +|---|---| +| **Name** | `get_chunks_metadata_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_get_chunks_metadata_latency_ms` | +--- + +## get_chunks_rate_limited_count + +Number of GetChunks RPC rate limited + +| | | +|---|---| +| **Name** | `get_chunks_rate_limited` | +| **Unit** | `count` | +| **Labels** | `reason` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_get_chunks_rate_limited_count` | diff --git a/relay/metrics/relay_metrics.go b/relay/metrics/relay_metrics.go index 81fca84ad..066e211aa 100644 --- a/relay/metrics/relay_metrics.go +++ b/relay/metrics/relay_metrics.go @@ -140,3 +140,8 @@ func (m *RelayMetrics) Stop() error { func (m *RelayMetrics) GetGRPCServerOption() grpc.ServerOption { return m.grpcServerOption } + +// WriteMetricsDocumentation writes the metrics for the churner to a markdown file. +func (m *RelayMetrics) WriteMetricsDocumentation() error { + return m.metricsServer.WriteMetricsDocumentation("relay/mdoc/relay-metrics.md") +} From 671f0c8e8948959fe4452f113fe89ee4bb3178be Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Wed, 27 Nov 2024 10:09:48 -0600 Subject: [PATCH 07/30] Add time window to metrics doc Signed-off-by: Cody Littley --- common/metrics/metrics.go | 3 +++ common/metrics/metrics_server.go | 4 ++++ common/metrics/mock_metrics.go | 4 ++++ common/metrics/running_average_metric.go | 8 ++++++++ relay/mdoc/relay-metrics.md | 4 +++- 5 files changed, 22 insertions(+), 1 deletion(-) diff --git a/common/metrics/metrics.go b/common/metrics/metrics.go index b52f576b4..4a201d260 100644 --- a/common/metrics/metrics.go +++ b/common/metrics/metrics.go @@ -162,4 +162,7 @@ type RunningAverageMetric interface { // Update adds a new value to the RunningAverage. Update(value float64, label ...any) + + // GetTimeWindow returns the time window used to calculate the running average. + GetTimeWindow() time.Duration } diff --git a/common/metrics/metrics_server.go b/common/metrics/metrics_server.go index 2fba427aa..c6cb9c9e5 100644 --- a/common/metrics/metrics_server.go +++ b/common/metrics/metrics_server.go @@ -439,6 +439,10 @@ func (m *metrics) GenerateMetricsDocumentation() string { if metric.Type() == "latency" { sb.Write([]byte(fmt.Sprintf("| **Quantiles** | %s |\n", m.quantilesMap[*id]))) } + if metric.Type() == "running average" { + sb.Write([]byte(fmt.Sprintf( + "| **Time Window** | `%s` |\n", metric.(*runningAverageMetric).GetTimeWindow()))) + } sb.Write([]byte(fmt.Sprintf("| **Fully Qualified Name** | `%s_%s_%s` |\n", m.namespace, id.name, id.unit))) } diff --git a/common/metrics/mock_metrics.go b/common/metrics/mock_metrics.go index 2f5deaf09..b55d205b3 100644 --- a/common/metrics/mock_metrics.go +++ b/common/metrics/mock_metrics.go @@ -195,3 +195,7 @@ func (m *mockRunningAverageMetric) LabelFields() []string { func (m *mockRunningAverageMetric) Update(value float64, label ...any) { } + +func (m *mockRunningAverageMetric) GetTimeWindow() time.Duration { + return 0 +} diff --git a/common/metrics/running_average_metric.go b/common/metrics/running_average_metric.go index 3a1dc4466..8456644e0 100644 --- a/common/metrics/running_average_metric.go +++ b/common/metrics/running_average_metric.go @@ -32,6 +32,9 @@ type runningAverageMetric struct { // runningAverage is the running average used to calculate the average of the metric. runningAverage *RunningAverage + + // timeWindow is the time window used to calculate the running average. + timeWindow time.Duration } // newRunningAverageMetric creates a new RunningAverageMetric instance. @@ -66,6 +69,7 @@ func newRunningAverageMetric( vec: vec, labeler: labeler, runningAverage: NewRunningAverage(timeWindow), + timeWindow: timeWindow, }, nil } @@ -103,3 +107,7 @@ func (m *runningAverageMetric) Update(value float64, label ...any) { average := m.runningAverage.Update(time.Now(), value) m.vec.WithLabelValues(values...).Set(average) } + +func (m *runningAverageMetric) GetTimeWindow() time.Duration { + return m.timeWindow +} diff --git a/relay/mdoc/relay-metrics.md b/relay/mdoc/relay-metrics.md index 2e29a30c8..541909a1e 100644 --- a/relay/mdoc/relay-metrics.md +++ b/relay/mdoc/relay-metrics.md @@ -1,6 +1,6 @@ # Metrics Documentation for namespace 'relay' -This documentation was automatically generated at time `2024-11-27T10:04:53-06:00` +This documentation was automatically generated at time `2024-11-27T10:08:20-06:00` There are a total of `8` registered metrics. @@ -15,6 +15,7 @@ Average data size in a GetChunks request | **Name** | `average_get_chunks_data` | | **Unit** | `bytes` | | **Type** | `running average` | +| **Time Window** | `1m0s` | | **Fully Qualified Name** | `relay_average_get_chunks_data_bytes` | --- @@ -27,6 +28,7 @@ Average number of keys in a GetChunks request | **Name** | `average_get_chunks_key` | | **Unit** | `count` | | **Type** | `running average` | +| **Time Window** | `1m0s` | | **Fully Qualified Name** | `relay_average_get_chunks_key_count` | --- From 4adb7eae306fbe6a95935b2322008512f713d200 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Wed, 27 Nov 2024 10:24:24 -0600 Subject: [PATCH 08/30] Added GetBlob metrics. Signed-off-by: Cody Littley --- relay/limiter/blob_rate_limiter.go | 15 +++++- relay/limiter/blob_rate_limiter_test.go | 6 +-- relay/mdoc/main.go | 4 +- relay/mdoc/relay-metrics.md | 69 ++++++++++++++++++++++++- relay/{metrics => }/relay_metrics.go | 62 +++++++++++++++++++++- relay/server.go | 41 +++++++++------ 6 files changed, 171 insertions(+), 26 deletions(-) rename relay/{metrics => }/relay_metrics.go (72%) diff --git a/relay/limiter/blob_rate_limiter.go b/relay/limiter/blob_rate_limiter.go index 1131af863..d17751d29 100644 --- a/relay/limiter/blob_rate_limiter.go +++ b/relay/limiter/blob_rate_limiter.go @@ -2,6 +2,7 @@ package limiter import ( "fmt" + "github.com/Layr-Labs/eigenda/common/metrics" "golang.org/x/time/rate" "sync" "time" @@ -23,12 +24,15 @@ type BlobRateLimiter struct { // operationsInFlight is the number of GetBlob operations currently in flight. operationsInFlight int + // limitCounter is used to track rate limiting events, ignored if nil + limitCounter metrics.CountMetric + // this lock is used to provide thread safety lock sync.Mutex } // NewBlobRateLimiter creates a new BlobRateLimiter. -func NewBlobRateLimiter(config *Config) *BlobRateLimiter { +func NewBlobRateLimiter(config *Config, limiterCounter metrics.CountMetric) *BlobRateLimiter { globalGetBlobOpLimiter := rate.NewLimiter( rate.Limit(config.MaxGetBlobOpsPerSecond), config.GetBlobOpsBurstiness) @@ -57,10 +61,16 @@ func (l *BlobRateLimiter) BeginGetBlobOperation(now time.Time) error { defer l.lock.Unlock() if l.operationsInFlight >= l.config.MaxConcurrentGetBlobOps { + if l.limitCounter != nil { + l.limitCounter.Increment(RateLimitLabel{"global concurrency"}) + } return fmt.Errorf("global concurrent request limit %d exceeded for getBlob operations, try again later", l.config.MaxConcurrentGetBlobOps) } if l.opLimiter.TokensAt(now) < 1 { + if l.limitCounter != nil { + l.limitCounter.Increment(RateLimitLabel{"global rate"}) + } return fmt.Errorf("global rate limit %0.1fhz exceeded for getBlob operations, try again later", l.config.MaxGetBlobOpsPerSecond) } @@ -98,6 +108,9 @@ func (l *BlobRateLimiter) RequestGetBlobBandwidth(now time.Time, bytes uint32) e allowed := l.bandwidthLimiter.AllowN(now, int(bytes)) if !allowed { + if l.limitCounter != nil { + l.limitCounter.Increment(RateLimitLabel{"global bandwidth"}) + } return fmt.Errorf("global rate limit %dMib/s exceeded for getBlob bandwidth, try again later", int(l.config.MaxGetBlobBytesPerSecond/1024/1024)) } diff --git a/relay/limiter/blob_rate_limiter_test.go b/relay/limiter/blob_rate_limiter_test.go index 2966b6bea..fc482adb4 100644 --- a/relay/limiter/blob_rate_limiter_test.go +++ b/relay/limiter/blob_rate_limiter_test.go @@ -38,7 +38,7 @@ func TestConcurrentBlobOperations(t *testing.T) { // Make the burstiness limit high enough that we won't be rate limited config.GetBlobOpsBurstiness = concurrencyLimit * 100 - limiter := NewBlobRateLimiter(config) + limiter := NewBlobRateLimiter(config, nil) // time starts at current time, but advances manually afterward now := time.Now() @@ -69,7 +69,7 @@ func TestGetBlobOpRateLimit(t *testing.T) { config.GetBlobOpsBurstiness = int(config.MaxGetBlobOpsPerSecond) + rand.Intn(10) config.MaxConcurrentGetBlobOps = 1 - limiter := NewBlobRateLimiter(config) + limiter := NewBlobRateLimiter(config, nil) // time starts at current time, but advances manually afterward now := time.Now() @@ -129,7 +129,7 @@ func TestGetBlobBandwidthLimit(t *testing.T) { config.MaxGetBlobBytesPerSecond = float64(1024 + rand.Intn(1024*1024)) config.GetBlobBytesBurstiness = int(config.MaxGetBlobBytesPerSecond) + rand.Intn(1024*1024) - limiter := NewBlobRateLimiter(config) + limiter := NewBlobRateLimiter(config, nil) // time starts at current time, but advances manually afterward now := time.Now() diff --git a/relay/mdoc/main.go b/relay/mdoc/main.go index 56c68999f..1a320aa25 100644 --- a/relay/mdoc/main.go +++ b/relay/mdoc/main.go @@ -2,7 +2,7 @@ package main import ( "github.com/Layr-Labs/eigenda/common" - "github.com/Layr-Labs/eigenda/relay/metrics" + "github.com/Layr-Labs/eigenda/relay" ) // main generates documentation for relay metrics. @@ -12,7 +12,7 @@ func main() { panic(err) } - metrics, err := metrics.NewRelayMetrics(logger, 0) + metrics, err := relay.NewRelayMetrics(logger, 0) if err != nil { panic(err) } diff --git a/relay/mdoc/relay-metrics.md b/relay/mdoc/relay-metrics.md index 541909a1e..02f24a340 100644 --- a/relay/mdoc/relay-metrics.md +++ b/relay/mdoc/relay-metrics.md @@ -1,11 +1,24 @@ # Metrics Documentation for namespace 'relay' -This documentation was automatically generated at time `2024-11-27T10:08:20-06:00` +This documentation was automatically generated at time `2024-11-27T10:22:37-06:00` -There are a total of `8` registered metrics. +There are a total of `13` registered metrics. --- +## average_get_blob_data_bytes + +Average data size of requested blobs + +| | | +|---|---| +| **Name** | `average_get_blob_data` | +| **Unit** | `bytes` | +| **Type** | `running average` | +| **Time Window** | `1m0s` | +| **Fully Qualified Name** | `relay_average_get_blob_data_bytes` | +--- + ## average_get_chunks_data_bytes Average data size in a GetChunks request @@ -32,6 +45,58 @@ Average number of keys in a GetChunks request | **Fully Qualified Name** | `relay_average_get_chunks_key_count` | --- +## get_blob_data_latency_ms + +Latency of the GetBlob RPC data retrieval + +| | | +|---|---| +| **Name** | `get_blob_data_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_get_blob_data_latency_ms` | +--- + +## get_blob_latency_ms + +Latency of the GetBlob RPC + +| | | +|---|---| +| **Name** | `get_blob_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_get_blob_latency_ms` | +--- + +## get_blob_metadata_latency_ms + +Latency of the GetBlob RPC metadata retrieval + +| | | +|---|---| +| **Name** | `get_blob_metadata_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_get_blob_metadata_latency_ms` | +--- + +## get_blob_rate_limited_count + +Number of GetBlob RPC rate limited + +| | | +|---|---| +| **Name** | `get_blob_rate_limited` | +| **Unit** | `count` | +| **Labels** | `reason` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_get_blob_rate_limited_count` | +--- + ## get_chunks_auth_failure_count Number of GetChunks RPC authentication failures diff --git a/relay/metrics/relay_metrics.go b/relay/relay_metrics.go similarity index 72% rename from relay/metrics/relay_metrics.go rename to relay/relay_metrics.go index 066e211aa..bd2c23283 100644 --- a/relay/metrics/relay_metrics.go +++ b/relay/relay_metrics.go @@ -1,4 +1,4 @@ -package metrics +package relay import ( "github.com/Layr-Labs/eigenda/common/metrics" @@ -13,6 +13,9 @@ type RelayMetrics struct { metricsServer metrics.Metrics grpcServerOption grpc.ServerOption + // TODO (after cache changes merge): add metrics for cache + + // GetChunks metrics GetChunksLatency metrics.LatencyMetric GetChunksAuthenticationLatency metrics.LatencyMetric GetChunksMetadataLatency metrics.LatencyMetric @@ -21,6 +24,13 @@ type RelayMetrics struct { GetChunksRateLimited metrics.CountMetric GetChunksAverageKeyCount metrics.RunningAverageMetric GetChunksAverageDataSize metrics.RunningAverageMetric + + // GetBlob metrics + GetBlobLatency metrics.LatencyMetric + GetBlobMetadataLatency metrics.LatencyMetric + GetBlobDataLatency metrics.LatencyMetric + GetBlobRateLimited metrics.CountMetric + GetBlobAverageDataSize metrics.RunningAverageMetric } // NewRelayMetrics creates a new RelayMetrics instance, which encapsulates all metrics related to the relay. @@ -112,6 +122,51 @@ func NewRelayMetrics(logger logging.Logger, port int) (*RelayMetrics, error) { return nil, err } + getBlobLatencyMetric, err := server.NewLatencyMetric( + "get_blob_latency", + "Latency of the GetBlob RPC", + nil, + standardQuantiles...) + if err != nil { + return nil, err + } + + getBlobMetadataLatencyMetric, err := server.NewLatencyMetric( + "get_blob_metadata_latency", + "Latency of the GetBlob RPC metadata retrieval", + nil, + standardQuantiles...) + if err != nil { + return nil, err + } + + getBlobDataLatencyMetric, err := server.NewLatencyMetric( + "get_blob_data_latency", + "Latency of the GetBlob RPC data retrieval", + nil, + standardQuantiles...) + if err != nil { + return nil, err + } + + getBlobRateLimited, err := server.NewCountMetric( + "get_blob_rate_limited", + "Number of GetBlob RPC rate limited", + limiter.RateLimitLabel{}) + if err != nil { + return nil, err + } + + getBlobAverageDataSize, err := server.NewRunningAverageMetric( + "average_get_blob_data", + "bytes", + "Average data size of requested blobs", + time.Minute, + nil) + if err != nil { + return nil, err + } + return &RelayMetrics{ metricsServer: server, grpcServerOption: grpcServerOption, @@ -123,6 +178,11 @@ func NewRelayMetrics(logger logging.Logger, port int) (*RelayMetrics, error) { GetChunksRateLimited: getChunksRateLimited, GetChunksAverageKeyCount: getChunksAverageKeyCount, GetChunksAverageDataSize: getChunksAverageDataSize, + GetBlobLatency: getBlobLatencyMetric, + GetBlobMetadataLatency: getBlobMetadataLatencyMetric, + GetBlobDataLatency: getBlobDataLatencyMetric, + GetBlobRateLimited: getBlobRateLimited, + GetBlobAverageDataSize: getBlobAverageDataSize, }, nil } diff --git a/relay/server.go b/relay/server.go index a47a5c7ed..c44d27440 100644 --- a/relay/server.go +++ b/relay/server.go @@ -4,7 +4,6 @@ import ( "context" "errors" "fmt" - "github.com/Layr-Labs/eigenda/relay/metrics" "net" "time" @@ -60,7 +59,7 @@ type Server struct { chainReader core.Reader // metrics encapsulates the metrics for the relay server. - relayMetrics *metrics.RelayMetrics + metrics *RelayMetrics } type Config struct { @@ -191,7 +190,7 @@ func NewServer( } } - relayMetrics, err := metrics.NewRelayMetrics(logger, config.MetricsPort) + metrics, err := NewRelayMetrics(logger, config.MetricsPort) if err != nil { return nil, fmt.Errorf("error creating metrics: %w", err) } @@ -202,15 +201,17 @@ func NewServer( metadataProvider: mp, blobProvider: bp, chunkProvider: cp, - blobRateLimiter: limiter.NewBlobRateLimiter(&config.RateLimits), - chunkRateLimiter: limiter.NewChunkRateLimiter(&config.RateLimits, relayMetrics.GetChunksRateLimited), + blobRateLimiter: limiter.NewBlobRateLimiter(&config.RateLimits, metrics.GetBlobRateLimited), + chunkRateLimiter: limiter.NewChunkRateLimiter(&config.RateLimits, metrics.GetChunksRateLimited), authenticator: authenticator, - relayMetrics: relayMetrics, + metrics: metrics, }, nil } // GetBlob retrieves a blob stored by the relay. func (s *Server) GetBlob(ctx context.Context, request *pb.GetBlobRequest) (*pb.GetBlobReply, error) { + start := time.Now() + if s.config.Timeouts.GetBlobTimeout > 0 { var cancel context.CancelFunc ctx, cancel = context.WithTimeout(ctx, s.config.Timeouts.GetBlobTimeout) @@ -239,6 +240,9 @@ func (s *Server) GetBlob(ctx context.Context, request *pb.GetBlobRequest) (*pb.G return nil, fmt.Errorf("blob not found") } + finishedFetchingMetadata := time.Now() + s.metrics.GetBlobMetadataLatency.ReportLatency(finishedFetchingMetadata.Sub(start)) + err = s.blobRateLimiter.RequestGetBlobBandwidth(time.Now(), metadata.blobSizeBytes) if err != nil { return nil, err @@ -249,10 +253,13 @@ func (s *Server) GetBlob(ctx context.Context, request *pb.GetBlobRequest) (*pb.G return nil, fmt.Errorf("error fetching blob %s: %w", key.Hex(), err) } + s.metrics.GetBlobAverageDataSize.Update(float64(len(data))) + s.metrics.GetBlobDataLatency.ReportLatency(time.Since(finishedFetchingMetadata)) + s.metrics.GetBlobLatency.ReportLatency(time.Since(start)) + reply := &pb.GetBlobReply{ Blob: data, } - return reply, nil } @@ -273,7 +280,7 @@ func (s *Server) GetChunks(ctx context.Context, request *pb.GetChunksRequest) (* return nil, fmt.Errorf( "too many chunk requests provided, max is %d", s.config.MaxKeysPerGetChunksRequest) } - s.relayMetrics.GetChunksAverageKeyCount.Update(float64(len(request.ChunkRequests))) + s.metrics.GetChunksAverageKeyCount.Update(float64(len(request.ChunkRequests))) if s.authenticator != nil { client, ok := peer.FromContext(ctx) @@ -284,14 +291,14 @@ func (s *Server) GetChunks(ctx context.Context, request *pb.GetChunksRequest) (* err := s.authenticator.AuthenticateGetChunksRequest(ctx, clientAddress, request, time.Now()) if err != nil { - s.relayMetrics.GetChunksAuthFailures.Increment() + s.metrics.GetChunksAuthFailures.Increment() return nil, fmt.Errorf("auth failed: %w", err) } } finishedAuthenticating := time.Now() if s.authenticator != nil { - s.relayMetrics.GetChunksAuthenticationLatency.ReportLatency(finishedAuthenticating.Sub(start)) + s.metrics.GetChunksAuthenticationLatency.ReportLatency(finishedAuthenticating.Sub(start)) } clientID := string(request.OperatorId) @@ -314,7 +321,7 @@ func (s *Server) GetChunks(ctx context.Context, request *pb.GetChunksRequest) (* } finishedFetchingMetadata := time.Now() - s.relayMetrics.GetChunksMetadataLatency.ReportLatency(finishedFetchingMetadata.Sub(finishedAuthenticating)) + s.metrics.GetChunksMetadataLatency.ReportLatency(finishedFetchingMetadata.Sub(finishedAuthenticating)) requiredBandwidth, err := computeChunkRequestRequiredBandwidth(request, mMap) if err != nil { @@ -324,7 +331,7 @@ func (s *Server) GetChunks(ctx context.Context, request *pb.GetChunksRequest) (* if err != nil { return nil, err } - s.relayMetrics.GetChunksAverageDataSize.Update(float64(requiredBandwidth)) + s.metrics.GetChunksAverageDataSize.Update(float64(requiredBandwidth)) frames, err := s.chunkProvider.GetFrames(ctx, mMap) if err != nil { @@ -337,8 +344,8 @@ func (s *Server) GetChunks(ctx context.Context, request *pb.GetChunksRequest) (* } finishedFetchingData := time.Now() - s.relayMetrics.GetChunksDataLatency.ReportLatency(finishedFetchingData.Sub(finishedFetchingMetadata)) - s.relayMetrics.GetChunksLatency.ReportLatency(time.Since(start)) + s.metrics.GetChunksDataLatency.ReportLatency(finishedFetchingData.Sub(finishedFetchingMetadata)) + s.metrics.GetChunksLatency.ReportLatency(time.Since(start)) return &pb.GetChunksReply{ Data: bytesToSend, @@ -459,7 +466,7 @@ func computeChunkRequestRequiredBandwidth(request *pb.GetChunksRequest, mMap met // Start starts the server listening for requests. This method will block until the server is stopped. func (s *Server) Start(ctx context.Context) error { - err := s.relayMetrics.Start() + err := s.metrics.Start() if err != nil { return fmt.Errorf("error starting metrics server: %w", err) } @@ -479,7 +486,7 @@ func (s *Server) Start(ctx context.Context) error { opt := grpc.MaxRecvMsgSize(s.config.MaxGRPCMessageSize) - s.grpcServer = grpc.NewServer(opt, s.relayMetrics.GetGRPCServerOption()) + s.grpcServer = grpc.NewServer(opt, s.metrics.GetGRPCServerOption()) reflection.Register(s.grpcServer) pb.RegisterRelayServer(s.grpcServer, s) @@ -521,7 +528,7 @@ func (s *Server) Stop() error { s.grpcServer.Stop() } - err := s.relayMetrics.Stop() + err := s.metrics.Stop() if err != nil { return fmt.Errorf("error stopping metrics server: %w", err) } From 5579a889ed2ad5e77b6953e0250dd524e346bd02 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Wed, 27 Nov 2024 10:26:15 -0600 Subject: [PATCH 09/30] Cleanup. Signed-off-by: Cody Littley --- relay/relay_metrics.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/relay/relay_metrics.go b/relay/relay_metrics.go index bd2c23283..5aac47bfc 100644 --- a/relay/relay_metrics.go +++ b/relay/relay_metrics.go @@ -13,7 +13,7 @@ type RelayMetrics struct { metricsServer metrics.Metrics grpcServerOption grpc.ServerOption - // TODO (after cache changes merge): add metrics for cache + // TODO (cody-littley): after cache changes merge, add metrics for cache // GetChunks metrics GetChunksLatency metrics.LatencyMetric From 2b84f218b954b2c60ba9a7be4a43a20488f040d4 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Wed, 27 Nov 2024 11:04:02 -0600 Subject: [PATCH 10/30] Cleanup test Signed-off-by: Cody Littley --- relay/server_test.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/relay/server_test.go b/relay/server_test.go index e5990cc84..462ea7e9f 100644 --- a/relay/server_test.go +++ b/relay/server_test.go @@ -2,7 +2,6 @@ package relay import ( "context" - "fmt" "math/rand" "testing" "time" @@ -396,9 +395,8 @@ func TestReadWriteChunks(t *testing.T) { expectedData := make(map[v2.BlobKey][]*encoding.Frame) fragmentInfoMap := make(map[v2.BlobKey]*encoding.FragmentInfo) - blobCount := 100 // TODO revert this to 10 + blobCount := 10 for i := 0; i < blobCount; i++ { - fmt.Printf("blob %d\n", i) // TODO remove this header, _, chunks := randomBlobChunks(t) blobKey, err := header.BlobKey() From fb0cad5b58bb1e511cec7a86b94313a7b15ce7c4 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Wed, 27 Nov 2024 11:11:24 -0600 Subject: [PATCH 11/30] Add locking for running average metric. Signed-off-by: Cody Littley --- common/metrics/running_average_metric.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/common/metrics/running_average_metric.go b/common/metrics/running_average_metric.go index 8456644e0..f0e263e92 100644 --- a/common/metrics/running_average_metric.go +++ b/common/metrics/running_average_metric.go @@ -5,6 +5,7 @@ import ( "github.com/Layr-Labs/eigensdk-go/logging" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" + "sync" "time" ) @@ -35,6 +36,9 @@ type runningAverageMetric struct { // timeWindow is the time window used to calculate the running average. timeWindow time.Duration + + // lock is used to provide thread safety for the running average calculator. + lock sync.Mutex } // newRunningAverageMetric creates a new RunningAverageMetric instance. @@ -104,7 +108,9 @@ func (m *runningAverageMetric) Update(value float64, label ...any) { m.logger.Errorf("error extracting values from label: %v", err) } + m.lock.Lock() average := m.runningAverage.Update(time.Now(), value) + m.lock.Unlock() m.vec.WithLabelValues(values...).Set(average) } From dfd2925815196cbca5d806d90be9f93aa3e8505f Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Wed, 27 Nov 2024 12:14:17 -0600 Subject: [PATCH 12/30] Add cache metrics. Signed-off-by: Cody Littley --- go.mod | 1 - relay/blob_provider.go | 11 +- relay/blob_provider_test.go | 6 +- relay/cache/cache_accessor.go | 77 +++++++- relay/cache/cache_accessor_metrics.go | 98 +++++++++ relay/cache/cache_accessor_test.go | 40 +--- relay/cache/fifo-cache.go | 20 +- relay/cache/fifo_cache_test.go | 6 +- relay/chunk_provider.go | 11 +- relay/chunk_provider_test.go | 6 +- relay/mdoc/relay-metrics.md | 262 ++++++++++++++++++++++++- relay/metadata_provider.go | 14 +- relay/metadata_provider_test.go | 15 +- relay/{relay_metrics.go => metrics.go} | 24 ++- relay/server.go | 19 +- 15 files changed, 527 insertions(+), 83 deletions(-) create mode 100644 relay/cache/cache_accessor_metrics.go rename relay/{relay_metrics.go => metrics.go} (89%) diff --git a/go.mod b/go.mod index bab065561..dc262e397 100644 --- a/go.mod +++ b/go.mod @@ -86,7 +86,6 @@ require ( github.com/docker/docker v25.0.5+incompatible // indirect github.com/docker/go-connections v0.5.0 // indirect github.com/docker/go-units v0.5.0 // indirect - github.com/emirpasic/gods v1.18.1 // indirect github.com/ethereum/c-kzg-4844 v1.0.0 // indirect github.com/ethereum/go-verkle v0.1.1-0.20240306133620-7d920df305f0 // indirect github.com/gabriel-vasile/mimetype v1.4.2 // indirect diff --git a/relay/blob_provider.go b/relay/blob_provider.go index 70cc31066..e8e84e1d9 100644 --- a/relay/blob_provider.go +++ b/relay/blob_provider.go @@ -33,7 +33,8 @@ func newBlobProvider( blobStore *blobstore.BlobStore, blobCacheSize uint64, maxIOConcurrency int, - fetchTimeout time.Duration) (*blobProvider, error) { + fetchTimeout time.Duration, + metrics *cache.CacheAccessorMetrics) (*blobProvider, error) { server := &blobProvider{ ctx: ctx, @@ -42,9 +43,13 @@ func newBlobProvider( fetchTimeout: fetchTimeout, } - c := cache.NewFIFOCache[v2.BlobKey, []byte](blobCacheSize, computeBlobCacheWeight) + cacheAccessor, err := cache.NewCacheAccessor[v2.BlobKey, []byte]( + computeBlobCacheWeight, + blobCacheSize, + maxIOConcurrency, + server.fetchBlob, + metrics) - cacheAccessor, err := cache.NewCacheAccessor[v2.BlobKey, []byte](c, maxIOConcurrency, server.fetchBlob) if err != nil { return nil, fmt.Errorf("error creating blob cache: %w", err) } diff --git a/relay/blob_provider_test.go b/relay/blob_provider_test.go index 22368a5d5..8ac2e6657 100644 --- a/relay/blob_provider_test.go +++ b/relay/blob_provider_test.go @@ -41,7 +41,8 @@ func TestReadWrite(t *testing.T) { blobStore, 1024*1024*32, 32, - 10*time.Second) + 10*time.Second, + nil) require.NoError(t, err) // Read the blobs back. @@ -78,7 +79,8 @@ func TestNonExistentBlob(t *testing.T) { blobStore, 1024*1024*32, 32, - 10*time.Second) + 10*time.Second, + nil) require.NoError(t, err) for i := 0; i < 10; i++ { diff --git a/relay/cache/cache_accessor.go b/relay/cache/cache_accessor.go index a6389538b..bfebf1c6b 100644 --- a/relay/cache/cache_accessor.go +++ b/relay/cache/cache_accessor.go @@ -4,6 +4,7 @@ import ( "context" "golang.org/x/sync/semaphore" "sync" + "time" ) // CacheAccessor is an interface for accessing a resource that is cached. It assumes that cache misses @@ -47,6 +48,9 @@ type cacheAccessor[K comparable, V any] struct { // cache is the underlying cache that this wrapper manages. cache Cache[K, V] + // weightCalculator is the function used to calculate the weight of a key-value pair. + calculator WeightCalculator[K, V] + // concurrencyLimiter is a channel used to limit the number of concurrent lookups that can be in progress. concurrencyLimiter chan struct{} @@ -55,17 +59,29 @@ type cacheAccessor[K comparable, V any] struct { // accessor is the function used to fetch values that are not in the cache. accessor Accessor[K, V] + + // insertionTimes is a map of keys to the time they were inserted into the cache. Used to calculate the average + // lifespan of items in the cache. + insertionTimes map[K]time.Time + + // metrics is used to record metrics about the cache accessor's performance. + metrics *CacheAccessorMetrics } -// NewCacheAccessor creates a new CacheAccessor. The cacheSize parameter specifies the maximum number of items -// that can be stored in the cache. The concurrencyLimit parameter specifies the maximum number of concurrent -// lookups that can be in progress at any given time. If a greater number of lookups are requested, the excess -// lookups will block until a lookup completes. If concurrencyLimit is zero, then no limits are imposed. The accessor -// parameter is the function used to fetch values that are not in the cache. +// NewCacheAccessor creates a new CacheAccessor. +// +// The concurrencyLimit parameter specifies the maximum number of concurrent lookups that can be in progress at any +// given time. If a greater number of lookups are requested, the excess lookups will block until a lookup completes. +// If concurrencyLimit is zero, then no limits are imposed. The accessor parameter is the function used to fetch values that are not in the cache. +// +// If metrics is not nil, it will be used to record metrics about the cache accessor's performance. +// If nil, no metrics will be recorded. func NewCacheAccessor[K comparable, V any]( - cache Cache[K, V], + calculator WeightCalculator[K, V], + maxWeight uint64, concurrencyLimit int, - accessor Accessor[K, V]) (CacheAccessor[K, V], error) { + accessor Accessor[K, V], + metrics *CacheAccessorMetrics) (CacheAccessor[K, V], error) { lookupsInProgress := make(map[K]*accessResult[V]) @@ -74,11 +90,30 @@ func NewCacheAccessor[K comparable, V any]( concurrencyLimiter = make(chan struct{}, concurrencyLimit) } + insertionTimes := make(map[K]time.Time) + var evictionHandler func(K, V) + if metrics != nil { + // If metrics are enabled, track the amount of time each item spends in the cache. + // Thread safety is provided by the cacheLock. + evictionHandler = func(key K, _ V) { + if insertionTime, ok := insertionTimes[key]; ok { + lifespan := time.Since(insertionTime).Milliseconds() + metrics.averageLifespan.Update(float64(lifespan)) + delete(insertionTimes, key) + } + } + } + + cache := NewFIFOCache(maxWeight, calculator, evictionHandler) + return &cacheAccessor[K, V]{ cache: cache, + calculator: calculator, concurrencyLimiter: concurrencyLimiter, accessor: accessor, lookupsInProgress: lookupsInProgress, + insertionTimes: insertionTimes, + metrics: metrics, }, nil } @@ -97,6 +132,10 @@ func (c *cacheAccessor[K, V]) Get(ctx context.Context, key K) (V, error) { v, ok := c.cache.Get(key) if ok { c.cacheLock.Unlock() + + if c.metrics != nil { + c.metrics.cacheHits.Increment() + } return v, nil } @@ -109,6 +148,10 @@ func (c *cacheAccessor[K, V]) Get(ctx context.Context, key K) (V, error) { c.cacheLock.Unlock() + if c.metrics != nil { + c.metrics.cacheMisses.Increment() + } + if alreadyLoading { // The result is being fetched on another goroutine. Wait for it to finish. return c.waitForResult(ctx, result) @@ -150,11 +193,31 @@ func (c *cacheAccessor[K, V]) fetchResult(ctx context.Context, key K, result *ac <-c.concurrencyLimiter } + if c.metrics != nil { + start := time.Now() + defer func() { + c.metrics.cacheMissLatency.ReportLatency(time.Since(start)) + }() + } + c.cacheLock.Lock() // Update the cache if the fetch was successful. if err == nil { c.cache.Put(key, value) + + if c.metrics != nil { + c.insertionTimes[key] = time.Now() + size := c.cache.Size() + weight := c.cache.Weight() + c.metrics.size.Set(float64(size)) + c.metrics.weight.Set(float64(weight)) + var averageWeight float64 + if size > 0 { + averageWeight = float64(weight) / float64(size) + } + c.metrics.averageWeight.Set(averageWeight) + } } // Provide the result to all other goroutines that may be waiting for it. diff --git a/relay/cache/cache_accessor_metrics.go b/relay/cache/cache_accessor_metrics.go new file mode 100644 index 000000000..e4a4d50fc --- /dev/null +++ b/relay/cache/cache_accessor_metrics.go @@ -0,0 +1,98 @@ +package cache + +import ( + "fmt" + "github.com/Layr-Labs/eigenda/common/metrics" + "time" +) + +// CacheAccessorMetrics provides metrics for a CacheAccessor. +type CacheAccessorMetrics struct { + cacheHits metrics.CountMetric + cacheMisses metrics.CountMetric + size metrics.GaugeMetric + weight metrics.GaugeMetric + averageWeight metrics.GaugeMetric + averageLifespan metrics.RunningAverageMetric + cacheMissLatency metrics.LatencyMetric +} + +// NewCacheAccessorMetrics creates a new CacheAccessorMetrics. +func NewCacheAccessorMetrics( + server metrics.Metrics, + cacheName string) (*CacheAccessorMetrics, error) { + + cacheHits, err := server.NewCountMetric( + fmt.Sprintf("%s_cache_hit", cacheName), + fmt.Sprintf("Number of cache hits in the %s cache", cacheName), + nil) + if err != nil { + return nil, err + } + + cacheMisses, err := server.NewCountMetric( + fmt.Sprintf("%s_cache_miss", cacheName), + fmt.Sprintf("Number of cache misses in the %s cache", cacheName), + nil) + if err != nil { + return nil, err + } + + size, err := server.NewGaugeMetric( + fmt.Sprintf("%s_cache", cacheName), + "size", + fmt.Sprintf("Number of items in the %s cache", cacheName), + nil) + if err != nil { + return nil, err + } + + weight, err := server.NewGaugeMetric( + fmt.Sprintf("%s_cache", cacheName), + "weight", + fmt.Sprintf("Total weight of items in the %s cache", cacheName), + nil) + if err != nil { + return nil, err + } + + averageWeight, err := server.NewGaugeMetric( + fmt.Sprintf("%s_cache_average", cacheName), + "weight", + fmt.Sprintf("Average weight of items currently in the %s cache", cacheName), + nil) + if err != nil { + return nil, err + } + + averageLifespan, err := server.NewRunningAverageMetric( + fmt.Sprintf("%s_cache_average_lifespan", cacheName), + "ms", + fmt.Sprintf("Average time an item remains in the %s cache before being evicted.", cacheName), + time.Minute, + nil) + if err != nil { + return nil, err + } + + cacheMissLatency, err := server.NewLatencyMetric( + fmt.Sprintf("%s_cache_miss_latency", cacheName), + fmt.Sprintf("Latency of cache misses in the %s cache", cacheName), + nil, + &metrics.Quantile{Quantile: 0.5, Error: 0.05}, + &metrics.Quantile{Quantile: 0.9, Error: 0.05}, + &metrics.Quantile{Quantile: 0.99, Error: 0.05}) + if err != nil { + return nil, err + } + + return &CacheAccessorMetrics{ + cacheHits: cacheHits, + cacheMisses: cacheMisses, + size: size, + weight: weight, + averageWeight: averageWeight, + averageLifespan: averageLifespan, + cacheMissLatency: cacheMissLatency, + }, nil +} diff --git a/relay/cache/cache_accessor_test.go b/relay/cache/cache_accessor_test.go index 0f2ac501d..16f33d0b0 100644 --- a/relay/cache/cache_accessor_test.go +++ b/relay/cache/cache_accessor_test.go @@ -32,11 +32,8 @@ func TestRandomOperationsSingleThread(t *testing.T) { return &str, nil } cacheSize := rand.Intn(dataSize) + 1 - c := NewFIFOCache[int, *string](uint64(cacheSize), func(key int, value *string) uint64 { - return 1 - }) - ca, err := NewCacheAccessor[int, *string](c, 0, accessor) + ca, err := NewCacheAccessor[int, *string](nil, uint64(cacheSize), 0, accessor, nil) require.NoError(t, err) for i := 0; i < dataSize; i++ { @@ -83,11 +80,7 @@ func TestCacheMisses(t *testing.T) { return &str, nil } - c := NewFIFOCache[int, *string](uint64(cacheSize), func(key int, value *string) uint64 { - return 1 - }) - - ca, err := NewCacheAccessor[int, *string](c, 0, accessor) + ca, err := NewCacheAccessor[int, *string](nil, uint64(cacheSize), 0, accessor, nil) require.NoError(t, err) // Get the first cacheSize keys. This should fill the cache. @@ -150,11 +143,7 @@ func ParallelAccessTest(t *testing.T, sleepEnabled bool) { } cacheSize := rand.Intn(dataSize) + 1 - c := NewFIFOCache[int, *string](uint64(cacheSize), func(key int, value *string) uint64 { - return 1 - }) - - ca, err := NewCacheAccessor[int, *string](c, 0, accessor) + ca, err := NewCacheAccessor[int, *string](nil, uint64(cacheSize), 0, accessor, nil) require.NoError(t, err) // Lock the accessor. This will cause all cache misses to block. @@ -223,11 +212,7 @@ func TestParallelAccessWithError(t *testing.T) { } cacheSize := 100 - c := NewFIFOCache[int, *string](uint64(cacheSize), func(key int, value *string) uint64 { - return 1 - }) - - ca, err := NewCacheAccessor[int, *string](c, 0, accessor) + ca, err := NewCacheAccessor[int, *string](nil, uint64(cacheSize), 0, accessor, nil) require.NoError(t, err) // Lock the accessor. This will cause all cache misses to block. @@ -299,11 +284,8 @@ func TestConcurrencyLimiter(t *testing.T) { } cacheSize := 100 - c := NewFIFOCache[int, *string](uint64(cacheSize), func(key int, value *string) uint64 { - return 1 - }) - ca, err := NewCacheAccessor[int, *string](c, maxConcurrency, accessor) + ca, err := NewCacheAccessor[int, *string](nil, uint64(cacheSize), 0, accessor, nil) require.NoError(t, err) wg := sync.WaitGroup{} @@ -357,11 +339,7 @@ func TestOriginalRequesterTimesOut(t *testing.T) { } cacheSize := rand.Intn(dataSize) + 1 - c := NewFIFOCache[int, *string](uint64(cacheSize), func(key int, value *string) uint64 { - return 1 - }) - - ca, err := NewCacheAccessor[int, *string](c, 0, accessor) + ca, err := NewCacheAccessor[int, *string](nil, uint64(cacheSize), 0, accessor, nil) require.NoError(t, err) // Lock the accessor. This will cause all cache misses to block. @@ -449,11 +427,7 @@ func TestSecondaryRequesterTimesOut(t *testing.T) { } cacheSize := rand.Intn(dataSize) + 1 - c := NewFIFOCache[int, *string](uint64(cacheSize), func(key int, value *string) uint64 { - return 1 - }) - - ca, err := NewCacheAccessor[int, *string](c, 0, accessor) + ca, err := NewCacheAccessor[int, *string](nil, uint64(cacheSize), 0, accessor, nil) require.NoError(t, err) // Lock the accessor. This will cause all cache misses to block. diff --git a/relay/cache/fifo-cache.go b/relay/cache/fifo-cache.go index 1c2e7c6ab..d6e44d3d4 100644 --- a/relay/cache/fifo-cache.go +++ b/relay/cache/fifo-cache.go @@ -16,15 +16,30 @@ type FIFOCache[K comparable, V any] struct { maxWeight uint64 data map[K]V expirationQueue queues.Queue + evictionHandler func(K, V) } -// NewFIFOCache creates a new FIFOCache. -func NewFIFOCache[K comparable, V any](maxWeight uint64, calculator WeightCalculator[K, V]) *FIFOCache[K, V] { +// NewFIFOCache creates a new FIFOCache. If the calculator is nil, the weight of each key-value pair will be 1. +// If the evictionHandler is nil it is ignored. +func NewFIFOCache[K comparable, V any]( + maxWeight uint64, + calculator WeightCalculator[K, V], + evictionHandler func(K, V)) *FIFOCache[K, V] { + + if calculator == nil { + calculator = func(K, V) uint64 { return 1 } + } + + if evictionHandler == nil { + evictionHandler = func(K, V) {} + } + return &FIFOCache[K, V]{ maxWeight: maxWeight, data: make(map[K]V), weightCalculator: calculator, expirationQueue: linkedlistqueue.New(), + evictionHandler: evictionHandler, } } @@ -61,6 +76,7 @@ func (f *FIFOCache[K, V]) Put(key K, value V) { weightToEvict := f.weightCalculator(keyToEvict, f.data[keyToEvict]) delete(f.data, keyToEvict) f.currentWeight -= weightToEvict + f.evictionHandler(keyToEvict, f.data[keyToEvict]) } } diff --git a/relay/cache/fifo_cache_test.go b/relay/cache/fifo_cache_test.go index da4de5ad1..3709c1911 100644 --- a/relay/cache/fifo_cache_test.go +++ b/relay/cache/fifo_cache_test.go @@ -11,9 +11,7 @@ func TestExpirationOrder(t *testing.T) { tu.InitializeRandom() maxWeight := uint64(10 + rand.Intn(10)) - c := NewFIFOCache[int, int](maxWeight, func(key int, value int) uint64 { - return 1 - }) + c := NewFIFOCache[int, int](maxWeight, nil, nil) require.Equal(t, uint64(0), c.Weight()) require.Equal(t, 0, c.Size()) @@ -85,7 +83,7 @@ func TestWeightedValues(t *testing.T) { return uint64(key) } - c := NewFIFOCache[int, int](maxWeight, weightCalculator) + c := NewFIFOCache[int, int](maxWeight, weightCalculator, nil) expectedValues := make(map[int]int) diff --git a/relay/chunk_provider.go b/relay/chunk_provider.go index 5bc292673..256607459 100644 --- a/relay/chunk_provider.go +++ b/relay/chunk_provider.go @@ -50,7 +50,8 @@ func newChunkProvider( cacheSize uint64, maxIOConcurrency int, proofFetchTimeout time.Duration, - coefficientFetchTimeout time.Duration) (*chunkProvider, error) { + coefficientFetchTimeout time.Duration, + metrics *cache.CacheAccessorMetrics) (*chunkProvider, error) { server := &chunkProvider{ ctx: ctx, @@ -60,12 +61,12 @@ func newChunkProvider( coefficientFetchTimeout: coefficientFetchTimeout, } - c := cache.NewFIFOCache[blobKeyWithMetadata, []*encoding.Frame](cacheSize, computeFramesCacheWeight) - cacheAccessor, err := cache.NewCacheAccessor[blobKeyWithMetadata, []*encoding.Frame]( - c, + computeFramesCacheWeight, + cacheSize, maxIOConcurrency, - server.fetchFrames) + server.fetchFrames, + metrics) if err != nil { return nil, err } diff --git a/relay/chunk_provider_test.go b/relay/chunk_provider_test.go index 06ec215b8..99a85345d 100644 --- a/relay/chunk_provider_test.go +++ b/relay/chunk_provider_test.go @@ -52,7 +52,8 @@ func TestFetchingIndividualBlobs(t *testing.T) { 1024*1024*32, 32, 10*time.Second, - 10*time.Second) + 10*time.Second, + nil) require.NoError(t, err) // Read it back. @@ -139,7 +140,8 @@ func TestFetchingBatchedBlobs(t *testing.T) { 1024*1024*32, 32, 10*time.Second, - 10*time.Second) + 10*time.Second, + nil) require.NoError(t, err) // Read it back. diff --git a/relay/mdoc/relay-metrics.md b/relay/mdoc/relay-metrics.md index 02f24a340..46ea716b1 100644 --- a/relay/mdoc/relay-metrics.md +++ b/relay/mdoc/relay-metrics.md @@ -1,8 +1,8 @@ # Metrics Documentation for namespace 'relay' -This documentation was automatically generated at time `2024-11-27T10:22:37-06:00` +This documentation was automatically generated at time `2024-11-27T12:13:08-06:00` -There are a total of `13` registered metrics. +There are a total of `34` registered metrics. --- @@ -45,6 +45,178 @@ Average number of keys in a GetChunks request | **Fully Qualified Name** | `relay_average_get_chunks_key_count` | --- +## blob_cache_size + +Number of items in the blob cache + +| | | +|---|---| +| **Name** | `blob_cache` | +| **Unit** | `size` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_blob_cache_size` | +--- + +## blob_cache_weight + +Total weight of items in the blob cache + +| | | +|---|---| +| **Name** | `blob_cache` | +| **Unit** | `weight` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_blob_cache_weight` | +--- + +## blob_cache_average_weight + +Average weight of items currently in the blob cache + +| | | +|---|---| +| **Name** | `blob_cache_average` | +| **Unit** | `weight` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_blob_cache_average_weight` | +--- + +## blob_cache_average_lifespan_ms + +Average time an item remains in the blob cache before being evicted. + +| | | +|---|---| +| **Name** | `blob_cache_average_lifespan` | +| **Unit** | `ms` | +| **Type** | `running average` | +| **Time Window** | `1m0s` | +| **Fully Qualified Name** | `relay_blob_cache_average_lifespan_ms` | +--- + +## blob_cache_hit_count + +Number of cache hits in the blob cache + +| | | +|---|---| +| **Name** | `blob_cache_hit` | +| **Unit** | `count` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_blob_cache_hit_count` | +--- + +## blob_cache_miss_count + +Number of cache misses in the blob cache + +| | | +|---|---| +| **Name** | `blob_cache_miss` | +| **Unit** | `count` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_blob_cache_miss_count` | +--- + +## blob_cache_miss_latency_ms + +Latency of cache misses in the blob cache + +| | | +|---|---| +| **Name** | `blob_cache_miss_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_blob_cache_miss_latency_ms` | +--- + +## chunk_cache_size + +Number of items in the chunk cache + +| | | +|---|---| +| **Name** | `chunk_cache` | +| **Unit** | `size` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_chunk_cache_size` | +--- + +## chunk_cache_weight + +Total weight of items in the chunk cache + +| | | +|---|---| +| **Name** | `chunk_cache` | +| **Unit** | `weight` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_chunk_cache_weight` | +--- + +## chunk_cache_average_weight + +Average weight of items currently in the chunk cache + +| | | +|---|---| +| **Name** | `chunk_cache_average` | +| **Unit** | `weight` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_chunk_cache_average_weight` | +--- + +## chunk_cache_average_lifespan_ms + +Average time an item remains in the chunk cache before being evicted. + +| | | +|---|---| +| **Name** | `chunk_cache_average_lifespan` | +| **Unit** | `ms` | +| **Type** | `running average` | +| **Time Window** | `1m0s` | +| **Fully Qualified Name** | `relay_chunk_cache_average_lifespan_ms` | +--- + +## chunk_cache_hit_count + +Number of cache hits in the chunk cache + +| | | +|---|---| +| **Name** | `chunk_cache_hit` | +| **Unit** | `count` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_chunk_cache_hit_count` | +--- + +## chunk_cache_miss_count + +Number of cache misses in the chunk cache + +| | | +|---|---| +| **Name** | `chunk_cache_miss` | +| **Unit** | `count` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_chunk_cache_miss_count` | +--- + +## chunk_cache_miss_latency_ms + +Latency of cache misses in the chunk cache + +| | | +|---|---| +| **Name** | `chunk_cache_miss_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_chunk_cache_miss_latency_ms` | +--- + ## get_blob_data_latency_ms Latency of the GetBlob RPC data retrieval @@ -172,3 +344,89 @@ Number of GetChunks RPC rate limited | **Labels** | `reason` | | **Type** | `counter` | | **Fully Qualified Name** | `relay_get_chunks_rate_limited_count` | +--- + +## metadata_cache_size + +Number of items in the metadata cache + +| | | +|---|---| +| **Name** | `metadata_cache` | +| **Unit** | `size` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_metadata_cache_size` | +--- + +## metadata_cache_weight + +Total weight of items in the metadata cache + +| | | +|---|---| +| **Name** | `metadata_cache` | +| **Unit** | `weight` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_metadata_cache_weight` | +--- + +## metadata_cache_average_weight + +Average weight of items currently in the metadata cache + +| | | +|---|---| +| **Name** | `metadata_cache_average` | +| **Unit** | `weight` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_metadata_cache_average_weight` | +--- + +## metadata_cache_average_lifespan_ms + +Average time an item remains in the metadata cache before being evicted. + +| | | +|---|---| +| **Name** | `metadata_cache_average_lifespan` | +| **Unit** | `ms` | +| **Type** | `running average` | +| **Time Window** | `1m0s` | +| **Fully Qualified Name** | `relay_metadata_cache_average_lifespan_ms` | +--- + +## metadata_cache_hit_count + +Number of cache hits in the metadata cache + +| | | +|---|---| +| **Name** | `metadata_cache_hit` | +| **Unit** | `count` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_metadata_cache_hit_count` | +--- + +## metadata_cache_miss_count + +Number of cache misses in the metadata cache + +| | | +|---|---| +| **Name** | `metadata_cache_miss` | +| **Unit** | `count` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_metadata_cache_miss_count` | +--- + +## metadata_cache_miss_latency_ms + +Latency of cache misses in the metadata cache + +| | | +|---|---| +| **Name** | `metadata_cache_miss_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_metadata_cache_miss_latency_ms` | diff --git a/relay/metadata_provider.go b/relay/metadata_provider.go index e1f188bb9..9bdb70964 100644 --- a/relay/metadata_provider.go +++ b/relay/metadata_provider.go @@ -58,7 +58,8 @@ func newMetadataProvider( maxIOConcurrency int, relayIDs []v2.RelayKey, fetchTimeout time.Duration, - blobParamsMap *v2.BlobVersionParameterMap) (*metadataProvider, error) { + blobParamsMap *v2.BlobVersionParameterMap, + metrics *cache.CacheAccessorMetrics) (*metadataProvider, error) { relayIDSet := make(map[v2.RelayKey]struct{}, len(relayIDs)) for _, id := range relayIDs { @@ -74,15 +75,12 @@ func newMetadataProvider( } server.blobParamsMap.Store(blobParamsMap) - c := cache.NewFIFOCache[v2.BlobKey, *blobMetadata](uint64(metadataCacheSize), - func(key v2.BlobKey, value *blobMetadata) uint64 { - return uint64(1) - }) - metadataCache, err := cache.NewCacheAccessor[v2.BlobKey, *blobMetadata]( - c, + nil, + uint64(metadataCacheSize), maxIOConcurrency, - server.fetchMetadata) + server.fetchMetadata, + metrics) if err != nil { return nil, fmt.Errorf("error creating metadata cache: %w", err) } diff --git a/relay/metadata_provider_test.go b/relay/metadata_provider_test.go index b48e157ec..093cfd9f6 100644 --- a/relay/metadata_provider_test.go +++ b/relay/metadata_provider_test.go @@ -32,7 +32,8 @@ func TestGetNonExistentBlob(t *testing.T) { 32, nil, 10*time.Second, - v2.NewBlobVersionParameterMap(mockBlobParamsMap())) + v2.NewBlobVersionParameterMap(mockBlobParamsMap()), + nil) require.NoError(t, err) // Try to fetch a non-existent blobs @@ -98,7 +99,8 @@ func TestFetchingIndividualMetadata(t *testing.T) { 32, nil, 10*time.Second, - v2.NewBlobVersionParameterMap(mockBlobParamsMap())) + v2.NewBlobVersionParameterMap(mockBlobParamsMap()), + nil) require.NoError(t, err) @@ -183,7 +185,8 @@ func TestBatchedFetch(t *testing.T) { 32, nil, 10*time.Second, - v2.NewBlobVersionParameterMap(mockBlobParamsMap())) + v2.NewBlobVersionParameterMap(mockBlobParamsMap()), + nil) require.NoError(t, err) // Each iteration, choose a random subset of the keys to fetch @@ -289,7 +292,8 @@ func TestIndividualFetchWithSharding(t *testing.T) { 32, shardList, 10*time.Second, - v2.NewBlobVersionParameterMap(mockBlobParamsMap())) + v2.NewBlobVersionParameterMap(mockBlobParamsMap()), + nil) require.NoError(t, err) // Fetch the metadata from the server. @@ -421,7 +425,8 @@ func TestBatchedFetchWithSharding(t *testing.T) { 32, shardList, 10*time.Second, - v2.NewBlobVersionParameterMap(mockBlobParamsMap())) + v2.NewBlobVersionParameterMap(mockBlobParamsMap()), + nil) require.NoError(t, err) // Each iteration, choose two random keys to fetch. There will be a 25% chance that both blobs map to valid shards. diff --git a/relay/relay_metrics.go b/relay/metrics.go similarity index 89% rename from relay/relay_metrics.go rename to relay/metrics.go index 5aac47bfc..ebf50e3f1 100644 --- a/relay/relay_metrics.go +++ b/relay/metrics.go @@ -2,6 +2,7 @@ package relay import ( "github.com/Layr-Labs/eigenda/common/metrics" + "github.com/Layr-Labs/eigenda/relay/cache" "github.com/Layr-Labs/eigenda/relay/limiter" "github.com/Layr-Labs/eigensdk-go/logging" grpcprom "github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus" @@ -13,7 +14,10 @@ type RelayMetrics struct { metricsServer metrics.Metrics grpcServerOption grpc.ServerOption - // TODO (cody-littley): after cache changes merge, add metrics for cache + // Cache metrics + MetadataCacheMetrics *cache.CacheAccessorMetrics + ChunkCacheMetrics *cache.CacheAccessorMetrics + BlobCacheMetrics *cache.CacheAccessorMetrics // GetChunks metrics GetChunksLatency metrics.LatencyMetric @@ -50,6 +54,21 @@ func NewRelayMetrics(logger logging.Logger, port int) (*RelayMetrics, error) { metrics.NewQuantile(0.99), } + metadataCacheMetrics, err := cache.NewCacheAccessorMetrics(server, "metadata") + if err != nil { + return nil, err + } + + chunkCacheMetrics, err := cache.NewCacheAccessorMetrics(server, "chunk") + if err != nil { + return nil, err + } + + blobCacheMetrics, err := cache.NewCacheAccessorMetrics(server, "blob") + if err != nil { + return nil, err + } + getChunksLatencyMetric, err := server.NewLatencyMetric( "get_chunks_latency", "Latency of the GetChunks RPC", @@ -169,6 +188,9 @@ func NewRelayMetrics(logger logging.Logger, port int) (*RelayMetrics, error) { return &RelayMetrics{ metricsServer: server, + MetadataCacheMetrics: metadataCacheMetrics, + ChunkCacheMetrics: chunkCacheMetrics, + BlobCacheMetrics: blobCacheMetrics, grpcServerOption: grpcServerOption, GetChunksLatency: getChunksLatencyMetric, GetChunksAuthenticationLatency: getChunksAuthenticationLatencyMetric, diff --git a/relay/server.go b/relay/server.go index 78703ca2f..61d920dfe 100644 --- a/relay/server.go +++ b/relay/server.go @@ -141,6 +141,11 @@ func NewServer( return nil, fmt.Errorf("error fetching blob params: %w", err) } + metrics, err := NewRelayMetrics(logger, config.MetricsPort) + if err != nil { + return nil, fmt.Errorf("error creating metrics: %w", err) + } + mp, err := newMetadataProvider( ctx, logger, @@ -149,7 +154,8 @@ func NewServer( config.MetadataMaxConcurrency, config.RelayIDs, config.Timeouts.InternalGetMetadataTimeout, - v2.NewBlobVersionParameterMap(blobParams)) + v2.NewBlobVersionParameterMap(blobParams), + metrics.MetadataCacheMetrics) if err != nil { return nil, fmt.Errorf("error creating metadata provider: %w", err) @@ -161,7 +167,8 @@ func NewServer( blobStore, config.BlobCacheBytes, config.BlobMaxConcurrency, - config.Timeouts.InternalGetBlobTimeout) + config.Timeouts.InternalGetBlobTimeout, + metrics.BlobCacheMetrics) if err != nil { return nil, fmt.Errorf("error creating blob provider: %w", err) } @@ -173,7 +180,8 @@ func NewServer( config.ChunkCacheSize, config.ChunkMaxConcurrency, config.Timeouts.InternalGetProofsTimeout, - config.Timeouts.InternalGetCoefficientsTimeout) + config.Timeouts.InternalGetCoefficientsTimeout, + metrics.ChunkCacheMetrics) if err != nil { return nil, fmt.Errorf("error creating chunk provider: %w", err) } @@ -190,11 +198,6 @@ func NewServer( } } - metrics, err := NewRelayMetrics(logger, config.MetricsPort) - if err != nil { - return nil, fmt.Errorf("error creating metrics: %w", err) - } - return &Server{ config: config, logger: logger, From 24f5f5dbefe9d0439cea56cf8e17e39d7e945ee9 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Wed, 27 Nov 2024 12:22:07 -0600 Subject: [PATCH 13/30] Fix test bug Signed-off-by: Cody Littley --- relay/cache/cache_accessor_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/relay/cache/cache_accessor_test.go b/relay/cache/cache_accessor_test.go index 16f33d0b0..9f431551f 100644 --- a/relay/cache/cache_accessor_test.go +++ b/relay/cache/cache_accessor_test.go @@ -285,7 +285,7 @@ func TestConcurrencyLimiter(t *testing.T) { cacheSize := 100 - ca, err := NewCacheAccessor[int, *string](nil, uint64(cacheSize), 0, accessor, nil) + ca, err := NewCacheAccessor[int, *string](nil, uint64(cacheSize), maxConcurrency, accessor, nil) require.NoError(t, err) wg := sync.WaitGroup{} From c3adb700b87c975219af4267127944bdf087e87d Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Tue, 3 Dec 2024 10:27:02 -0600 Subject: [PATCH 14/30] Made suggested change. Signed-off-by: Cody Littley --- common/metrics/metrics.go | 20 --- common/metrics/metrics_server.go | 42 ------- common/metrics/mock_metrics.go | 42 ------- common/metrics/running_average.go | 70 ----------- common/metrics/running_average_metric.go | 119 ------------------ relay/cache/cache_accessor.go | 2 +- relay/cache/cache_accessor_metrics.go | 12 +- relay/mdoc/relay-metrics.md | 152 +++++++++++------------ relay/metrics.go | 34 +++-- relay/server.go | 6 +- 10 files changed, 97 insertions(+), 402 deletions(-) delete mode 100644 common/metrics/running_average.go delete mode 100644 common/metrics/running_average_metric.go diff --git a/common/metrics/metrics.go b/common/metrics/metrics.go index 4a201d260..4fbf5d054 100644 --- a/common/metrics/metrics.go +++ b/common/metrics/metrics.go @@ -66,15 +66,6 @@ type Metrics interface { source func() float64, label ...any) error - // NewRunningAverageMetric creates a new GaugeMetric instance that keeps track of the average of a series of values - // over a given time window. Each value within the window is given equal weight. - NewRunningAverageMetric( - name string, - unit string, - description string, - timeWindow time.Duration, - labelTemplate any) (RunningAverageMetric, error) - // RegisterExternalMetrics registers prometheus collectors created outside the metrics framework. RegisterExternalMetrics(collectors ...prometheus.Collector) } @@ -155,14 +146,3 @@ type LatencyMetric interface { // when creating the metric, an error will be returned. ReportLatency(latency time.Duration, label ...any) } - -// RunningAverageMetric tracks the average of a series of values over a given time window. -type RunningAverageMetric interface { - Metric - - // Update adds a new value to the RunningAverage. - Update(value float64, label ...any) - - // GetTimeWindow returns the time window used to calculate the running average. - GetTimeWindow() time.Duration -} diff --git a/common/metrics/metrics_server.go b/common/metrics/metrics_server.go index c6cb9c9e5..7a5868dd0 100644 --- a/common/metrics/metrics_server.go +++ b/common/metrics/metrics_server.go @@ -350,44 +350,6 @@ func (m *metrics) NewAutoGauge( return nil } -func (m *metrics) NewRunningAverageMetric( - name string, - unit string, - description string, - timeWindow time.Duration, - labelTemplate any) (RunningAverageMetric, error) { - - if !m.isAlive.Load() { - return nil, errors.New("metrics server is not alive") - } - - id, err := newMetricID(name, unit) - if err != nil { - return nil, err - } - - preExistingMetric, ok := m.metricMap[id] - if ok { - return preExistingMetric.(RunningAverageMetric), nil - } - - metric, err := newRunningAverageMetric( - m.logger, - m.registry, - m.namespace, - name, - unit, - description, - timeWindow, - labelTemplate) - if err != nil { - return nil, err - } - - m.metricMap[id] = metric - return metric, nil -} - func (m *metrics) GenerateMetricsDocumentation() string { sb := &strings.Builder{} @@ -439,10 +401,6 @@ func (m *metrics) GenerateMetricsDocumentation() string { if metric.Type() == "latency" { sb.Write([]byte(fmt.Sprintf("| **Quantiles** | %s |\n", m.quantilesMap[*id]))) } - if metric.Type() == "running average" { - sb.Write([]byte(fmt.Sprintf( - "| **Time Window** | `%s` |\n", metric.(*runningAverageMetric).GetTimeWindow()))) - } sb.Write([]byte(fmt.Sprintf("| **Fully Qualified Name** | `%s_%s_%s` |\n", m.namespace, id.name, id.unit))) } diff --git a/common/metrics/mock_metrics.go b/common/metrics/mock_metrics.go index b55d205b3..244f268c9 100644 --- a/common/metrics/mock_metrics.go +++ b/common/metrics/mock_metrics.go @@ -63,15 +63,6 @@ func (m *mockMetrics) NewAutoGauge( return nil } -func (m *mockMetrics) NewRunningAverageMetric( - name string, - unit string, - description string, - timeWindow time.Duration, - labelTemplate any) (RunningAverageMetric, error) { - return &mockRunningAverageMetric{}, nil -} - func (m *mockMetrics) RegisterExternalMetrics(collectors ...prometheus.Collector) { } @@ -166,36 +157,3 @@ func (m *mockLatencyMetric) LabelFields() []string { func (m *mockLatencyMetric) ReportLatency(latency time.Duration, label ...any) { } - -var _ RunningAverageMetric = &mockRunningAverageMetric{} - -type mockRunningAverageMetric struct { -} - -func (m *mockRunningAverageMetric) Name() string { - return "" -} - -func (m *mockRunningAverageMetric) Unit() string { - return "" -} - -func (m *mockRunningAverageMetric) Description() string { - return "" -} - -func (m *mockRunningAverageMetric) Type() string { - return "" -} - -func (m *mockRunningAverageMetric) LabelFields() []string { - return make([]string, 0) -} - -func (m *mockRunningAverageMetric) Update(value float64, label ...any) { - -} - -func (m *mockRunningAverageMetric) GetTimeWindow() time.Duration { - return 0 -} diff --git a/common/metrics/running_average.go b/common/metrics/running_average.go deleted file mode 100644 index 409a31a49..000000000 --- a/common/metrics/running_average.go +++ /dev/null @@ -1,70 +0,0 @@ -package metrics - -import ( - "github.com/emirpasic/gods/queues" - "github.com/emirpasic/gods/queues/linkedlistqueue" - "time" -) - -// RunningAverage keeps track of the average of a series of values over a given time window. -type RunningAverage struct { - maxAge time.Duration - sum float64 - count int - entries queues.Queue -} - -// NewRunningAverage creates a new RunningAverage with the given time window. -func NewRunningAverage(maxAge time.Duration) *RunningAverage { - return &RunningAverage{ - maxAge: maxAge, - entries: linkedlistqueue.New(), - } -} - -type runningAverageEntry struct { - value float64 - time time.Time -} - -// Update adds a new value to the RunningAverage and returns the new average. -func (a *RunningAverage) Update(now time.Time, value float64) float64 { - a.count++ - a.sum += value - a.entries.Enqueue(&runningAverageEntry{value: value, time: now}) - return a.GetAverage(now) -} - -// GetAverage returns the current average of the RunningAverage. -func (a *RunningAverage) GetAverage(now time.Time) float64 { - a.cleanup(now) - if a.count == 0 { - return 0 - } - return a.sum / float64(a.count) -} - -// cleanup removes old entries from the RunningAverage. -func (a *RunningAverage) cleanup(now time.Time) { - - for { - v, ok := a.entries.Peek() - if !ok { - break - } - entry := v.(*runningAverageEntry) - - if now.Sub(entry.time) <= a.maxAge { - break - } - - a.entries.Dequeue() - a.sum -= entry.value - a.count-- - } - - if a.count == 0 { - // clear away any cruft from accumulated floating point errors if we have no entries - a.sum = 0 - } -} diff --git a/common/metrics/running_average_metric.go b/common/metrics/running_average_metric.go deleted file mode 100644 index f0e263e92..000000000 --- a/common/metrics/running_average_metric.go +++ /dev/null @@ -1,119 +0,0 @@ -package metrics - -import ( - "fmt" - "github.com/Layr-Labs/eigensdk-go/logging" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" - "sync" - "time" -) - -var _ RunningAverageMetric = &runningAverageMetric{} - -type runningAverageMetric struct { - - // logger is the logger used to log errors. - logger logging.Logger - - // name is the name of the metric. - name string - - // unit is the unit of the metric. - unit string - - // description is the description of the metric. - description string - - // vec is the prometheus gauge vector used to store the metric. - vec *prometheus.GaugeVec - - // lm is the label maker used to create labels for this metric. - labeler *labelMaker - - // runningAverage is the running average used to calculate the average of the metric. - runningAverage *RunningAverage - - // timeWindow is the time window used to calculate the running average. - timeWindow time.Duration - - // lock is used to provide thread safety for the running average calculator. - lock sync.Mutex -} - -// newRunningAverageMetric creates a new RunningAverageMetric instance. -func newRunningAverageMetric( - logger logging.Logger, - registry *prometheus.Registry, - namespace string, - name string, - unit string, - description string, - timeWindow time.Duration, - labelTemplate any) (RunningAverageMetric, error) { - - labeler, err := newLabelMaker(labelTemplate) - if err != nil { - return nil, err - } - - vec := promauto.With(registry).NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: namespace, - Name: fmt.Sprintf("%s_%s", name, unit), - }, - labeler.getKeys(), - ) - - return &runningAverageMetric{ - logger: logger, - name: name, - unit: unit, - description: description, - vec: vec, - labeler: labeler, - runningAverage: NewRunningAverage(timeWindow), - timeWindow: timeWindow, - }, nil -} - -func (m *runningAverageMetric) Name() string { - return m.name -} - -func (m *runningAverageMetric) Unit() string { - return m.unit -} - -func (m *runningAverageMetric) Description() string { - return m.description -} - -func (m *runningAverageMetric) Type() string { - return "running average" -} - -func (m *runningAverageMetric) LabelFields() []string { - return m.labeler.getKeys() -} - -func (m *runningAverageMetric) Update(value float64, label ...any) { - var l any - if len(label) > 0 { - l = label[0] - } - - values, err := m.labeler.extractValues(l) - if err != nil { - m.logger.Errorf("error extracting values from label: %v", err) - } - - m.lock.Lock() - average := m.runningAverage.Update(time.Now(), value) - m.lock.Unlock() - m.vec.WithLabelValues(values...).Set(average) -} - -func (m *runningAverageMetric) GetTimeWindow() time.Duration { - return m.timeWindow -} diff --git a/relay/cache/cache_accessor.go b/relay/cache/cache_accessor.go index bfebf1c6b..e4c9b0b69 100644 --- a/relay/cache/cache_accessor.go +++ b/relay/cache/cache_accessor.go @@ -98,7 +98,7 @@ func NewCacheAccessor[K comparable, V any]( evictionHandler = func(key K, _ V) { if insertionTime, ok := insertionTimes[key]; ok { lifespan := time.Since(insertionTime).Milliseconds() - metrics.averageLifespan.Update(float64(lifespan)) + metrics.lifespan.Set(float64(lifespan)) delete(insertionTimes, key) } } diff --git a/relay/cache/cache_accessor_metrics.go b/relay/cache/cache_accessor_metrics.go index e4a4d50fc..59c721b46 100644 --- a/relay/cache/cache_accessor_metrics.go +++ b/relay/cache/cache_accessor_metrics.go @@ -3,7 +3,6 @@ package cache import ( "fmt" "github.com/Layr-Labs/eigenda/common/metrics" - "time" ) // CacheAccessorMetrics provides metrics for a CacheAccessor. @@ -13,7 +12,7 @@ type CacheAccessorMetrics struct { size metrics.GaugeMetric weight metrics.GaugeMetric averageWeight metrics.GaugeMetric - averageLifespan metrics.RunningAverageMetric + lifespan metrics.GaugeMetric cacheMissLatency metrics.LatencyMetric } @@ -65,11 +64,10 @@ func NewCacheAccessorMetrics( return nil, err } - averageLifespan, err := server.NewRunningAverageMetric( - fmt.Sprintf("%s_cache_average_lifespan", cacheName), + lifespan, err := server.NewGaugeMetric( + fmt.Sprintf("%s_cache_lifespan", cacheName), "ms", - fmt.Sprintf("Average time an item remains in the %s cache before being evicted.", cacheName), - time.Minute, + fmt.Sprintf("Time an item remains in the %s cache before being evicted.", cacheName), nil) if err != nil { return nil, err @@ -92,7 +90,7 @@ func NewCacheAccessorMetrics( size: size, weight: weight, averageWeight: averageWeight, - averageLifespan: averageLifespan, + lifespan: lifespan, cacheMissLatency: cacheMissLatency, }, nil } diff --git a/relay/mdoc/relay-metrics.md b/relay/mdoc/relay-metrics.md index 46ea716b1..959c3a549 100644 --- a/relay/mdoc/relay-metrics.md +++ b/relay/mdoc/relay-metrics.md @@ -1,50 +1,11 @@ # Metrics Documentation for namespace 'relay' -This documentation was automatically generated at time `2024-11-27T12:13:08-06:00` +This documentation was automatically generated at time `2024-12-03T10:26:19-06:00` There are a total of `34` registered metrics. --- -## average_get_blob_data_bytes - -Average data size of requested blobs - -| | | -|---|---| -| **Name** | `average_get_blob_data` | -| **Unit** | `bytes` | -| **Type** | `running average` | -| **Time Window** | `1m0s` | -| **Fully Qualified Name** | `relay_average_get_blob_data_bytes` | ---- - -## average_get_chunks_data_bytes - -Average data size in a GetChunks request - -| | | -|---|---| -| **Name** | `average_get_chunks_data` | -| **Unit** | `bytes` | -| **Type** | `running average` | -| **Time Window** | `1m0s` | -| **Fully Qualified Name** | `relay_average_get_chunks_data_bytes` | ---- - -## average_get_chunks_key_count - -Average number of keys in a GetChunks request - -| | | -|---|---| -| **Name** | `average_get_chunks_key` | -| **Unit** | `count` | -| **Type** | `running average` | -| **Time Window** | `1m0s` | -| **Fully Qualified Name** | `relay_average_get_chunks_key_count` | ---- - ## blob_cache_size Number of items in the blob cache @@ -81,19 +42,6 @@ Average weight of items currently in the blob cache | **Fully Qualified Name** | `relay_blob_cache_average_weight` | --- -## blob_cache_average_lifespan_ms - -Average time an item remains in the blob cache before being evicted. - -| | | -|---|---| -| **Name** | `blob_cache_average_lifespan` | -| **Unit** | `ms` | -| **Type** | `running average` | -| **Time Window** | `1m0s` | -| **Fully Qualified Name** | `relay_blob_cache_average_lifespan_ms` | ---- - ## blob_cache_hit_count Number of cache hits in the blob cache @@ -106,6 +54,18 @@ Number of cache hits in the blob cache | **Fully Qualified Name** | `relay_blob_cache_hit_count` | --- +## blob_cache_lifespan_ms + +Time an item remains in the blob cache before being evicted. + +| | | +|---|---| +| **Name** | `blob_cache_lifespan` | +| **Unit** | `ms` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_blob_cache_lifespan_ms` | +--- + ## blob_cache_miss_count Number of cache misses in the blob cache @@ -167,19 +127,6 @@ Average weight of items currently in the chunk cache | **Fully Qualified Name** | `relay_chunk_cache_average_weight` | --- -## chunk_cache_average_lifespan_ms - -Average time an item remains in the chunk cache before being evicted. - -| | | -|---|---| -| **Name** | `chunk_cache_average_lifespan` | -| **Unit** | `ms` | -| **Type** | `running average` | -| **Time Window** | `1m0s` | -| **Fully Qualified Name** | `relay_chunk_cache_average_lifespan_ms` | ---- - ## chunk_cache_hit_count Number of cache hits in the chunk cache @@ -192,6 +139,18 @@ Number of cache hits in the chunk cache | **Fully Qualified Name** | `relay_chunk_cache_hit_count` | --- +## chunk_cache_lifespan_ms + +Time an item remains in the chunk cache before being evicted. + +| | | +|---|---| +| **Name** | `chunk_cache_lifespan` | +| **Unit** | `ms` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_chunk_cache_lifespan_ms` | +--- + ## chunk_cache_miss_count Number of cache misses in the chunk cache @@ -230,6 +189,18 @@ Latency of the GetBlob RPC data retrieval | **Fully Qualified Name** | `relay_get_blob_data_latency_ms` | --- +## get_blob_data_size_bytes + +Data size of requested blobs. + +| | | +|---|---| +| **Name** | `get_blob_data_size` | +| **Unit** | `bytes` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_get_blob_data_size_bytes` | +--- + ## get_blob_latency_ms Latency of the GetBlob RPC @@ -307,6 +278,30 @@ Latency of the GetChunks RPC data retrieval | **Fully Qualified Name** | `relay_get_chunks_data_latency_ms` | --- +## get_chunks_data_size_bytes + +Data size in a GetChunks request. + +| | | +|---|---| +| **Name** | `get_chunks_data_size` | +| **Unit** | `bytes` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_get_chunks_data_size_bytes` | +--- + +## get_chunks_key_count + +Number of keys in a GetChunks request. + +| | | +|---|---| +| **Name** | `get_chunks_key` | +| **Unit** | `count` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_get_chunks_key_count` | +--- + ## get_chunks_latency_ms Latency of the GetChunks RPC @@ -382,19 +377,6 @@ Average weight of items currently in the metadata cache | **Fully Qualified Name** | `relay_metadata_cache_average_weight` | --- -## metadata_cache_average_lifespan_ms - -Average time an item remains in the metadata cache before being evicted. - -| | | -|---|---| -| **Name** | `metadata_cache_average_lifespan` | -| **Unit** | `ms` | -| **Type** | `running average` | -| **Time Window** | `1m0s` | -| **Fully Qualified Name** | `relay_metadata_cache_average_lifespan_ms` | ---- - ## metadata_cache_hit_count Number of cache hits in the metadata cache @@ -407,6 +389,18 @@ Number of cache hits in the metadata cache | **Fully Qualified Name** | `relay_metadata_cache_hit_count` | --- +## metadata_cache_lifespan_ms + +Time an item remains in the metadata cache before being evicted. + +| | | +|---|---| +| **Name** | `metadata_cache_lifespan` | +| **Unit** | `ms` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_metadata_cache_lifespan_ms` | +--- + ## metadata_cache_miss_count Number of cache misses in the metadata cache diff --git a/relay/metrics.go b/relay/metrics.go index ebf50e3f1..8a5bb034b 100644 --- a/relay/metrics.go +++ b/relay/metrics.go @@ -7,7 +7,6 @@ import ( "github.com/Layr-Labs/eigensdk-go/logging" grpcprom "github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus" "google.golang.org/grpc" - "time" ) type RelayMetrics struct { @@ -26,15 +25,15 @@ type RelayMetrics struct { GetChunksDataLatency metrics.LatencyMetric GetChunksAuthFailures metrics.CountMetric GetChunksRateLimited metrics.CountMetric - GetChunksAverageKeyCount metrics.RunningAverageMetric - GetChunksAverageDataSize metrics.RunningAverageMetric + GetChunksKeyCount metrics.GaugeMetric + GetChunksDataSize metrics.GaugeMetric // GetBlob metrics GetBlobLatency metrics.LatencyMetric GetBlobMetadataLatency metrics.LatencyMetric GetBlobDataLatency metrics.LatencyMetric GetBlobRateLimited metrics.CountMetric - GetBlobAverageDataSize metrics.RunningAverageMetric + GetBlobDataSize metrics.GaugeMetric } // NewRelayMetrics creates a new RelayMetrics instance, which encapsulates all metrics related to the relay. @@ -121,21 +120,19 @@ func NewRelayMetrics(logger logging.Logger, port int) (*RelayMetrics, error) { return nil, err } - getChunksAverageKeyCount, err := server.NewRunningAverageMetric( - "average_get_chunks_key", + getChunksKeyCount, err := server.NewGaugeMetric( + "get_chunks_key", "count", - "Average number of keys in a GetChunks request", - time.Minute, + "Number of keys in a GetChunks request.", nil) if err != nil { return nil, err } - getChunksAverageDataSize, err := server.NewRunningAverageMetric( - "average_get_chunks_data", + getChunksDataSize, err := server.NewGaugeMetric( + "get_chunks_data_size", "bytes", - "Average data size in a GetChunks request", - time.Minute, + "Data size in a GetChunks request.", nil) if err != nil { return nil, err @@ -176,11 +173,10 @@ func NewRelayMetrics(logger logging.Logger, port int) (*RelayMetrics, error) { return nil, err } - getBlobAverageDataSize, err := server.NewRunningAverageMetric( - "average_get_blob_data", + getBlobDataSize, err := server.NewGaugeMetric( + "get_blob_data_size", "bytes", - "Average data size of requested blobs", - time.Minute, + "Data size of requested blobs.", nil) if err != nil { return nil, err @@ -198,13 +194,13 @@ func NewRelayMetrics(logger logging.Logger, port int) (*RelayMetrics, error) { GetChunksDataLatency: getChunksDataLatencyMetric, GetChunksAuthFailures: getChunksAuthFailures, GetChunksRateLimited: getChunksRateLimited, - GetChunksAverageKeyCount: getChunksAverageKeyCount, - GetChunksAverageDataSize: getChunksAverageDataSize, + GetChunksKeyCount: getChunksKeyCount, + GetChunksDataSize: getChunksDataSize, GetBlobLatency: getBlobLatencyMetric, GetBlobMetadataLatency: getBlobMetadataLatencyMetric, GetBlobDataLatency: getBlobDataLatencyMetric, GetBlobRateLimited: getBlobRateLimited, - GetBlobAverageDataSize: getBlobAverageDataSize, + GetBlobDataSize: getBlobDataSize, }, nil } diff --git a/relay/server.go b/relay/server.go index 61d920dfe..54294af46 100644 --- a/relay/server.go +++ b/relay/server.go @@ -256,7 +256,7 @@ func (s *Server) GetBlob(ctx context.Context, request *pb.GetBlobRequest) (*pb.G return nil, fmt.Errorf("error fetching blob %s: %w", key.Hex(), err) } - s.metrics.GetBlobAverageDataSize.Update(float64(len(data))) + s.metrics.GetBlobDataSize.Set(float64(len(data))) s.metrics.GetBlobDataLatency.ReportLatency(time.Since(finishedFetchingMetadata)) s.metrics.GetBlobLatency.ReportLatency(time.Since(start)) @@ -283,7 +283,7 @@ func (s *Server) GetChunks(ctx context.Context, request *pb.GetChunksRequest) (* return nil, fmt.Errorf( "too many chunk requests provided, max is %d", s.config.MaxKeysPerGetChunksRequest) } - s.metrics.GetChunksAverageKeyCount.Update(float64(len(request.ChunkRequests))) + s.metrics.GetChunksKeyCount.Set(float64(len(request.ChunkRequests))) if s.authenticator != nil { client, ok := peer.FromContext(ctx) @@ -334,7 +334,7 @@ func (s *Server) GetChunks(ctx context.Context, request *pb.GetChunksRequest) (* if err != nil { return nil, err } - s.metrics.GetChunksAverageDataSize.Update(float64(requiredBandwidth)) + s.metrics.GetChunksDataSize.Set(float64(requiredBandwidth)) frames, err := s.chunkProvider.GetFrames(ctx, mMap) if err != nil { From 5c8c173637e2a5af2e4d20f89a624ac492bd6876 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Tue, 3 Dec 2024 11:38:09 -0600 Subject: [PATCH 15/30] Added metrics for v2 DA node. Signed-off-by: Cody Littley --- node/cmd/main.go | 5 +- node/config.go | 6 +- node/flags/flags.go | 12 +++- node/grpc/run.go | 9 ++- node/grpc/server_v2.go | 38 +++++++++-- node/grpc/server_v2_test.go | 3 +- node/grpc/v2_metrics.go | 132 ++++++++++++++++++++++++++++++++++++ node/mock/store_v2.go | 6 +- node/node.go | 4 +- node/store_v2.go | 36 ++++++---- node/store_v2_test.go | 6 +- test/integration_test.go | 4 +- 12 files changed, 223 insertions(+), 38 deletions(-) create mode 100644 node/grpc/v2_metrics.go diff --git a/node/cmd/main.go b/node/cmd/main.go index 0225b33d7..afbdd0158 100644 --- a/node/cmd/main.go +++ b/node/cmd/main.go @@ -86,7 +86,10 @@ func NodeMain(ctx *cli.Context) error { // Creates the GRPC server. server := nodegrpc.NewServer(config, node, logger, ratelimiter) - serverV2 := nodegrpc.NewServerV2(config, node, logger, ratelimiter) + serverV2, err := nodegrpc.NewServerV2(config, node, logger, ratelimiter) + if err != nil { + return fmt.Errorf("failed to create server v2: %v", err) + } err = nodegrpc.RunServers(server, serverV2, config, logger) return err diff --git a/node/config.go b/node/config.go index c5244e55b..af90ea7af 100644 --- a/node/config.go +++ b/node/config.go @@ -52,7 +52,7 @@ type Config struct { EnableNodeApi bool NodeApiPort string EnableMetrics bool - MetricsPort string + MetricsPort int OnchainMetricsInterval int64 Timeout time.Duration RegisterNodeAtStart bool @@ -62,6 +62,7 @@ type Config struct { OverrideStoreDurationBlocks int64 QuorumIDList []core.QuorumID DbPath string + DBSizePollPeriod time.Duration LogPath string PrivateBls string ID core.OperatorID @@ -206,7 +207,7 @@ func NewConfig(ctx *cli.Context) (*Config, error) { EnableNodeApi: ctx.GlobalBool(flags.EnableNodeApiFlag.Name), NodeApiPort: ctx.GlobalString(flags.NodeApiPortFlag.Name), EnableMetrics: ctx.GlobalBool(flags.EnableMetricsFlag.Name), - MetricsPort: ctx.GlobalString(flags.MetricsPortFlag.Name), + MetricsPort: ctx.GlobalInt(flags.MetricsPortFlag.Name), OnchainMetricsInterval: ctx.GlobalInt64(flags.OnchainMetricsIntervalFlag.Name), Timeout: timeout, RegisterNodeAtStart: registerNodeAtStart, @@ -217,6 +218,7 @@ func NewConfig(ctx *cli.Context) (*Config, error) { OverrideStoreDurationBlocks: ctx.GlobalInt64(flags.OverrideStoreDurationBlocksFlag.Name), QuorumIDList: ids, DbPath: ctx.GlobalString(flags.DbPathFlag.Name), + DBSizePollPeriod: ctx.GlobalDuration(flags.DBSizePollPeriodFlag.Name), PrivateBls: privateBls, EthClientConfig: ethClientConfig, EncoderConfig: kzg.ReadCLIConfig(ctx), diff --git a/node/flags/flags.go b/node/flags/flags.go index 40c1237a7..9d17e7e72 100644 --- a/node/flags/flags.go +++ b/node/flags/flags.go @@ -66,11 +66,11 @@ var ( Required: true, EnvVar: common.PrefixEnvVar(EnvVarPrefix, "ENABLE_METRICS"), } - MetricsPortFlag = cli.StringFlag{ + MetricsPortFlag = cli.IntFlag{ Name: common.PrefixFlag(FlagPrefix, "metrics-port"), Usage: "Port at which node listens for metrics calls", Required: false, - Value: "9091", + Value: 9091, EnvVar: common.PrefixEnvVar(EnvVarPrefix, "METRICS_PORT"), } OnchainMetricsIntervalFlag = cli.StringFlag{ @@ -98,6 +98,13 @@ var ( Required: true, EnvVar: common.PrefixEnvVar(EnvVarPrefix, "DB_PATH"), } + DBSizePollPeriodFlag = cli.DurationFlag{ + Name: common.PrefixFlag(FlagPrefix, "db-size-poll-period"), + Usage: "The period at which the database size is polled. If set to 0, the database size is not polled.", + Required: false, + Value: 10 * time.Minute, + EnvVar: common.PrefixEnvVar(EnvVarPrefix, "DB_SIZE_POLL_PERIOD"), + } // The files for encrypted private keys. BlsKeyFileFlag = cli.StringFlag{ Name: common.PrefixFlag(FlagPrefix, "bls-key-file"), @@ -376,6 +383,7 @@ var optionalFlags = []cli.Flag{ OnchainStateRefreshIntervalFlag, PprofHttpPort, EnablePprof, + DBSizePollPeriodFlag, } func init() { diff --git a/node/grpc/run.go b/node/grpc/run.go index 5fea03d62..9019a8b65 100644 --- a/node/grpc/run.go +++ b/node/grpc/run.go @@ -24,6 +24,11 @@ func RunServers(serverV1 *Server, serverV2 *ServerV2, config *node.Config, logge return errors.New("node V2 server is not configured") } + err := serverV2.metrics.Start() + if err != nil { + return fmt.Errorf("failed to start metrics: %v", err) + } + go func() { for { addr := fmt.Sprintf("%s:%s", localhost, config.InternalDispersalPort) @@ -33,7 +38,7 @@ func RunServers(serverV1 *Server, serverV2 *ServerV2, config *node.Config, logge } opt := grpc.MaxRecvMsgSize(60 * 1024 * 1024 * 1024) // 60 GiB - gs := grpc.NewServer(opt) + gs := grpc.NewServer(opt, serverV2.metrics.GetGRPCServerOption()) // Register reflection service on gRPC server // This makes "grpcurl -plaintext localhost:9000 list" command work @@ -60,7 +65,7 @@ func RunServers(serverV1 *Server, serverV2 *ServerV2, config *node.Config, logge } opt := grpc.MaxRecvMsgSize(1024 * 1024 * 300) // 300 MiB - gs := grpc.NewServer(opt) + gs := grpc.NewServer(opt, serverV2.metrics.GetGRPCServerOption()) // Register reflection service on gRPC server // This makes "grpcurl -plaintext localhost:9000 list" command work diff --git a/node/grpc/server_v2.go b/node/grpc/server_v2.go index 4f46a70d5..c7e8f06f4 100644 --- a/node/grpc/server_v2.go +++ b/node/grpc/server_v2.go @@ -4,8 +4,6 @@ import ( "context" "encoding/hex" "fmt" - "runtime" - "github.com/Layr-Labs/eigenda/api" pb "github.com/Layr-Labs/eigenda/api/grpc/node/v2" "github.com/Layr-Labs/eigenda/common" @@ -15,6 +13,8 @@ import ( "github.com/Layr-Labs/eigenda/node" "github.com/Layr-Labs/eigensdk-go/logging" "github.com/shirou/gopsutil/mem" + "runtime" + "time" ) // ServerV2 implements the Node v2 proto APIs. @@ -26,6 +26,7 @@ type ServerV2 struct { node *node.Node ratelimiter common.RateLimiter logger logging.Logger + metrics *V2Metrics } // NewServerV2 creates a new Server instance with the provided parameters. @@ -33,14 +34,20 @@ func NewServerV2( config *node.Config, node *node.Node, logger logging.Logger, - ratelimiter common.RateLimiter, -) *ServerV2 { + ratelimiter common.RateLimiter) (*ServerV2, error) { + + metrics, err := NewV2Metrics(logger, config.MetricsPort, config.DbPath, config.DBSizePollPeriod) + if err != nil { + return nil, err + } + return &ServerV2{ config: config, node: node, ratelimiter: ratelimiter, logger: logger, - } + metrics: metrics, + }, nil } func (s *ServerV2) NodeInfo(ctx context.Context, in *pb.NodeInfoRequest) (*pb.NodeInfoReply, error) { @@ -58,6 +65,8 @@ func (s *ServerV2) NodeInfo(ctx context.Context, in *pb.NodeInfoRequest) (*pb.No } func (s *ServerV2) StoreChunks(ctx context.Context, in *pb.StoreChunksRequest) (*pb.StoreChunksReply, error) { + start := time.Now() + if !s.config.EnableV2 { return nil, api.NewErrorInvalidArg("v2 API is disabled") } @@ -92,7 +101,7 @@ func (s *ServerV2) StoreChunks(ctx context.Context, in *pb.StoreChunksRequest) ( } storeChan := make(chan storeResult) go func() { - keys, err := s.node.StoreV2.StoreBatch(batch, rawBundles) + keys, size, err := s.node.StoreV2.StoreBatch(batch, rawBundles) if err != nil { storeChan <- storeResult{ keys: nil, @@ -101,6 +110,8 @@ func (s *ServerV2) StoreChunks(ctx context.Context, in *pb.StoreChunksRequest) ( return } + s.metrics.StoreChunksDataSize.Set(float64(size)) + storeChan <- storeResult{ keys: keys, err: nil, @@ -124,6 +135,10 @@ func (s *ServerV2) StoreChunks(ctx context.Context, in *pb.StoreChunksRequest) ( } sig := s.node.KeyPair.SignMessage(batchHeaderHash).Bytes() + + timeElapsed := time.Since(start) + s.metrics.StoreChunksLatency.ReportLatency(timeElapsed) + return &pb.StoreChunksReply{ Signature: sig[:], }, nil @@ -144,6 +159,8 @@ func (s *ServerV2) validateStoreChunksRequest(req *pb.StoreChunksRequest) (*core } func (s *ServerV2) GetChunks(ctx context.Context, in *pb.GetChunksRequest) (*pb.GetChunksReply, error) { + start := time.Now() + if !s.config.EnableV2 { return nil, api.NewErrorInvalidArg("v2 API is disabled") } @@ -166,6 +183,15 @@ func (s *ServerV2) GetChunks(ctx context.Context, in *pb.GetChunksRequest) (*pb. return nil, api.NewErrorInternal(fmt.Sprintf("failed to get chunks: %v", err)) } + var size uint64 + for _, chunk := range chunks { + size += uint64(len(chunk)) + } + s.metrics.GetChunksDataSize.Set(float64(size)) + + elapsed := time.Since(start) + s.metrics.GetChunksLatency.ReportLatency(elapsed) + return &pb.GetChunksReply{ Chunks: chunks, }, nil diff --git a/node/grpc/server_v2_test.go b/node/grpc/server_v2_test.go index 249bd6bb7..9d92d5c8e 100644 --- a/node/grpc/server_v2_test.go +++ b/node/grpc/server_v2_test.go @@ -79,7 +79,8 @@ func newTestComponents(t *testing.T, config *node.Config) *testComponents { RelayClient: relay, } node.BlobVersionParams.Store(v2.NewBlobVersionParameterMap(blobParamsMap)) - server := grpc.NewServerV2(config, node, logger, ratelimiter) + server, err := grpc.NewServerV2(config, node, logger, ratelimiter) + require.NoError(t, err) return &testComponents{ server: server, node: node, diff --git a/node/grpc/v2_metrics.go b/node/grpc/v2_metrics.go new file mode 100644 index 000000000..769e14c33 --- /dev/null +++ b/node/grpc/v2_metrics.go @@ -0,0 +1,132 @@ +package grpc + +import ( + "github.com/Layr-Labs/eigenda/common/metrics" + "github.com/Layr-Labs/eigensdk-go/logging" + grpcprom "github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus" + "google.golang.org/grpc" + "os" + "path/filepath" + "time" +) + +// V2Metrics encapsulates metrics for the v2 DA node. +type V2Metrics struct { + metricsServer metrics.Metrics + grpcServerOption grpc.ServerOption + + StoreChunksLatency metrics.LatencyMetric + StoreChunksDataSize metrics.GaugeMetric + + GetChunksLatency metrics.LatencyMetric + GetChunksDataSize metrics.GaugeMetric +} + +// NewV2Metrics creates a new V2Metrics instance. dbSizePollPeriod is the period at which the database size is polled. +// If set to 0, the database size is not polled. +func NewV2Metrics( + logger logging.Logger, + port int, + dbDir string, + dbSizePollPeriod time.Duration) (*V2Metrics, error) { + + server := metrics.NewMetrics(logger, "eigenda_node", port) + + grpcMetrics := grpcprom.NewServerMetrics() + server.RegisterExternalMetrics(grpcMetrics) + grpcServerOption := grpc.UnaryInterceptor( + grpcMetrics.UnaryServerInterceptor(), + ) + + storeChunksLatency, err := server.NewLatencyMetric( + "store_chunks_latency", + "The latency of a StoreChunks() RPC call.", + nil, + metrics.NewQuantile(0.5), + metrics.NewQuantile(0.9), + metrics.NewQuantile(0.99)) + if err != nil { + return nil, err + } + + storeChunksDataSize, err := server.NewGaugeMetric( + "store_chunks_data_size", + "bytes", + "The size of the data requested to be stored by StoreChunks() RPC calls.", + nil) + if err != nil { + return nil, err + } + + getChunksLatency, err := server.NewLatencyMetric( + "get_chunks_latency", + "The latency of a GetChunks() RPC call.", + nil, + metrics.NewQuantile(0.5), + metrics.NewQuantile(0.9), + metrics.NewQuantile(0.99)) + if err != nil { + return nil, err + } + + getChunksDataSize, err := server.NewGaugeMetric( + "get_chunks_data_size", + "bytes", + "The size of the data requested to be retrieved by GetChunks() RPC calls.", + nil) + if err != nil { + return nil, err + } + + if dbSizePollPeriod.Nanoseconds() > 0 { + err = server.NewAutoGauge( + "db_size", + "bytes", + "The size of the leveldb database.", + dbSizePollPeriod, + func() float64 { + var size int64 + err = filepath.Walk(dbDir, func(_ string, info os.FileInfo, err error) error { + if err != nil { + return err + } + if !info.IsDir() { + size += info.Size() + } + return err + }) + if err != nil { + logger.Errorf("failed to get database size (for metrics reporting): %v", err) + return -1.0 + } + return float64(size) + }) + if err != nil { + return nil, err + } + } + + return &V2Metrics{ + metricsServer: server, + grpcServerOption: grpcServerOption, + StoreChunksLatency: storeChunksLatency, + StoreChunksDataSize: storeChunksDataSize, + GetChunksLatency: getChunksLatency, + GetChunksDataSize: getChunksDataSize, + }, nil +} + +// Start starts the metrics server. +func (m *V2Metrics) Start() error { + return m.metricsServer.Start() +} + +// Stop stops the metrics server. +func (m *V2Metrics) Stop() error { + return m.metricsServer.Stop() +} + +// GetGRPCServerOption returns the gRPC server option that enables automatic GRPC metrics collection. +func (m *V2Metrics) GetGRPCServerOption() grpc.ServerOption { + return m.grpcServerOption +} diff --git a/node/mock/store_v2.go b/node/mock/store_v2.go index e7d473c3d..a14a034ee 100644 --- a/node/mock/store_v2.go +++ b/node/mock/store_v2.go @@ -19,12 +19,12 @@ func NewMockStoreV2() *MockStoreV2 { return &MockStoreV2{} } -func (m *MockStoreV2) StoreBatch(batch *corev2.Batch, rawBundles []*node.RawBundles) ([]kvstore.Key, error) { +func (m *MockStoreV2) StoreBatch(batch *corev2.Batch, rawBundles []*node.RawBundles) ([]kvstore.Key, uint64, error) { args := m.Called(batch, rawBundles) if args.Get(0) == nil { - return nil, args.Error(1) + return nil, 0, args.Error(1) } - return args.Get(0).([]kvstore.Key), args.Error(1) + return args.Get(0).([]kvstore.Key), 0, args.Error(1) } func (m *MockStoreV2) DeleteKeys(keys []kvstore.Key) error { diff --git a/node/node.go b/node/node.go index 07055e55c..3b8ab30f4 100644 --- a/node/node.go +++ b/node/node.go @@ -103,7 +103,7 @@ func NewNode( nodeLogger := logger.With("component", "Node") - eigenMetrics := metrics.NewEigenMetrics(AppName, ":"+config.MetricsPort, reg, logger.With("component", "EigenMetrics")) + eigenMetrics := metrics.NewEigenMetrics(AppName, fmt.Sprintf(":%d", config.MetricsPort), reg, logger.With("component", "EigenMetrics")) rpcCallsCollector := rpccalls.NewCollector(AppName, reg) // Make sure config folder exists. @@ -175,7 +175,7 @@ func NewNode( // Setup Node Api nodeApi := nodeapi.NewNodeApi(AppName, SemVer, ":"+config.NodeApiPort, logger.With("component", "NodeApi")) - metrics := NewMetrics(eigenMetrics, reg, logger, ":"+config.MetricsPort, config.ID, config.OnchainMetricsInterval, tx, cst) + metrics := NewMetrics(eigenMetrics, reg, logger, fmt.Sprintf(":%d", config.MetricsPort), config.ID, config.OnchainMetricsInterval, tx, cst) // Make validator v, err := verifier.NewVerifier(&config.EncoderConfig, false) diff --git a/node/store_v2.go b/node/store_v2.go index 62da00f54..c7b409aa0 100644 --- a/node/store_v2.go +++ b/node/store_v2.go @@ -19,7 +19,9 @@ const ( ) type StoreV2 interface { - StoreBatch(batch *corev2.Batch, rawBundles []*RawBundles) ([]kvstore.Key, error) + // StoreBatch stores a batch and its raw bundles in the database. Returns the keys of the stored data + // and the size of the stored data, in bytes. + StoreBatch(batch *corev2.Batch, rawBundles []*RawBundles) ([]kvstore.Key, uint64, error) DeleteKeys(keys []kvstore.Key) error GetChunks(blobKey corev2.BlobKey, quorum core.QuorumID) ([][]byte, error) } @@ -40,81 +42,85 @@ func NewLevelDBStoreV2(db kvstore.TableStore, logger logging.Logger) *storeV2 { } } -func (s *storeV2) StoreBatch(batch *corev2.Batch, rawBundles []*RawBundles) ([]kvstore.Key, error) { +func (s *storeV2) StoreBatch(batch *corev2.Batch, rawBundles []*RawBundles) ([]kvstore.Key, uint64, error) { if len(rawBundles) == 0 { - return nil, fmt.Errorf("no raw bundles") + return nil, 0, fmt.Errorf("no raw bundles") } if len(rawBundles) != len(batch.BlobCertificates) { - return nil, fmt.Errorf("mismatch between raw bundles (%d) and blob certificates (%d)", len(rawBundles), len(batch.BlobCertificates)) + return nil, 0, fmt.Errorf("mismatch between raw bundles (%d) and blob certificates (%d)", len(rawBundles), len(batch.BlobCertificates)) } dbBatch := s.db.NewTTLBatch() + var size uint64 keys := make([]kvstore.Key, 0) batchHeaderKeyBuilder, err := s.db.GetKeyBuilder(BatchHeaderTableName) if err != nil { - return nil, fmt.Errorf("failed to get key builder for batch header: %v", err) + return nil, 0, fmt.Errorf("failed to get key builder for batch header: %v", err) } batchHeaderHash, err := batch.BatchHeader.Hash() if err != nil { - return nil, fmt.Errorf("failed to hash batch header: %v", err) + return nil, 0, fmt.Errorf("failed to hash batch header: %v", err) } // Store batch header batchHeaderKey := batchHeaderKeyBuilder.Key(batchHeaderHash[:]) if _, err = s.db.Get(batchHeaderKey); err == nil { - return nil, ErrBatchAlreadyExist + return nil, 0, ErrBatchAlreadyExist } batchHeaderBytes, err := batch.BatchHeader.Serialize() if err != nil { - return nil, fmt.Errorf("failed to serialize batch header: %v", err) + return nil, 0, fmt.Errorf("failed to serialize batch header: %v", err) } keys = append(keys, batchHeaderKey) dbBatch.PutWithTTL(batchHeaderKey, batchHeaderBytes, s.ttl) + size += uint64(len(batchHeaderBytes)) // Store blob shards for _, bundles := range rawBundles { // Store blob certificate blobCertificateKeyBuilder, err := s.db.GetKeyBuilder(BlobCertificateTableName) if err != nil { - return nil, fmt.Errorf("failed to get key builder for blob certificate: %v", err) + return nil, 0, fmt.Errorf("failed to get key builder for blob certificate: %v", err) } blobKey, err := bundles.BlobCertificate.BlobHeader.BlobKey() if err != nil { - return nil, fmt.Errorf("failed to get blob key: %v", err) + return nil, 0, fmt.Errorf("failed to get blob key: %v", err) } blobCertificateKey := blobCertificateKeyBuilder.Key(blobKey[:]) blobCertificateBytes, err := bundles.BlobCertificate.Serialize() if err != nil { - return nil, fmt.Errorf("failed to serialize blob certificate: %v", err) + return nil, 0, fmt.Errorf("failed to serialize blob certificate: %v", err) } keys = append(keys, blobCertificateKey) dbBatch.PutWithTTL(blobCertificateKey, blobCertificateBytes, s.ttl) + size += uint64(len(blobCertificateBytes)) // Store bundles for quorum, bundle := range bundles.Bundles { bundlesKeyBuilder, err := s.db.GetKeyBuilder(BundleTableName) if err != nil { - return nil, fmt.Errorf("failed to get key builder for bundles: %v", err) + return nil, 0, fmt.Errorf("failed to get key builder for bundles: %v", err) } k, err := BundleKey(blobKey, quorum) if err != nil { - return nil, fmt.Errorf("failed to get key for bundles: %v", err) + return nil, 0, fmt.Errorf("failed to get key for bundles: %v", err) } keys = append(keys, bundlesKeyBuilder.Key(k)) dbBatch.PutWithTTL(bundlesKeyBuilder.Key(k), bundle, s.ttl) + size += uint64(len(bundle)) } } if err := dbBatch.Apply(); err != nil { - return nil, fmt.Errorf("failed to apply batch: %v", err) + return nil, 0, fmt.Errorf("failed to apply batch: %v", err) } - return keys, nil + return keys, size, nil } func (s *storeV2) DeleteKeys(keys []kvstore.Key) error { diff --git a/node/store_v2_test.go b/node/store_v2_test.go index 2a78c8080..5519c5153 100644 --- a/node/store_v2_test.go +++ b/node/store_v2_test.go @@ -41,7 +41,7 @@ func TestStoreBatchV2(t *testing.T) { defer func() { _ = db.Shutdown() }() - keys, err := s.StoreBatch(batch, rawBundles) + keys, _, err := s.StoreBatch(batch, rawBundles) require.NoError(t, err) require.Len(t, keys, 10) @@ -92,7 +92,7 @@ func TestStoreBatchV2(t *testing.T) { } // Try to store the same batch again - _, err = s.StoreBatch(batch, rawBundles) + _, _, err = s.StoreBatch(batch, rawBundles) require.ErrorIs(t, err, node.ErrBatchAlreadyExist) // Check deletion @@ -151,7 +151,7 @@ func TestGetChunks(t *testing.T) { defer func() { _ = db.Shutdown() }() - _, err := s.StoreBatch(batch, rawBundles) + _, _, err := s.StoreBatch(batch, rawBundles) require.NoError(t, err) chunks, err := s.GetChunks(blobKeys[0], 0) diff --git a/test/integration_test.go b/test/integration_test.go index 3df7de45e..429d73d87 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -6,6 +6,7 @@ import ( "encoding/hex" "errors" "fmt" + "github.com/stretchr/testify/require" "log" "math" "math/big" @@ -418,7 +419,8 @@ func mustMakeOperators(t *testing.T, cst *coremock.ChainDataMock, logger logging ratelimiter := &commonmock.NoopRatelimiter{} serverV1 := nodegrpc.NewServer(config, n, logger, ratelimiter) - serverV2 := nodegrpc.NewServerV2(config, n, logger, ratelimiter) + serverV2, err := nodegrpc.NewServerV2(config, n, logger, ratelimiter) + require.NoError(t, err) ops[id] = TestOperator{ Node: n, From 17956540414febce7f59bc98157368db47b52024 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Tue, 3 Dec 2024 11:45:30 -0600 Subject: [PATCH 16/30] Added metrics documentation. Signed-off-by: Cody Littley --- metrics.md | 1 + node/grpc/v2_metrics.go | 5 +++ node/mdoc/main.go | 25 ++++++++++++++ node/mdoc/node-metrics.md | 68 +++++++++++++++++++++++++++++++++++++++ 4 files changed, 99 insertions(+) create mode 100644 node/mdoc/main.go create mode 100644 node/mdoc/node-metrics.md diff --git a/metrics.md b/metrics.md index 7138d98ca..6edb7f33a 100644 --- a/metrics.md +++ b/metrics.md @@ -1,5 +1,6 @@ # EigenDA Metrics Documentation - [churner](operators/churner/mdoc/churner-metrics.md) +- [DA node](node/mdoc/node-metrics.md) - [relay](relay/mdoc/relay-metrics.md) diff --git a/node/grpc/v2_metrics.go b/node/grpc/v2_metrics.go index 769e14c33..fb0b44837 100644 --- a/node/grpc/v2_metrics.go +++ b/node/grpc/v2_metrics.go @@ -130,3 +130,8 @@ func (m *V2Metrics) Stop() error { func (m *V2Metrics) GetGRPCServerOption() grpc.ServerOption { return m.grpcServerOption } + +// WriteMetricsDocumentation writes the metrics for the DA node to a markdown file. +func (m *V2Metrics) WriteMetricsDocumentation() error { + return m.metricsServer.WriteMetricsDocumentation("node/mdoc/node-metrics.md") +} diff --git a/node/mdoc/main.go b/node/mdoc/main.go new file mode 100644 index 000000000..0e54efd5a --- /dev/null +++ b/node/mdoc/main.go @@ -0,0 +1,25 @@ +package main + +import ( + "github.com/Layr-Labs/eigenda/common" + nodegrpc "github.com/Layr-Labs/eigenda/node/grpc" + "time" +) + +// main generates documentation for relay metrics. +func main() { + logger, err := common.NewLogger(common.DefaultLoggerConfig()) + if err != nil { + panic(err) + } + + metrics, err := nodegrpc.NewV2Metrics(logger, 0, "", time.Second) + if err != nil { + panic(err) + } + + err = metrics.WriteMetricsDocumentation() + if err != nil { + panic(err) + } +} diff --git a/node/mdoc/node-metrics.md b/node/mdoc/node-metrics.md new file mode 100644 index 000000000..6b5257a78 --- /dev/null +++ b/node/mdoc/node-metrics.md @@ -0,0 +1,68 @@ +# Metrics Documentation for namespace 'eigenda_node' + +This documentation was automatically generated at time `2024-12-03T11:40:32-06:00` + +There are a total of `5` registered metrics. + +--- + +## db_size_bytes + +The size of the leveldb database. + +| | | +|---|---| +| **Name** | `db_size` | +| **Unit** | `bytes` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `eigenda_node_db_size_bytes` | +--- + +## get_chunks_data_size_bytes + +The size of the data requested to be retrieved by GetChunks() RPC calls. + +| | | +|---|---| +| **Name** | `get_chunks_data_size` | +| **Unit** | `bytes` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `eigenda_node_get_chunks_data_size_bytes` | +--- + +## get_chunks_latency_ms + +The latency of a GetChunks() RPC call. + +| | | +|---|---| +| **Name** | `get_chunks_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `eigenda_node_get_chunks_latency_ms` | +--- + +## store_chunks_data_size_bytes + +The size of the data requested to be stored by StoreChunks() RPC calls. + +| | | +|---|---| +| **Name** | `store_chunks_data_size` | +| **Unit** | `bytes` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `eigenda_node_store_chunks_data_size_bytes` | +--- + +## store_chunks_latency_ms + +The latency of a StoreChunks() RPC call. + +| | | +|---|---| +| **Name** | `store_chunks_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `eigenda_node_store_chunks_latency_ms` | From 5c9274c5d5d2e5f2b42e4f4b059bf27e1af8861b Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Fri, 6 Dec 2024 11:49:52 -0600 Subject: [PATCH 17/30] Revert deletions. Signed-off-by: Cody Littley --- metrics.md | 5 + relay/mdoc/relay-metrics.md | 426 ++++++++++++++++++++++++++++++++++++ 2 files changed, 431 insertions(+) create mode 100644 metrics.md create mode 100644 relay/mdoc/relay-metrics.md diff --git a/metrics.md b/metrics.md new file mode 100644 index 000000000..7138d98ca --- /dev/null +++ b/metrics.md @@ -0,0 +1,5 @@ +# EigenDA Metrics Documentation + +- [churner](operators/churner/mdoc/churner-metrics.md) +- [relay](relay/mdoc/relay-metrics.md) + diff --git a/relay/mdoc/relay-metrics.md b/relay/mdoc/relay-metrics.md new file mode 100644 index 000000000..959c3a549 --- /dev/null +++ b/relay/mdoc/relay-metrics.md @@ -0,0 +1,426 @@ +# Metrics Documentation for namespace 'relay' + +This documentation was automatically generated at time `2024-12-03T10:26:19-06:00` + +There are a total of `34` registered metrics. + +--- + +## blob_cache_size + +Number of items in the blob cache + +| | | +|---|---| +| **Name** | `blob_cache` | +| **Unit** | `size` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_blob_cache_size` | +--- + +## blob_cache_weight + +Total weight of items in the blob cache + +| | | +|---|---| +| **Name** | `blob_cache` | +| **Unit** | `weight` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_blob_cache_weight` | +--- + +## blob_cache_average_weight + +Average weight of items currently in the blob cache + +| | | +|---|---| +| **Name** | `blob_cache_average` | +| **Unit** | `weight` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_blob_cache_average_weight` | +--- + +## blob_cache_hit_count + +Number of cache hits in the blob cache + +| | | +|---|---| +| **Name** | `blob_cache_hit` | +| **Unit** | `count` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_blob_cache_hit_count` | +--- + +## blob_cache_lifespan_ms + +Time an item remains in the blob cache before being evicted. + +| | | +|---|---| +| **Name** | `blob_cache_lifespan` | +| **Unit** | `ms` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_blob_cache_lifespan_ms` | +--- + +## blob_cache_miss_count + +Number of cache misses in the blob cache + +| | | +|---|---| +| **Name** | `blob_cache_miss` | +| **Unit** | `count` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_blob_cache_miss_count` | +--- + +## blob_cache_miss_latency_ms + +Latency of cache misses in the blob cache + +| | | +|---|---| +| **Name** | `blob_cache_miss_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_blob_cache_miss_latency_ms` | +--- + +## chunk_cache_size + +Number of items in the chunk cache + +| | | +|---|---| +| **Name** | `chunk_cache` | +| **Unit** | `size` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_chunk_cache_size` | +--- + +## chunk_cache_weight + +Total weight of items in the chunk cache + +| | | +|---|---| +| **Name** | `chunk_cache` | +| **Unit** | `weight` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_chunk_cache_weight` | +--- + +## chunk_cache_average_weight + +Average weight of items currently in the chunk cache + +| | | +|---|---| +| **Name** | `chunk_cache_average` | +| **Unit** | `weight` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_chunk_cache_average_weight` | +--- + +## chunk_cache_hit_count + +Number of cache hits in the chunk cache + +| | | +|---|---| +| **Name** | `chunk_cache_hit` | +| **Unit** | `count` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_chunk_cache_hit_count` | +--- + +## chunk_cache_lifespan_ms + +Time an item remains in the chunk cache before being evicted. + +| | | +|---|---| +| **Name** | `chunk_cache_lifespan` | +| **Unit** | `ms` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_chunk_cache_lifespan_ms` | +--- + +## chunk_cache_miss_count + +Number of cache misses in the chunk cache + +| | | +|---|---| +| **Name** | `chunk_cache_miss` | +| **Unit** | `count` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_chunk_cache_miss_count` | +--- + +## chunk_cache_miss_latency_ms + +Latency of cache misses in the chunk cache + +| | | +|---|---| +| **Name** | `chunk_cache_miss_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_chunk_cache_miss_latency_ms` | +--- + +## get_blob_data_latency_ms + +Latency of the GetBlob RPC data retrieval + +| | | +|---|---| +| **Name** | `get_blob_data_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_get_blob_data_latency_ms` | +--- + +## get_blob_data_size_bytes + +Data size of requested blobs. + +| | | +|---|---| +| **Name** | `get_blob_data_size` | +| **Unit** | `bytes` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_get_blob_data_size_bytes` | +--- + +## get_blob_latency_ms + +Latency of the GetBlob RPC + +| | | +|---|---| +| **Name** | `get_blob_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_get_blob_latency_ms` | +--- + +## get_blob_metadata_latency_ms + +Latency of the GetBlob RPC metadata retrieval + +| | | +|---|---| +| **Name** | `get_blob_metadata_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_get_blob_metadata_latency_ms` | +--- + +## get_blob_rate_limited_count + +Number of GetBlob RPC rate limited + +| | | +|---|---| +| **Name** | `get_blob_rate_limited` | +| **Unit** | `count` | +| **Labels** | `reason` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_get_blob_rate_limited_count` | +--- + +## get_chunks_auth_failure_count + +Number of GetChunks RPC authentication failures + +| | | +|---|---| +| **Name** | `get_chunks_auth_failure` | +| **Unit** | `count` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_get_chunks_auth_failure_count` | +--- + +## get_chunks_authentication_latency_ms + +Latency of the GetChunks RPC client authentication + +| | | +|---|---| +| **Name** | `get_chunks_authentication_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_get_chunks_authentication_latency_ms` | +--- + +## get_chunks_data_latency_ms + +Latency of the GetChunks RPC data retrieval + +| | | +|---|---| +| **Name** | `get_chunks_data_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_get_chunks_data_latency_ms` | +--- + +## get_chunks_data_size_bytes + +Data size in a GetChunks request. + +| | | +|---|---| +| **Name** | `get_chunks_data_size` | +| **Unit** | `bytes` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_get_chunks_data_size_bytes` | +--- + +## get_chunks_key_count + +Number of keys in a GetChunks request. + +| | | +|---|---| +| **Name** | `get_chunks_key` | +| **Unit** | `count` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_get_chunks_key_count` | +--- + +## get_chunks_latency_ms + +Latency of the GetChunks RPC + +| | | +|---|---| +| **Name** | `get_chunks_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_get_chunks_latency_ms` | +--- + +## get_chunks_metadata_latency_ms + +Latency of the GetChunks RPC metadata retrieval + +| | | +|---|---| +| **Name** | `get_chunks_metadata_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_get_chunks_metadata_latency_ms` | +--- + +## get_chunks_rate_limited_count + +Number of GetChunks RPC rate limited + +| | | +|---|---| +| **Name** | `get_chunks_rate_limited` | +| **Unit** | `count` | +| **Labels** | `reason` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_get_chunks_rate_limited_count` | +--- + +## metadata_cache_size + +Number of items in the metadata cache + +| | | +|---|---| +| **Name** | `metadata_cache` | +| **Unit** | `size` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_metadata_cache_size` | +--- + +## metadata_cache_weight + +Total weight of items in the metadata cache + +| | | +|---|---| +| **Name** | `metadata_cache` | +| **Unit** | `weight` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_metadata_cache_weight` | +--- + +## metadata_cache_average_weight + +Average weight of items currently in the metadata cache + +| | | +|---|---| +| **Name** | `metadata_cache_average` | +| **Unit** | `weight` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_metadata_cache_average_weight` | +--- + +## metadata_cache_hit_count + +Number of cache hits in the metadata cache + +| | | +|---|---| +| **Name** | `metadata_cache_hit` | +| **Unit** | `count` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_metadata_cache_hit_count` | +--- + +## metadata_cache_lifespan_ms + +Time an item remains in the metadata cache before being evicted. + +| | | +|---|---| +| **Name** | `metadata_cache_lifespan` | +| **Unit** | `ms` | +| **Type** | `gauge` | +| **Fully Qualified Name** | `relay_metadata_cache_lifespan_ms` | +--- + +## metadata_cache_miss_count + +Number of cache misses in the metadata cache + +| | | +|---|---| +| **Name** | `metadata_cache_miss` | +| **Unit** | `count` | +| **Type** | `counter` | +| **Fully Qualified Name** | `relay_metadata_cache_miss_count` | +--- + +## metadata_cache_miss_latency_ms + +Latency of cache misses in the metadata cache + +| | | +|---|---| +| **Name** | `metadata_cache_miss_latency` | +| **Unit** | `ms` | +| **Type** | `latency` | +| **Quantiles** | `0.500`, `0.900`, `0.990` | +| **Fully Qualified Name** | `relay_metadata_cache_miss_latency_ms` | From 4d4bfe9caff5c81bc1051b5dac27f81fad9a914a Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Fri, 6 Dec 2024 11:53:41 -0600 Subject: [PATCH 18/30] Remove documentation. Signed-off-by: Cody Littley --- node/mdoc/main.go | 25 -------------- node/mdoc/node-metrics.md | 68 --------------------------------------- 2 files changed, 93 deletions(-) delete mode 100644 node/mdoc/main.go delete mode 100644 node/mdoc/node-metrics.md diff --git a/node/mdoc/main.go b/node/mdoc/main.go deleted file mode 100644 index 0e54efd5a..000000000 --- a/node/mdoc/main.go +++ /dev/null @@ -1,25 +0,0 @@ -package main - -import ( - "github.com/Layr-Labs/eigenda/common" - nodegrpc "github.com/Layr-Labs/eigenda/node/grpc" - "time" -) - -// main generates documentation for relay metrics. -func main() { - logger, err := common.NewLogger(common.DefaultLoggerConfig()) - if err != nil { - panic(err) - } - - metrics, err := nodegrpc.NewV2Metrics(logger, 0, "", time.Second) - if err != nil { - panic(err) - } - - err = metrics.WriteMetricsDocumentation() - if err != nil { - panic(err) - } -} diff --git a/node/mdoc/node-metrics.md b/node/mdoc/node-metrics.md deleted file mode 100644 index 6b5257a78..000000000 --- a/node/mdoc/node-metrics.md +++ /dev/null @@ -1,68 +0,0 @@ -# Metrics Documentation for namespace 'eigenda_node' - -This documentation was automatically generated at time `2024-12-03T11:40:32-06:00` - -There are a total of `5` registered metrics. - ---- - -## db_size_bytes - -The size of the leveldb database. - -| | | -|---|---| -| **Name** | `db_size` | -| **Unit** | `bytes` | -| **Type** | `gauge` | -| **Fully Qualified Name** | `eigenda_node_db_size_bytes` | ---- - -## get_chunks_data_size_bytes - -The size of the data requested to be retrieved by GetChunks() RPC calls. - -| | | -|---|---| -| **Name** | `get_chunks_data_size` | -| **Unit** | `bytes` | -| **Type** | `gauge` | -| **Fully Qualified Name** | `eigenda_node_get_chunks_data_size_bytes` | ---- - -## get_chunks_latency_ms - -The latency of a GetChunks() RPC call. - -| | | -|---|---| -| **Name** | `get_chunks_latency` | -| **Unit** | `ms` | -| **Type** | `latency` | -| **Quantiles** | `0.500`, `0.900`, `0.990` | -| **Fully Qualified Name** | `eigenda_node_get_chunks_latency_ms` | ---- - -## store_chunks_data_size_bytes - -The size of the data requested to be stored by StoreChunks() RPC calls. - -| | | -|---|---| -| **Name** | `store_chunks_data_size` | -| **Unit** | `bytes` | -| **Type** | `gauge` | -| **Fully Qualified Name** | `eigenda_node_store_chunks_data_size_bytes` | ---- - -## store_chunks_latency_ms - -The latency of a StoreChunks() RPC call. - -| | | -|---|---| -| **Name** | `store_chunks_latency` | -| **Unit** | `ms` | -| **Type** | `latency` | -| **Quantiles** | `0.500`, `0.900`, `0.990` | -| **Fully Qualified Name** | `eigenda_node_store_chunks_latency_ms` | From d9d898cec094badc9b6d50e16bb6af79bf2a13ae Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Fri, 6 Dec 2024 12:20:20 -0600 Subject: [PATCH 19/30] Reimplement without metrics framework. Signed-off-by: Cody Littley --- node/grpc/run.go | 5 +- node/grpc/server_v2.go | 8 +- node/grpc/v2_metrics.go | 216 ++++++++++++++++++++++++++-------------- 3 files changed, 144 insertions(+), 85 deletions(-) diff --git a/node/grpc/run.go b/node/grpc/run.go index 9019a8b65..3f0a90073 100644 --- a/node/grpc/run.go +++ b/node/grpc/run.go @@ -24,10 +24,7 @@ func RunServers(serverV1 *Server, serverV2 *ServerV2, config *node.Config, logge return errors.New("node V2 server is not configured") } - err := serverV2.metrics.Start() - if err != nil { - return fmt.Errorf("failed to start metrics: %v", err) - } + serverV2.metrics.Start() go func() { for { diff --git a/node/grpc/server_v2.go b/node/grpc/server_v2.go index c7e8f06f4..179e876b7 100644 --- a/node/grpc/server_v2.go +++ b/node/grpc/server_v2.go @@ -110,7 +110,7 @@ func (s *ServerV2) StoreChunks(ctx context.Context, in *pb.StoreChunksRequest) ( return } - s.metrics.StoreChunksDataSize.Set(float64(size)) + s.metrics.ReportStoreChunksDataSize(size) storeChan <- storeResult{ keys: keys, @@ -137,7 +137,7 @@ func (s *ServerV2) StoreChunks(ctx context.Context, in *pb.StoreChunksRequest) ( sig := s.node.KeyPair.SignMessage(batchHeaderHash).Bytes() timeElapsed := time.Since(start) - s.metrics.StoreChunksLatency.ReportLatency(timeElapsed) + s.metrics.ReportStoreChunksLatency(timeElapsed) return &pb.StoreChunksReply{ Signature: sig[:], @@ -187,10 +187,10 @@ func (s *ServerV2) GetChunks(ctx context.Context, in *pb.GetChunksRequest) (*pb. for _, chunk := range chunks { size += uint64(len(chunk)) } - s.metrics.GetChunksDataSize.Set(float64(size)) + s.metrics.ReportGetChunksDataSize(size) elapsed := time.Since(start) - s.metrics.GetChunksLatency.ReportLatency(elapsed) + s.metrics.ReportGetChunksLatency(elapsed) return &pb.GetChunksReply{ Chunks: chunks, diff --git a/node/grpc/v2_metrics.go b/node/grpc/v2_metrics.go index fb0b44837..119f941fa 100644 --- a/node/grpc/v2_metrics.go +++ b/node/grpc/v2_metrics.go @@ -1,25 +1,42 @@ package grpc import ( - "github.com/Layr-Labs/eigenda/common/metrics" + "fmt" "github.com/Layr-Labs/eigensdk-go/logging" grpcprom "github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/collectors" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/prometheus/client_golang/prometheus/promhttp" "google.golang.org/grpc" + "net/http" "os" "path/filepath" + "strings" + "sync/atomic" "time" ) +const namespace = "eigenda_node" + // V2Metrics encapsulates metrics for the v2 DA node. type V2Metrics struct { - metricsServer metrics.Metrics + logger logging.Logger + + registry *prometheus.Registry + server *http.Server grpcServerOption grpc.ServerOption - StoreChunksLatency metrics.LatencyMetric - StoreChunksDataSize metrics.GaugeMetric + storeChunksLatency *prometheus.SummaryVec + storeChunksDataSize *prometheus.GaugeVec + + getChunksLatency *prometheus.SummaryVec + getChunksDataSize *prometheus.GaugeVec - GetChunksLatency metrics.LatencyMetric - GetChunksDataSize metrics.GaugeMetric + dbSize *prometheus.GaugeVec + dbSizePollPeriod time.Duration + dbDir string + isAlive *atomic.Bool } // NewV2Metrics creates a new V2Metrics instance. dbSizePollPeriod is the period at which the database size is polled. @@ -30,63 +47,109 @@ func NewV2Metrics( dbDir string, dbSizePollPeriod time.Duration) (*V2Metrics, error) { - server := metrics.NewMetrics(logger, "eigenda_node", port) + registry := prometheus.NewRegistry() + registry.MustRegister(collectors.NewProcessCollector(collectors.ProcessCollectorOpts{})) + registry.MustRegister(collectors.NewGoCollector()) + + logger.Infof("Starting metrics server at port %d", port) + addr := fmt.Sprintf(":%d", port) + mux := http.NewServeMux() + mux.Handle("/metrics", promhttp.HandlerFor( + registry, + promhttp.HandlerOpts{}, + )) + server := &http.Server{ + Addr: addr, + Handler: mux, + } grpcMetrics := grpcprom.NewServerMetrics() - server.RegisterExternalMetrics(grpcMetrics) + registry.MustRegister(grpcMetrics) grpcServerOption := grpc.UnaryInterceptor( grpcMetrics.UnaryServerInterceptor(), ) - storeChunksLatency, err := server.NewLatencyMetric( - "store_chunks_latency", - "The latency of a StoreChunks() RPC call.", - nil, - metrics.NewQuantile(0.5), - metrics.NewQuantile(0.9), - metrics.NewQuantile(0.99)) - if err != nil { - return nil, err - } + storeChunksLatency := promauto.With(registry).NewSummaryVec( + prometheus.SummaryOpts{ + Namespace: namespace, + Name: "store_chunks_latency_ms", + Help: "The latency of a StoreChunks() RPC call.", + Objectives: map[float64]float64{0.5: 0.05, 0.9: 0.01, 0.99: 0.001}, + }, + []string{}, + ) - storeChunksDataSize, err := server.NewGaugeMetric( - "store_chunks_data_size", - "bytes", - "The size of the data requested to be stored by StoreChunks() RPC calls.", - nil) - if err != nil { - return nil, err - } + storeChunksDataSize := promauto.With(registry).NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: namespace, + Name: "store_chunks_data_size_bytes", + Help: "The size of the data requested to be stored by StoreChunks() RPC calls.", + }, + []string{}, + ) - getChunksLatency, err := server.NewLatencyMetric( - "get_chunks_latency", - "The latency of a GetChunks() RPC call.", - nil, - metrics.NewQuantile(0.5), - metrics.NewQuantile(0.9), - metrics.NewQuantile(0.99)) - if err != nil { - return nil, err - } + getChunksLatency := promauto.With(registry).NewSummaryVec( + prometheus.SummaryOpts{ + Namespace: namespace, + Name: "get_chunks_latency_ms", + Help: "The latency of a GetChunks() RPC call.", + Objectives: map[float64]float64{0.5: 0.05, 0.9: 0.01, 0.99: 0.001}, + }, + []string{}, + ) - getChunksDataSize, err := server.NewGaugeMetric( - "get_chunks_data_size", - "bytes", - "The size of the data requested to be retrieved by GetChunks() RPC calls.", - nil) - if err != nil { - return nil, err - } + getChunksDataSize := promauto.With(registry).NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: namespace, + Name: "get_chunks_data_size_bytes", + Help: "The size of the data requested to be retrieved by GetChunks() RPC calls.", + }, + []string{}, + ) + + dbSize := promauto.With(registry).NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: namespace, + Name: "db_size_bytes", + Help: "The size of the leveldb database.", + }, + []string{}, + ) + isAlive := &atomic.Bool{} + isAlive.Store(true) + + return &V2Metrics{ + logger: logger, + registry: registry, + server: server, + grpcServerOption: grpcServerOption, + storeChunksLatency: storeChunksLatency, + storeChunksDataSize: storeChunksDataSize, + getChunksLatency: getChunksLatency, + getChunksDataSize: getChunksDataSize, + dbSize: dbSize, + dbSizePollPeriod: dbSizePollPeriod, + dbDir: dbDir, + isAlive: isAlive, + }, nil +} - if dbSizePollPeriod.Nanoseconds() > 0 { - err = server.NewAutoGauge( - "db_size", - "bytes", - "The size of the leveldb database.", - dbSizePollPeriod, - func() float64 { +// Start starts the metrics server. +func (m *V2Metrics) Start() { + go func() { + err := m.server.ListenAndServe() + if err != nil && !strings.Contains(err.Error(), "http: Server closed") { + m.logger.Errorf("metrics server error: %v", err) + } + }() + + if m.dbSizePollPeriod.Nanoseconds() > 0 { + go func() { + ticker := time.NewTicker(m.dbSizePollPeriod) + + for m.isAlive.Load() { var size int64 - err = filepath.Walk(dbDir, func(_ string, info os.FileInfo, err error) error { + err := filepath.Walk(m.dbDir, func(_ string, info os.FileInfo, err error) error { if err != nil { return err } @@ -95,35 +158,21 @@ func NewV2Metrics( } return err }) + if err != nil { - logger.Errorf("failed to get database size (for metrics reporting): %v", err) - return -1.0 + m.logger.Errorf("failed to get database size: %v", err) + } else { + m.dbSize.WithLabelValues().Set(float64(size)) } - return float64(size) - }) - if err != nil { - return nil, err - } + <-ticker.C + } + }() } - - return &V2Metrics{ - metricsServer: server, - grpcServerOption: grpcServerOption, - StoreChunksLatency: storeChunksLatency, - StoreChunksDataSize: storeChunksDataSize, - GetChunksLatency: getChunksLatency, - GetChunksDataSize: getChunksDataSize, - }, nil -} - -// Start starts the metrics server. -func (m *V2Metrics) Start() error { - return m.metricsServer.Start() } // Stop stops the metrics server. func (m *V2Metrics) Stop() error { - return m.metricsServer.Stop() + return m.server.Close() } // GetGRPCServerOption returns the gRPC server option that enables automatic GRPC metrics collection. @@ -131,7 +180,20 @@ func (m *V2Metrics) GetGRPCServerOption() grpc.ServerOption { return m.grpcServerOption } -// WriteMetricsDocumentation writes the metrics for the DA node to a markdown file. -func (m *V2Metrics) WriteMetricsDocumentation() error { - return m.metricsServer.WriteMetricsDocumentation("node/mdoc/node-metrics.md") +func (m *V2Metrics) ReportStoreChunksLatency(latency time.Duration) { + m.storeChunksLatency.WithLabelValues().Observe( + float64(latency.Nanoseconds()) / float64(time.Millisecond)) +} + +func (m *V2Metrics) ReportStoreChunksDataSize(size uint64) { + m.storeChunksDataSize.WithLabelValues().Set(float64(size)) +} + +func (m *V2Metrics) ReportGetChunksLatency(latency time.Duration) { + m.getChunksLatency.WithLabelValues().Observe( + float64(latency.Nanoseconds()) / float64(time.Millisecond)) +} + +func (m *V2Metrics) ReportGetChunksDataSize(size uint64) { + m.getChunksDataSize.WithLabelValues().Set(float64(size)) } From 8bd8ff1afdf93b4a47889a5146e1a0704fe53922 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Fri, 6 Dec 2024 12:21:56 -0600 Subject: [PATCH 20/30] Cleanup. Signed-off-by: Cody Littley --- node/grpc/v2_metrics.go | 44 +++++++++++++++++++++-------------------- 1 file changed, 23 insertions(+), 21 deletions(-) diff --git a/node/grpc/v2_metrics.go b/node/grpc/v2_metrics.go index 119f941fa..b66199636 100644 --- a/node/grpc/v2_metrics.go +++ b/node/grpc/v2_metrics.go @@ -143,31 +143,33 @@ func (m *V2Metrics) Start() { } }() - if m.dbSizePollPeriod.Nanoseconds() > 0 { - go func() { - ticker := time.NewTicker(m.dbSizePollPeriod) - - for m.isAlive.Load() { - var size int64 - err := filepath.Walk(m.dbDir, func(_ string, info os.FileInfo, err error) error { - if err != nil { - return err - } - if !info.IsDir() { - size += info.Size() - } - return err - }) + if m.dbSizePollPeriod.Nanoseconds() == 0 { + return + } + go func() { + ticker := time.NewTicker(m.dbSizePollPeriod) + for m.isAlive.Load() { + var size int64 + err := filepath.Walk(m.dbDir, func(_ string, info os.FileInfo, err error) error { if err != nil { - m.logger.Errorf("failed to get database size: %v", err) - } else { - m.dbSize.WithLabelValues().Set(float64(size)) + return err + } + if !info.IsDir() { + size += info.Size() } - <-ticker.C + return err + }) + + if err != nil { + m.logger.Errorf("failed to get database size: %v", err) + } else { + m.dbSize.WithLabelValues().Set(float64(size)) } - }() - } + <-ticker.C + } + }() + } // Stop stops the metrics server. From 2070eee3e25255849b535f365a9b86f685361240 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Fri, 6 Dec 2024 12:23:56 -0600 Subject: [PATCH 21/30] Stop background thread when metrics are stopped. Signed-off-by: Cody Littley --- node/grpc/v2_metrics.go | 1 + 1 file changed, 1 insertion(+) diff --git a/node/grpc/v2_metrics.go b/node/grpc/v2_metrics.go index b66199636..db4bcdb2e 100644 --- a/node/grpc/v2_metrics.go +++ b/node/grpc/v2_metrics.go @@ -174,6 +174,7 @@ func (m *V2Metrics) Start() { // Stop stops the metrics server. func (m *V2Metrics) Stop() error { + m.isAlive.Store(false) return m.server.Close() } From cffa88412e94a235d99ede1be518c7b67208504f Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Fri, 6 Dec 2024 12:28:11 -0600 Subject: [PATCH 22/30] Revert unintentional change Signed-off-by: Cody Littley --- relay/mdoc/main.go | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) create mode 100644 relay/mdoc/main.go diff --git a/relay/mdoc/main.go b/relay/mdoc/main.go new file mode 100644 index 000000000..56c68999f --- /dev/null +++ b/relay/mdoc/main.go @@ -0,0 +1,24 @@ +package main + +import ( + "github.com/Layr-Labs/eigenda/common" + "github.com/Layr-Labs/eigenda/relay/metrics" +) + +// main generates documentation for relay metrics. +func main() { + logger, err := common.NewLogger(common.DefaultLoggerConfig()) + if err != nil { + panic(err) + } + + metrics, err := metrics.NewRelayMetrics(logger, 0) + if err != nil { + panic(err) + } + + err = metrics.WriteMetricsDocumentation() + if err != nil { + panic(err) + } +} From 5c511c9e144f71cc5536d645195b0a5f54ec2ac8 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Tue, 10 Dec 2024 10:07:48 -0600 Subject: [PATCH 23/30] Made suggested changes. Signed-off-by: Cody Littley --- node/grpc/server_v2.go | 9 ++++----- node/grpc/v2_metrics.go | 2 +- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/node/grpc/server_v2.go b/node/grpc/server_v2.go index 179e876b7..240a6b59a 100644 --- a/node/grpc/server_v2.go +++ b/node/grpc/server_v2.go @@ -136,8 +136,7 @@ func (s *ServerV2) StoreChunks(ctx context.Context, in *pb.StoreChunksRequest) ( sig := s.node.KeyPair.SignMessage(batchHeaderHash).Bytes() - timeElapsed := time.Since(start) - s.metrics.ReportStoreChunksLatency(timeElapsed) + s.metrics.ReportStoreChunksLatency(time.Since(start)) return &pb.StoreChunksReply{ Signature: sig[:], @@ -183,9 +182,9 @@ func (s *ServerV2) GetChunks(ctx context.Context, in *pb.GetChunksRequest) (*pb. return nil, api.NewErrorInternal(fmt.Sprintf("failed to get chunks: %v", err)) } - var size uint64 - for _, chunk := range chunks { - size += uint64(len(chunk)) + size := 0 + if len(chunks) > 0 { + size = len(chunks[0]) * len(chunks) } s.metrics.ReportGetChunksDataSize(size) diff --git a/node/grpc/v2_metrics.go b/node/grpc/v2_metrics.go index db4bcdb2e..686fb6888 100644 --- a/node/grpc/v2_metrics.go +++ b/node/grpc/v2_metrics.go @@ -197,6 +197,6 @@ func (m *V2Metrics) ReportGetChunksLatency(latency time.Duration) { float64(latency.Nanoseconds()) / float64(time.Millisecond)) } -func (m *V2Metrics) ReportGetChunksDataSize(size uint64) { +func (m *V2Metrics) ReportGetChunksDataSize(size int) { m.getChunksDataSize.WithLabelValues().Set(float64(size)) } From a15117fbe06ad026cf3ddd301dbe6f318f5a9b2b Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Tue, 10 Dec 2024 10:29:06 -0600 Subject: [PATCH 24/30] Don't start two metrics servers. Signed-off-by: Cody Littley --- node/cmd/main.go | 5 ++++- node/grpc/server_v2.go | 6 ++++-- node/grpc/v2_metrics.go | 31 ++----------------------------- 3 files changed, 10 insertions(+), 32 deletions(-) diff --git a/node/cmd/main.go b/node/cmd/main.go index afbdd0158..a54e7bdfd 100644 --- a/node/cmd/main.go +++ b/node/cmd/main.go @@ -85,8 +85,11 @@ func NodeMain(ctx *cli.Context) error { } // Creates the GRPC server. + + // TODO(cody-littley): the metrics server is currently started by eigenmetrics, which is in another repo. + // When we fully remove v1 support, we need to start the metrics server inside the v2 metrics code. server := nodegrpc.NewServer(config, node, logger, ratelimiter) - serverV2, err := nodegrpc.NewServerV2(config, node, logger, ratelimiter) + serverV2, err := nodegrpc.NewServerV2(config, node, logger, ratelimiter, reg) if err != nil { return fmt.Errorf("failed to create server v2: %v", err) } diff --git a/node/grpc/server_v2.go b/node/grpc/server_v2.go index 240a6b59a..0c34e3535 100644 --- a/node/grpc/server_v2.go +++ b/node/grpc/server_v2.go @@ -12,6 +12,7 @@ import ( corev2 "github.com/Layr-Labs/eigenda/core/v2" "github.com/Layr-Labs/eigenda/node" "github.com/Layr-Labs/eigensdk-go/logging" + "github.com/prometheus/client_golang/prometheus" "github.com/shirou/gopsutil/mem" "runtime" "time" @@ -34,9 +35,10 @@ func NewServerV2( config *node.Config, node *node.Node, logger logging.Logger, - ratelimiter common.RateLimiter) (*ServerV2, error) { + ratelimiter common.RateLimiter, + registry *prometheus.Registry) (*ServerV2, error) { - metrics, err := NewV2Metrics(logger, config.MetricsPort, config.DbPath, config.DBSizePollPeriod) + metrics, err := NewV2Metrics(logger, registry, config.DbPath, config.DBSizePollPeriod) if err != nil { return nil, err } diff --git a/node/grpc/v2_metrics.go b/node/grpc/v2_metrics.go index 686fb6888..b9b41330a 100644 --- a/node/grpc/v2_metrics.go +++ b/node/grpc/v2_metrics.go @@ -1,18 +1,14 @@ package grpc import ( - "fmt" "github.com/Layr-Labs/eigensdk-go/logging" grpcprom "github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/collectors" "github.com/prometheus/client_golang/prometheus/promauto" - "github.com/prometheus/client_golang/prometheus/promhttp" "google.golang.org/grpc" - "net/http" "os" "path/filepath" - "strings" "sync/atomic" "time" ) @@ -24,7 +20,6 @@ type V2Metrics struct { logger logging.Logger registry *prometheus.Registry - server *http.Server grpcServerOption grpc.ServerOption storeChunksLatency *prometheus.SummaryVec @@ -43,26 +38,13 @@ type V2Metrics struct { // If set to 0, the database size is not polled. func NewV2Metrics( logger logging.Logger, - port int, + registry *prometheus.Registry, dbDir string, dbSizePollPeriod time.Duration) (*V2Metrics, error) { - registry := prometheus.NewRegistry() registry.MustRegister(collectors.NewProcessCollector(collectors.ProcessCollectorOpts{})) registry.MustRegister(collectors.NewGoCollector()) - logger.Infof("Starting metrics server at port %d", port) - addr := fmt.Sprintf(":%d", port) - mux := http.NewServeMux() - mux.Handle("/metrics", promhttp.HandlerFor( - registry, - promhttp.HandlerOpts{}, - )) - server := &http.Server{ - Addr: addr, - Handler: mux, - } - grpcMetrics := grpcprom.NewServerMetrics() registry.MustRegister(grpcMetrics) grpcServerOption := grpc.UnaryInterceptor( @@ -121,7 +103,6 @@ func NewV2Metrics( return &V2Metrics{ logger: logger, registry: registry, - server: server, grpcServerOption: grpcServerOption, storeChunksLatency: storeChunksLatency, storeChunksDataSize: storeChunksDataSize, @@ -136,13 +117,6 @@ func NewV2Metrics( // Start starts the metrics server. func (m *V2Metrics) Start() { - go func() { - err := m.server.ListenAndServe() - if err != nil && !strings.Contains(err.Error(), "http: Server closed") { - m.logger.Errorf("metrics server error: %v", err) - } - }() - if m.dbSizePollPeriod.Nanoseconds() == 0 { return } @@ -173,9 +147,8 @@ func (m *V2Metrics) Start() { } // Stop stops the metrics server. -func (m *V2Metrics) Stop() error { +func (m *V2Metrics) Stop() { m.isAlive.Store(false) - return m.server.Close() } // GetGRPCServerOption returns the gRPC server option that enables automatic GRPC metrics collection. From 1076a8f3e18b931c3b2f82d757e58fb7f260d853 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Tue, 10 Dec 2024 11:20:41 -0600 Subject: [PATCH 25/30] Fix compile issue. Signed-off-by: Cody Littley --- node/grpc/server_v2_test.go | 2 +- test/integration_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/node/grpc/server_v2_test.go b/node/grpc/server_v2_test.go index ffb57cb2d..cbfd9adcd 100644 --- a/node/grpc/server_v2_test.go +++ b/node/grpc/server_v2_test.go @@ -82,7 +82,7 @@ func newTestComponents(t *testing.T, config *node.Config) *testComponents { RelayClient: atomicRelayClient, } node.BlobVersionParams.Store(v2.NewBlobVersionParameterMap(blobParamsMap)) - server, err := grpc.NewServerV2(config, node, logger, ratelimiter) + server, err := grpc.NewServerV2(config, node, logger, ratelimiter, prometheus.NewRegistry()) require.NoError(t, err) return &testComponents{ server: server, diff --git a/test/integration_test.go b/test/integration_test.go index 72e20e9b9..f1497fe79 100644 --- a/test/integration_test.go +++ b/test/integration_test.go @@ -419,7 +419,7 @@ func mustMakeOperators(t *testing.T, cst *coremock.ChainDataMock, logger logging ratelimiter := &commonmock.NoopRatelimiter{} serverV1 := nodegrpc.NewServer(config, n, logger, ratelimiter) - serverV2, err := nodegrpc.NewServerV2(config, n, logger, ratelimiter) + serverV2, err := nodegrpc.NewServerV2(config, n, logger, ratelimiter, prometheus.NewRegistry()) require.NoError(t, err) ops[id] = TestOperator{ From 143b798c2c5f594bb2e6c3097d7899951a674b8d Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Thu, 12 Dec 2024 08:22:18 -0600 Subject: [PATCH 26/30] Enable debug code. Signed-off-by: Cody Littley --- inabox/deploy/utils.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/inabox/deploy/utils.go b/inabox/deploy/utils.go index f83dc2232..ad1abcbfd 100644 --- a/inabox/deploy/utils.go +++ b/inabox/deploy/utils.go @@ -306,8 +306,8 @@ func execCmd(name string, args []string, envVars []string) error { var out bytes.Buffer var stderr bytes.Buffer // TODO: When these are uncommented, the deployer sometimes fails to start anvil - // cmd.Stdout = &out - // cmd.Stderr = &stderr + cmd.Stdout = &out + cmd.Stderr = &stderr err := cmd.Run() if err != nil { From 168ded5d45ec96b943eac0126612ccb137ff358d Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Thu, 12 Dec 2024 08:40:08 -0600 Subject: [PATCH 27/30] Debug Signed-off-by: Cody Littley --- inabox/deploy/utils.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/inabox/deploy/utils.go b/inabox/deploy/utils.go index ad1abcbfd..a459d9513 100644 --- a/inabox/deploy/utils.go +++ b/inabox/deploy/utils.go @@ -306,8 +306,8 @@ func execCmd(name string, args []string, envVars []string) error { var out bytes.Buffer var stderr bytes.Buffer // TODO: When these are uncommented, the deployer sometimes fails to start anvil - cmd.Stdout = &out - cmd.Stderr = &stderr + cmd.Stdout = os.Stdout + cmd.Stderr = os.Stderr err := cmd.Run() if err != nil { From dd21f6180b63c215c5c81aa62046fb789e13a877 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Thu, 12 Dec 2024 08:59:56 -0600 Subject: [PATCH 28/30] Fix inabox bug. Signed-off-by: Cody Littley --- inabox/deploy/utils.go | 4 ++-- node/grpc/v2_metrics.go | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/inabox/deploy/utils.go b/inabox/deploy/utils.go index a459d9513..f83dc2232 100644 --- a/inabox/deploy/utils.go +++ b/inabox/deploy/utils.go @@ -306,8 +306,8 @@ func execCmd(name string, args []string, envVars []string) error { var out bytes.Buffer var stderr bytes.Buffer // TODO: When these are uncommented, the deployer sometimes fails to start anvil - cmd.Stdout = os.Stdout - cmd.Stderr = os.Stderr + // cmd.Stdout = &out + // cmd.Stderr = &stderr err := cmd.Run() if err != nil { diff --git a/node/grpc/v2_metrics.go b/node/grpc/v2_metrics.go index b9b41330a..c79facf3c 100644 --- a/node/grpc/v2_metrics.go +++ b/node/grpc/v2_metrics.go @@ -4,7 +4,6 @@ import ( "github.com/Layr-Labs/eigensdk-go/logging" grpcprom "github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/collectors" "github.com/prometheus/client_golang/prometheus/promauto" "google.golang.org/grpc" "os" @@ -42,8 +41,9 @@ func NewV2Metrics( dbDir string, dbSizePollPeriod time.Duration) (*V2Metrics, error) { - registry.MustRegister(collectors.NewProcessCollector(collectors.ProcessCollectorOpts{})) - registry.MustRegister(collectors.NewGoCollector()) + // These should be re-enabled once the legacy v1 metrics are removed. + //registry.MustRegister(collectors.NewProcessCollector(collectors.ProcessCollectorOpts{})) + //registry.MustRegister(collectors.NewGoCollector()) grpcMetrics := grpcprom.NewServerMetrics() registry.MustRegister(grpcMetrics) From 1bb1404fbbd06989741553424471962e85e4e595 Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Thu, 12 Dec 2024 09:08:42 -0600 Subject: [PATCH 29/30] Made suggested changes. Signed-off-by: Cody Littley --- node/config.go | 2 +- node/flags/flags.go | 11 +++++----- node/grpc/{v2_metrics.go => metrics_v2.go} | 24 +++++++++++----------- node/grpc/server_v2.go | 5 ++--- node/store_v2.go | 9 ++++++++ 5 files changed, 30 insertions(+), 21 deletions(-) rename node/grpc/{v2_metrics.go => metrics_v2.go} (88%) diff --git a/node/config.go b/node/config.go index 1a6ecd7f8..d1b15fd1d 100644 --- a/node/config.go +++ b/node/config.go @@ -219,7 +219,7 @@ func NewConfig(ctx *cli.Context) (*Config, error) { OverrideStoreDurationBlocks: ctx.GlobalInt64(flags.OverrideStoreDurationBlocksFlag.Name), QuorumIDList: ids, DbPath: ctx.GlobalString(flags.DbPathFlag.Name), - DBSizePollPeriod: ctx.GlobalDuration(flags.DBSizePollPeriodFlag.Name), + DBSizePollPeriod: ctx.GlobalDuration(flags.MetricsDBSizePollPeriodFlag.Name), PrivateBls: privateBls, EthClientConfig: ethClientConfig, EncoderConfig: kzg.ReadCLIConfig(ctx), diff --git a/node/flags/flags.go b/node/flags/flags.go index 3776ee221..010edfffb 100644 --- a/node/flags/flags.go +++ b/node/flags/flags.go @@ -98,12 +98,13 @@ var ( Required: true, EnvVar: common.PrefixEnvVar(EnvVarPrefix, "DB_PATH"), } - DBSizePollPeriodFlag = cli.DurationFlag{ - Name: common.PrefixFlag(FlagPrefix, "db-size-poll-period"), - Usage: "The period at which the database size is polled. If set to 0, the database size is not polled.", + MetricsDBSizePollPeriodFlag = cli.DurationFlag{ + Name: common.PrefixFlag(FlagPrefix, "metrics-db-size-poll-period"), + Usage: "The period at which the database size is polled for metrics. " + + "If set to 0, the database size is not polled.", Required: false, Value: 10 * time.Minute, - EnvVar: common.PrefixEnvVar(EnvVarPrefix, "DB_SIZE_POLL_PERIOD"), + EnvVar: common.PrefixEnvVar(EnvVarPrefix, "METRICS_DB_SIZE_POLL_PERIOD"), } // The files for encrypted private keys. BlsKeyFileFlag = cli.StringFlag{ @@ -391,7 +392,7 @@ var optionalFlags = []cli.Flag{ ChunkDownloadTimeoutFlag, PprofHttpPort, EnablePprof, - DBSizePollPeriodFlag, + MetricsDBSizePollPeriodFlag, } func init() { diff --git a/node/grpc/v2_metrics.go b/node/grpc/metrics_v2.go similarity index 88% rename from node/grpc/v2_metrics.go rename to node/grpc/metrics_v2.go index c79facf3c..973e1178b 100644 --- a/node/grpc/v2_metrics.go +++ b/node/grpc/metrics_v2.go @@ -14,8 +14,8 @@ import ( const namespace = "eigenda_node" -// V2Metrics encapsulates metrics for the v2 DA node. -type V2Metrics struct { +// MetricsV2 encapsulates metrics for the v2 DA node. +type MetricsV2 struct { logger logging.Logger registry *prometheus.Registry @@ -33,13 +33,13 @@ type V2Metrics struct { isAlive *atomic.Bool } -// NewV2Metrics creates a new V2Metrics instance. dbSizePollPeriod is the period at which the database size is polled. +// NewV2Metrics creates a new MetricsV2 instance. dbSizePollPeriod is the period at which the database size is polled. // If set to 0, the database size is not polled. func NewV2Metrics( logger logging.Logger, registry *prometheus.Registry, dbDir string, - dbSizePollPeriod time.Duration) (*V2Metrics, error) { + dbSizePollPeriod time.Duration) (*MetricsV2, error) { // These should be re-enabled once the legacy v1 metrics are removed. //registry.MustRegister(collectors.NewProcessCollector(collectors.ProcessCollectorOpts{})) @@ -100,7 +100,7 @@ func NewV2Metrics( isAlive := &atomic.Bool{} isAlive.Store(true) - return &V2Metrics{ + return &MetricsV2{ logger: logger, registry: registry, grpcServerOption: grpcServerOption, @@ -116,7 +116,7 @@ func NewV2Metrics( } // Start starts the metrics server. -func (m *V2Metrics) Start() { +func (m *MetricsV2) Start() { if m.dbSizePollPeriod.Nanoseconds() == 0 { return } @@ -147,29 +147,29 @@ func (m *V2Metrics) Start() { } // Stop stops the metrics server. -func (m *V2Metrics) Stop() { +func (m *MetricsV2) Stop() { m.isAlive.Store(false) } // GetGRPCServerOption returns the gRPC server option that enables automatic GRPC metrics collection. -func (m *V2Metrics) GetGRPCServerOption() grpc.ServerOption { +func (m *MetricsV2) GetGRPCServerOption() grpc.ServerOption { return m.grpcServerOption } -func (m *V2Metrics) ReportStoreChunksLatency(latency time.Duration) { +func (m *MetricsV2) ReportStoreChunksLatency(latency time.Duration) { m.storeChunksLatency.WithLabelValues().Observe( float64(latency.Nanoseconds()) / float64(time.Millisecond)) } -func (m *V2Metrics) ReportStoreChunksDataSize(size uint64) { +func (m *MetricsV2) ReportStoreChunksDataSize(size uint64) { m.storeChunksDataSize.WithLabelValues().Set(float64(size)) } -func (m *V2Metrics) ReportGetChunksLatency(latency time.Duration) { +func (m *MetricsV2) ReportGetChunksLatency(latency time.Duration) { m.getChunksLatency.WithLabelValues().Observe( float64(latency.Nanoseconds()) / float64(time.Millisecond)) } -func (m *V2Metrics) ReportGetChunksDataSize(size int) { +func (m *MetricsV2) ReportGetChunksDataSize(size int) { m.getChunksDataSize.WithLabelValues().Set(float64(size)) } diff --git a/node/grpc/server_v2.go b/node/grpc/server_v2.go index 0c34e3535..97019b19a 100644 --- a/node/grpc/server_v2.go +++ b/node/grpc/server_v2.go @@ -27,7 +27,7 @@ type ServerV2 struct { node *node.Node ratelimiter common.RateLimiter logger logging.Logger - metrics *V2Metrics + metrics *MetricsV2 } // NewServerV2 creates a new Server instance with the provided parameters. @@ -190,8 +190,7 @@ func (s *ServerV2) GetChunks(ctx context.Context, in *pb.GetChunksRequest) (*pb. } s.metrics.ReportGetChunksDataSize(size) - elapsed := time.Since(start) - s.metrics.ReportGetChunksLatency(elapsed) + s.metrics.ReportGetChunksLatency(time.Since(start)) return &pb.GetChunksReply{ Chunks: chunks, diff --git a/node/store_v2.go b/node/store_v2.go index 03b7e30fe..d7ff2c19f 100644 --- a/node/store_v2.go +++ b/node/store_v2.go @@ -19,10 +19,19 @@ const ( ) type StoreV2 interface { + // StoreBatch stores a batch and its raw bundles in the database. Returns the keys of the stored data // and the size of the stored data, in bytes. + // + // All modifications to the database within this method are performed atomically. StoreBatch(batch *corev2.Batch, rawBundles []*RawBundles) ([]kvstore.Key, uint64, error) + + // DeleteKeys deletes the keys from local storage. + // + // All modifications to the database within this method are performed atomically. DeleteKeys(keys []kvstore.Key) error + + // GetChunks returns the chunks of a blob with the given blob key and quorum. GetChunks(blobKey corev2.BlobKey, quorum core.QuorumID) ([][]byte, error) } From 864e7d05c3933213ed961949e5c9977117ed5bcd Mon Sep 17 00:00:00 2001 From: Cody Littley Date: Fri, 13 Dec 2024 13:51:29 -0600 Subject: [PATCH 30/30] Made suggested changes. Signed-off-by: Cody Littley --- common/common.go | 7 ++ disperser/apiserver/metrics_v2.go | 13 +-- disperser/controller/dispatcher_metrics.go | 37 +++---- .../controller/encoding_manager_metrics.go | 11 ++- node/config.go | 2 - node/flags/flags.go | 9 -- node/grpc/metrics_v2.go | 98 ++++--------------- node/grpc/run.go | 2 - node/grpc/server_v2.go | 4 +- relay/cache/cache_accessor_metrics.go | 3 +- relay/metrics/metrics.go | 16 +-- 11 files changed, 68 insertions(+), 134 deletions(-) diff --git a/common/common.go b/common/common.go index 90baeb87e..1105f0ca0 100644 --- a/common/common.go +++ b/common/common.go @@ -3,6 +3,7 @@ package common import ( "bytes" "crypto/sha256" + "time" "unsafe" "github.com/fxamacker/cbor/v2" @@ -62,3 +63,9 @@ func DecodeFromBytes[T any](b []byte) (T, error) { } return t, nil } + +// ToMilliseconds converts the given duration to milliseconds. Unlike duration.Milliseconds(), this function returns +// a float64 with nanosecond precision (at least, as much precision as floating point numbers allow). +func ToMilliseconds(duration time.Duration) float64 { + return float64(duration.Nanoseconds()) / float64(time.Millisecond) +} diff --git a/disperser/apiserver/metrics_v2.go b/disperser/apiserver/metrics_v2.go index 25f9e7fd3..9273aa9d5 100644 --- a/disperser/apiserver/metrics_v2.go +++ b/disperser/apiserver/metrics_v2.go @@ -1,6 +1,7 @@ package apiserver import ( + "github.com/Layr-Labs/eigenda/common" grpcprom "github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" @@ -116,15 +117,15 @@ func newAPIServerV2Metrics(registry *prometheus.Registry) *metricsV2 { } func (m *metricsV2) reportGetBlobCommitmentLatency(duration time.Duration) { - m.getBlobCommitmentLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.getBlobCommitmentLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *metricsV2) reportGetPaymentStateLatency(duration time.Duration) { - m.getPaymentStateLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.getPaymentStateLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *metricsV2) reportDisperseBlobLatency(duration time.Duration) { - m.disperseBlobLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.disperseBlobLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *metricsV2) reportDisperseBlobSize(size int) { @@ -133,13 +134,13 @@ func (m *metricsV2) reportDisperseBlobSize(size int) { func (m *metricsV2) reportValidateDispersalRequestLatency(duration time.Duration) { m.validateDispersalRequestLatency.WithLabelValues().Observe( - float64(duration.Nanoseconds()) / float64(time.Millisecond)) + common.ToMilliseconds(duration)) } func (m *metricsV2) reportStoreBlobLatency(duration time.Duration) { - m.storeBlobLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.storeBlobLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *metricsV2) reportGetBlobStatusLatency(duration time.Duration) { - m.getBlobStatusLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.getBlobStatusLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } diff --git a/disperser/controller/dispatcher_metrics.go b/disperser/controller/dispatcher_metrics.go index f8a667a6e..148342198 100644 --- a/disperser/controller/dispatcher_metrics.go +++ b/disperser/controller/dispatcher_metrics.go @@ -1,6 +1,7 @@ package controller import ( + "github.com/Layr-Labs/eigenda/common" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "time" @@ -248,51 +249,51 @@ func newDispatcherMetrics(registry *prometheus.Registry) *dispatcherMetrics { } func (m *dispatcherMetrics) reportHandleBatchLatency(duration time.Duration) { - m.handleBatchLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.handleBatchLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportNewBatchLatency(duration time.Duration) { - m.newBatchLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.newBatchLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportGetBlobMetadataLatency(duration time.Duration) { - m.getBlobMetadataLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.getBlobMetadataLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportGetOperatorStateLatency(duration time.Duration) { - m.getOperatorStateLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.getOperatorStateLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportGetBlobCertificatesLatency(duration time.Duration) { - m.getBlobCertificatesLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.getBlobCertificatesLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportBuildMerkleTreeLatency(duration time.Duration) { - m.buildMerkleTreeLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.buildMerkleTreeLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportPutBatchHeaderLatency(duration time.Duration) { - m.putBatchHeaderLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.putBatchHeaderLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportProofLatency(duration time.Duration) { - m.proofLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.proofLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportPutVerificationInfosLatency(duration time.Duration) { - m.putVerificationInfosLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.putVerificationInfosLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportPoolSubmissionLatency(duration time.Duration) { - m.poolSubmissionLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.poolSubmissionLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportPutDispersalRequestLatency(duration time.Duration) { - m.putDispersalRequestLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.putDispersalRequestLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportSendChunksLatency(duration time.Duration) { - m.sendChunksLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.sendChunksLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportSendChunksRetryCount(retries float64) { @@ -300,25 +301,25 @@ func (m *dispatcherMetrics) reportSendChunksRetryCount(retries float64) { } func (m *dispatcherMetrics) reportPutDispersalResponseLatency(duration time.Duration) { - m.putDispersalResponseLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.putDispersalResponseLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportHandleSignaturesLatency(duration time.Duration) { - m.handleSignaturesLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.handleSignaturesLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportReceiveSignaturesLatency(duration time.Duration) { - m.receiveSignaturesLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.receiveSignaturesLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportAggregateSignaturesLatency(duration time.Duration) { - m.aggregateSignaturesLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.aggregateSignaturesLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportPutAttestationLatency(duration time.Duration) { - m.putAttestationLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.putAttestationLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *dispatcherMetrics) reportUpdateBatchStatusLatency(duration time.Duration) { - m.updateBatchStatusLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.updateBatchStatusLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } diff --git a/disperser/controller/encoding_manager_metrics.go b/disperser/controller/encoding_manager_metrics.go index 28c34a0ed..791193203 100644 --- a/disperser/controller/encoding_manager_metrics.go +++ b/disperser/controller/encoding_manager_metrics.go @@ -1,6 +1,7 @@ package controller import ( + common "github.com/Layr-Labs/eigenda/common" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "time" @@ -123,23 +124,23 @@ func newEncodingManagerMetrics(registry *prometheus.Registry) *encodingManagerMe } func (m *encodingManagerMetrics) reportBatchSubmissionLatency(duration time.Duration) { - m.batchSubmissionLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.batchSubmissionLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *encodingManagerMetrics) reportBlobHandleLatency(duration time.Duration) { - m.blobHandleLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.blobHandleLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *encodingManagerMetrics) reportEncodingLatency(duration time.Duration) { - m.encodingLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.encodingLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *encodingManagerMetrics) reportPutBlobCertLatency(duration time.Duration) { - m.putBlobCertLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.putBlobCertLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *encodingManagerMetrics) reportUpdateBlobStatusLatency(duration time.Duration) { - m.updateBlobStatusLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.updateBlobStatusLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *encodingManagerMetrics) reportBatchSize(size int) { diff --git a/node/config.go b/node/config.go index d1b15fd1d..7bf051efd 100644 --- a/node/config.go +++ b/node/config.go @@ -62,7 +62,6 @@ type Config struct { OverrideStoreDurationBlocks int64 QuorumIDList []core.QuorumID DbPath string - DBSizePollPeriod time.Duration LogPath string PrivateBls string ID core.OperatorID @@ -219,7 +218,6 @@ func NewConfig(ctx *cli.Context) (*Config, error) { OverrideStoreDurationBlocks: ctx.GlobalInt64(flags.OverrideStoreDurationBlocksFlag.Name), QuorumIDList: ids, DbPath: ctx.GlobalString(flags.DbPathFlag.Name), - DBSizePollPeriod: ctx.GlobalDuration(flags.MetricsDBSizePollPeriodFlag.Name), PrivateBls: privateBls, EthClientConfig: ethClientConfig, EncoderConfig: kzg.ReadCLIConfig(ctx), diff --git a/node/flags/flags.go b/node/flags/flags.go index 010edfffb..79e1e7feb 100644 --- a/node/flags/flags.go +++ b/node/flags/flags.go @@ -98,14 +98,6 @@ var ( Required: true, EnvVar: common.PrefixEnvVar(EnvVarPrefix, "DB_PATH"), } - MetricsDBSizePollPeriodFlag = cli.DurationFlag{ - Name: common.PrefixFlag(FlagPrefix, "metrics-db-size-poll-period"), - Usage: "The period at which the database size is polled for metrics. " + - "If set to 0, the database size is not polled.", - Required: false, - Value: 10 * time.Minute, - EnvVar: common.PrefixEnvVar(EnvVarPrefix, "METRICS_DB_SIZE_POLL_PERIOD"), - } // The files for encrypted private keys. BlsKeyFileFlag = cli.StringFlag{ Name: common.PrefixFlag(FlagPrefix, "bls-key-file"), @@ -392,7 +384,6 @@ var optionalFlags = []cli.Flag{ ChunkDownloadTimeoutFlag, PprofHttpPort, EnablePprof, - MetricsDBSizePollPeriodFlag, } func init() { diff --git a/node/grpc/metrics_v2.go b/node/grpc/metrics_v2.go index 973e1178b..6dcfb52d4 100644 --- a/node/grpc/metrics_v2.go +++ b/node/grpc/metrics_v2.go @@ -1,14 +1,12 @@ package grpc import ( + "github.com/Layr-Labs/eigenda/common" "github.com/Layr-Labs/eigensdk-go/logging" grpcprom "github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "google.golang.org/grpc" - "os" - "path/filepath" - "sync/atomic" "time" ) @@ -21,25 +19,16 @@ type MetricsV2 struct { registry *prometheus.Registry grpcServerOption grpc.ServerOption - storeChunksLatency *prometheus.SummaryVec - storeChunksDataSize *prometheus.GaugeVec + storeChunksLatency *prometheus.SummaryVec + storeChunksRequestSize *prometheus.GaugeVec getChunksLatency *prometheus.SummaryVec getChunksDataSize *prometheus.GaugeVec - - dbSize *prometheus.GaugeVec - dbSizePollPeriod time.Duration - dbDir string - isAlive *atomic.Bool } // NewV2Metrics creates a new MetricsV2 instance. dbSizePollPeriod is the period at which the database size is polled. // If set to 0, the database size is not polled. -func NewV2Metrics( - logger logging.Logger, - registry *prometheus.Registry, - dbDir string, - dbSizePollPeriod time.Duration) (*MetricsV2, error) { +func NewV2Metrics(logger logging.Logger, registry *prometheus.Registry) (*MetricsV2, error) { // These should be re-enabled once the legacy v1 metrics are removed. //registry.MustRegister(collectors.NewProcessCollector(collectors.ProcessCollectorOpts{})) @@ -61,10 +50,10 @@ func NewV2Metrics( []string{}, ) - storeChunksDataSize := promauto.With(registry).NewGaugeVec( + storeChunksRequestSize := promauto.With(registry).NewGaugeVec( prometheus.GaugeOpts{ Namespace: namespace, - Name: "store_chunks_data_size_bytes", + Name: "store_chunks_request_size_bytes", Help: "The size of the data requested to be stored by StoreChunks() RPC calls.", }, []string{}, @@ -89,85 +78,32 @@ func NewV2Metrics( []string{}, ) - dbSize := promauto.With(registry).NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: namespace, - Name: "db_size_bytes", - Help: "The size of the leveldb database.", - }, - []string{}, - ) - isAlive := &atomic.Bool{} - isAlive.Store(true) - return &MetricsV2{ - logger: logger, - registry: registry, - grpcServerOption: grpcServerOption, - storeChunksLatency: storeChunksLatency, - storeChunksDataSize: storeChunksDataSize, - getChunksLatency: getChunksLatency, - getChunksDataSize: getChunksDataSize, - dbSize: dbSize, - dbSizePollPeriod: dbSizePollPeriod, - dbDir: dbDir, - isAlive: isAlive, + logger: logger, + registry: registry, + grpcServerOption: grpcServerOption, + storeChunksLatency: storeChunksLatency, + storeChunksRequestSize: storeChunksRequestSize, + getChunksLatency: getChunksLatency, + getChunksDataSize: getChunksDataSize, }, nil } -// Start starts the metrics server. -func (m *MetricsV2) Start() { - if m.dbSizePollPeriod.Nanoseconds() == 0 { - return - } - go func() { - ticker := time.NewTicker(m.dbSizePollPeriod) - - for m.isAlive.Load() { - var size int64 - err := filepath.Walk(m.dbDir, func(_ string, info os.FileInfo, err error) error { - if err != nil { - return err - } - if !info.IsDir() { - size += info.Size() - } - return err - }) - - if err != nil { - m.logger.Errorf("failed to get database size: %v", err) - } else { - m.dbSize.WithLabelValues().Set(float64(size)) - } - <-ticker.C - } - }() - -} - -// Stop stops the metrics server. -func (m *MetricsV2) Stop() { - m.isAlive.Store(false) -} - // GetGRPCServerOption returns the gRPC server option that enables automatic GRPC metrics collection. func (m *MetricsV2) GetGRPCServerOption() grpc.ServerOption { return m.grpcServerOption } func (m *MetricsV2) ReportStoreChunksLatency(latency time.Duration) { - m.storeChunksLatency.WithLabelValues().Observe( - float64(latency.Nanoseconds()) / float64(time.Millisecond)) + m.storeChunksLatency.WithLabelValues().Observe(common.ToMilliseconds(latency)) } -func (m *MetricsV2) ReportStoreChunksDataSize(size uint64) { - m.storeChunksDataSize.WithLabelValues().Set(float64(size)) +func (m *MetricsV2) ReportStoreChunksRequestSize(size uint64) { + m.storeChunksRequestSize.WithLabelValues().Set(float64(size)) } func (m *MetricsV2) ReportGetChunksLatency(latency time.Duration) { - m.getChunksLatency.WithLabelValues().Observe( - float64(latency.Nanoseconds()) / float64(time.Millisecond)) + m.getChunksLatency.WithLabelValues().Observe(common.ToMilliseconds(latency)) } func (m *MetricsV2) ReportGetChunksDataSize(size int) { diff --git a/node/grpc/run.go b/node/grpc/run.go index 3f0a90073..cbcb2d74c 100644 --- a/node/grpc/run.go +++ b/node/grpc/run.go @@ -24,8 +24,6 @@ func RunServers(serverV1 *Server, serverV2 *ServerV2, config *node.Config, logge return errors.New("node V2 server is not configured") } - serverV2.metrics.Start() - go func() { for { addr := fmt.Sprintf("%s:%s", localhost, config.InternalDispersalPort) diff --git a/node/grpc/server_v2.go b/node/grpc/server_v2.go index 97019b19a..620267cd0 100644 --- a/node/grpc/server_v2.go +++ b/node/grpc/server_v2.go @@ -38,7 +38,7 @@ func NewServerV2( ratelimiter common.RateLimiter, registry *prometheus.Registry) (*ServerV2, error) { - metrics, err := NewV2Metrics(logger, registry, config.DbPath, config.DBSizePollPeriod) + metrics, err := NewV2Metrics(logger, registry) if err != nil { return nil, err } @@ -112,7 +112,7 @@ func (s *ServerV2) StoreChunks(ctx context.Context, in *pb.StoreChunksRequest) ( return } - s.metrics.ReportStoreChunksDataSize(size) + s.metrics.ReportStoreChunksRequestSize(size) storeChan <- storeResult{ keys: keys, diff --git a/relay/cache/cache_accessor_metrics.go b/relay/cache/cache_accessor_metrics.go index e5ef45621..c873f2567 100644 --- a/relay/cache/cache_accessor_metrics.go +++ b/relay/cache/cache_accessor_metrics.go @@ -2,6 +2,7 @@ package cache import ( "fmt" + "github.com/Layr-Labs/eigenda/common" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "time" @@ -110,5 +111,5 @@ func (m *CacheAccessorMetrics) ReportAverageWeight(averageWeight float64) { } func (m *CacheAccessorMetrics) ReportCacheMissLatency(duration time.Duration) { - m.cacheMissLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.cacheMissLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } diff --git a/relay/metrics/metrics.go b/relay/metrics/metrics.go index faa19d1f7..702b7c7a0 100644 --- a/relay/metrics/metrics.go +++ b/relay/metrics/metrics.go @@ -2,6 +2,7 @@ package metrics import ( "fmt" + "github.com/Layr-Labs/eigenda/common" "github.com/Layr-Labs/eigenda/relay/cache" "github.com/Layr-Labs/eigensdk-go/logging" grpcprom "github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus" @@ -244,20 +245,19 @@ func (m *RelayMetrics) GetGRPCServerOption() grpc.ServerOption { } func (m *RelayMetrics) ReportChunkLatency(duration time.Duration) { - m.getChunksLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.getChunksLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *RelayMetrics) ReportChunkAuthenticationLatency(duration time.Duration) { - m.getChunksAuthenticationLatency.WithLabelValues().Observe( - float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.getChunksAuthenticationLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *RelayMetrics) ReportChunkMetadataLatency(duration time.Duration) { - m.getChunksMetadataLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.getChunksMetadataLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *RelayMetrics) ReportChunkDataLatency(duration time.Duration) { - m.getChunksDataLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.getChunksDataLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *RelayMetrics) ReportChunkAuthFailure() { @@ -277,15 +277,15 @@ func (m *RelayMetrics) ReportChunkDataSize(size int) { } func (m *RelayMetrics) ReportBlobLatency(duration time.Duration) { - m.getBlobLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.getBlobLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *RelayMetrics) ReportBlobMetadataLatency(duration time.Duration) { - m.getBlobMetadataLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.getBlobMetadataLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *RelayMetrics) ReportBlobDataLatency(duration time.Duration) { - m.getBlobDataLatency.WithLabelValues().Observe(float64(duration.Nanoseconds()) / float64(time.Millisecond)) + m.getBlobDataLatency.WithLabelValues().Observe(common.ToMilliseconds(duration)) } func (m *RelayMetrics) ReportBlobRateLimited(reason string) {