From 36a6794ea27ac9df65705605aad1da00c5fd5f9c Mon Sep 17 00:00:00 2001 From: Michal Pristas Date: Fri, 3 Nov 2023 20:55:18 +0100 Subject: [PATCH] Make replication stats return whole number (#28824) **Description:** I failed to reproduce []uint8 to int64 conversion but I was able to repro float64 to int64 conversion error. Different types may be due to different versions or values reported. The fix is forcing query to retrieve integer values. While this may seem like most obvious fix I'm not really aligned with it. What query is returning for is a lag as a decimal number (whole part is seconds) by forcing this to return just an int we kind of losing precision. `0.4s` are reported as `0` while it is `400ms`. My proposal here would consists of 2 options. First one is change reporting in a way that what we report is in fact time-span in `ms`. This could most likely be considered breaking. Second option (I'm more in favor of) is to change the type of what is reported (from int to float). This way unit is intact and does not break possible visualizations, but we gain precision and won't lose data. My first issue here so I wanted to get some feedback first before publishing something unreasonable. _EDIT_ Went with the option of deprecating metrics with second precision (still fixing conversion failures) and introducing alternative to these metrics with `_ms` suffix in name and millisecond precision. Old metrics are now behind a featuregate which is enabled by default for now. **Link to tracking Issue:** #26714 **Testing:** Setting up replicated postgres instances and testing method against this deployment. **Documentation:** - --------- Co-authored-by: Daniel Jaglowski --- .../replication-stats-conversion-failure.yaml | 27 + .../loadscraper/load_scraper_windows.go | 1 - receiver/postgresqlreceiver/client.go | 70 +- receiver/postgresqlreceiver/documentation.md | 18 + receiver/postgresqlreceiver/go.mod | 5 +- .../internal/metadata/generated_config.go | 4 + .../metadata/generated_config_test.go | 2 + .../internal/metadata/generated_metrics.go | 60 + .../metadata/generated_metrics_test.go | 21 + .../internal/metadata/testdata/config.yaml | 4 + receiver/postgresqlreceiver/metadata.yaml | 10 + receiver/postgresqlreceiver/scraper.go | 28 +- receiver/postgresqlreceiver/scraper_test.go | 44 +- .../multiple/expected_precise_lag.yaml | 1602 +++++++++++++++++ 14 files changed, 1876 insertions(+), 20 deletions(-) create mode 100755 .chloggen/replication-stats-conversion-failure.yaml create mode 100644 receiver/postgresqlreceiver/testdata/scraper/multiple/expected_precise_lag.yaml diff --git a/.chloggen/replication-stats-conversion-failure.yaml b/.chloggen/replication-stats-conversion-failure.yaml new file mode 100755 index 000000000000..4306ad92e81c --- /dev/null +++ b/.chloggen/replication-stats-conversion-failure.yaml @@ -0,0 +1,27 @@ +# Use this changelog template to create an entry for release notes. + +# One of 'breaking', 'deprecation', 'new_component', 'enhancement', 'bug_fix' +change_type: deprecation + +# The name of the component, or a single word describing the area of concern, (e.g. filelogreceiver) +component: postgresqlreceiver + +# A brief description of the change. Surround your text with quotes ("") if it needs to start with a backtick (`). +note: "Deprecation of postgresql replication lag metrics `postgresql.wal.lag` in favor of more precise 'postgresql.wal.delay'" + +# Mandatory: One or more tracking issues related to the change. You can use the PR number here if no issue exists. +issues: [26714] + +# (Optional) One or more lines of additional information to render under the primary note. +# These lines will be padded with 2 spaces and then inserted directly into the document. +# Use pipe (|) for multiline entries. +subtext: + +# If your change doesn't affect end users or the exported elements of any package, +# you should instead start your pull request title with [chore] or use the "Skip Changelog" label. +# Optional: The change log or logs in which this entry should be included. +# e.g. '[user]' or '[user, api]' +# Include 'user' if the change is relevant to end users. +# Include 'api' if there is a change to a library API. +# Default: '[user]' +change_logs: [user] diff --git a/receiver/hostmetricsreceiver/internal/scraper/loadscraper/load_scraper_windows.go b/receiver/hostmetricsreceiver/internal/scraper/loadscraper/load_scraper_windows.go index c0556569c3e6..ceca1f1ec9f8 100644 --- a/receiver/hostmetricsreceiver/internal/scraper/loadscraper/load_scraper_windows.go +++ b/receiver/hostmetricsreceiver/internal/scraper/loadscraper/load_scraper_windows.go @@ -140,7 +140,6 @@ func stopSampling(_ context.Context) error { // no load scraper is running nothing to do return nil } - // only stop sampling if all load scrapers have been closed scraperCount-- if scraperCount > 0 { diff --git a/receiver/postgresqlreceiver/client.go b/receiver/postgresqlreceiver/client.go index 729c5093f333..4a56fee3bd4a 100644 --- a/receiver/postgresqlreceiver/client.go +++ b/receiver/postgresqlreceiver/client.go @@ -15,9 +15,19 @@ import ( "github.com/lib/pq" "go.opentelemetry.io/collector/config/confignet" "go.opentelemetry.io/collector/config/configtls" + "go.opentelemetry.io/collector/featuregate" "go.uber.org/multierr" ) +const lagMetricsInSecondsFeatureGateID = "postgresqlreceiver.preciselagmetrics" + +var preciseLagMetricsFg = featuregate.GlobalRegistry().MustRegister( + lagMetricsInSecondsFeatureGateID, + featuregate.StageAlpha, + featuregate.WithRegisterDescription("Metric `postgresql.wal.lag` is replaced by more precise `postgresql.wal.delay`."), + featuregate.WithRegisterFromVersion("0.89.0"), +) + // databaseName is a name that refers to a database so that it can be uniquely referred to later // i.e. database1 type databaseName string @@ -484,18 +494,63 @@ func (c *postgreSQLClient) getMaxConnections(ctx context.Context) (int64, error) type replicationStats struct { clientAddr string pendingBytes int64 - flushLag int64 - replayLag int64 - writeLag int64 + flushLagInt int64 // Deprecated + replayLagInt int64 // Deprecated + writeLagInt int64 // Deprecated + flushLag float64 + replayLag float64 + writeLag float64 +} + +func (c *postgreSQLClient) getDeprecatedReplicationStats(ctx context.Context) ([]replicationStats, error) { + query := `SELECT + client_addr, + coalesce(pg_wal_lsn_diff(pg_current_wal_lsn(), replay_lsn), -1) AS replication_bytes_pending, + extract('epoch' from coalesce(write_lag, '-1 seconds'))::integer, + extract('epoch' from coalesce(flush_lag, '-1 seconds'))::integer, + extract('epoch' from coalesce(replay_lag, '-1 seconds'))::integer + FROM pg_stat_replication; + ` + rows, err := c.client.QueryContext(ctx, query) + if err != nil { + return nil, fmt.Errorf("unable to query pg_stat_replication: %w", err) + } + defer rows.Close() + var rs []replicationStats + var errors error + for rows.Next() { + var client string + var replicationBytes int64 + var writeLagInt, flushLagInt, replayLagInt int64 + err = rows.Scan(&client, &replicationBytes, + &writeLagInt, &flushLagInt, &replayLagInt) + if err != nil { + errors = multierr.Append(errors, err) + continue + } + rs = append(rs, replicationStats{ + clientAddr: client, + pendingBytes: replicationBytes, + replayLagInt: replayLagInt, + writeLagInt: writeLagInt, + flushLagInt: flushLagInt, + }) + } + + return rs, errors } func (c *postgreSQLClient) getReplicationStats(ctx context.Context) ([]replicationStats, error) { + if !preciseLagMetricsFg.IsEnabled() { + return c.getDeprecatedReplicationStats(ctx) + } + query := `SELECT client_addr, coalesce(pg_wal_lsn_diff(pg_current_wal_lsn(), replay_lsn), -1) AS replication_bytes_pending, - extract('epoch' from coalesce(write_lag, '-1 seconds')), - extract('epoch' from coalesce(flush_lag, '-1 seconds')), - extract('epoch' from coalesce(replay_lag, '-1 seconds')) + extract('epoch' from coalesce(write_lag, '-1 seconds'))::decimal AS write_lag_fractional, + extract('epoch' from coalesce(flush_lag, '-1 seconds'))::decimal AS flush_lag_fractional, + extract('epoch' from coalesce(replay_lag, '-1 seconds'))::decimal AS replay_lag_fractional FROM pg_stat_replication; ` rows, err := c.client.QueryContext(ctx, query) @@ -507,7 +562,8 @@ func (c *postgreSQLClient) getReplicationStats(ctx context.Context) ([]replicati var errors error for rows.Next() { var client string - var replicationBytes, writeLag, flushLag, replayLag int64 + var replicationBytes int64 + var writeLag, flushLag, replayLag float64 err = rows.Scan(&client, &replicationBytes, &writeLag, &flushLag, &replayLag) if err != nil { errors = multierr.Append(errors, err) diff --git a/receiver/postgresqlreceiver/documentation.md b/receiver/postgresqlreceiver/documentation.md index dbe3c7b15208..8e4d259d5357 100644 --- a/receiver/postgresqlreceiver/documentation.md +++ b/receiver/postgresqlreceiver/documentation.md @@ -293,6 +293,24 @@ The number of temp files. | ---- | ----------- | ---------- | ----------------------- | --------- | | {temp_file} | Sum | Int | Cumulative | true | +### postgresql.wal.delay + +Time between flushing recent WAL locally and receiving notification that the standby server has completed an operation with it. + +This metric requires WAL to be enabled with at least one replica. + + +| Unit | Metric Type | Value Type | +| ---- | ----------- | ---------- | +| s | Gauge | Double | + +#### Attributes + +| Name | Description | Values | +| ---- | ----------- | ------ | +| operation | The operation which is responsible for the lag. | Str: ``flush``, ``replay``, ``write`` | +| replication_client | The IP address of the client connected to this backend. If this field is "unix", it indicates either that the client is connected via a Unix socket. | Any Str | + ## Resource Attributes | Name | Description | Values | Enabled | diff --git a/receiver/postgresqlreceiver/go.mod b/receiver/postgresqlreceiver/go.mod index 8dae28666702..3e8326c1d847 100644 --- a/receiver/postgresqlreceiver/go.mod +++ b/receiver/postgresqlreceiver/go.mod @@ -5,6 +5,7 @@ go 1.20 require ( github.com/google/go-cmp v0.6.0 github.com/lib/pq v1.10.9 + github.com/open-telemetry/opentelemetry-collector-contrib/internal/common v0.88.0 github.com/open-telemetry/opentelemetry-collector-contrib/internal/coreinternal v0.88.0 github.com/open-telemetry/opentelemetry-collector-contrib/pkg/golden v0.0.0-00010101000000-000000000000 github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest v0.88.0 @@ -16,6 +17,7 @@ require ( go.opentelemetry.io/collector/config/configtls v0.88.1-0.20231026220224-6405e152a2d9 go.opentelemetry.io/collector/confmap v0.88.1-0.20231026220224-6405e152a2d9 go.opentelemetry.io/collector/consumer v0.88.1-0.20231026220224-6405e152a2d9 + go.opentelemetry.io/collector/featuregate v1.0.0-rcv0017.0.20231026220224-6405e152a2d9 go.opentelemetry.io/collector/pdata v1.0.0-rcv0017.0.20231026220224-6405e152a2d9 go.opentelemetry.io/collector/receiver v0.88.1-0.20231026220224-6405e152a2d9 go.uber.org/multierr v1.11.0 @@ -75,7 +77,6 @@ require ( go.opencensus.io v0.24.0 // indirect go.opentelemetry.io/collector v0.88.1-0.20231026220224-6405e152a2d9 // indirect go.opentelemetry.io/collector/config/configtelemetry v0.88.1-0.20231026220224-6405e152a2d9 // indirect - go.opentelemetry.io/collector/featuregate v1.0.0-rcv0017.0.20231026220224-6405e152a2d9 // indirect go.opentelemetry.io/otel v1.19.0 // indirect go.opentelemetry.io/otel/metric v1.19.0 // indirect go.opentelemetry.io/otel/trace v1.19.0 // indirect @@ -95,6 +96,8 @@ replace github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest replace github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil => ../../pkg/pdatautil +replace github.com/open-telemetry/opentelemetry-collector-contrib/internal/common => ../../internal/common + replace github.com/open-telemetry/opentelemetry-collector-contrib/internal/coreinternal => ../../internal/coreinternal retract ( diff --git a/receiver/postgresqlreceiver/internal/metadata/generated_config.go b/receiver/postgresqlreceiver/internal/metadata/generated_config.go index 209e9ae55553..9393c9ccb7dc 100644 --- a/receiver/postgresqlreceiver/internal/metadata/generated_config.go +++ b/receiver/postgresqlreceiver/internal/metadata/generated_config.go @@ -50,6 +50,7 @@ type MetricsConfig struct { PostgresqlTableVacuumCount MetricConfig `mapstructure:"postgresql.table.vacuum.count"` PostgresqlTempFiles MetricConfig `mapstructure:"postgresql.temp_files"` PostgresqlWalAge MetricConfig `mapstructure:"postgresql.wal.age"` + PostgresqlWalDelay MetricConfig `mapstructure:"postgresql.wal.delay"` PostgresqlWalLag MetricConfig `mapstructure:"postgresql.wal.lag"` } @@ -130,6 +131,9 @@ func DefaultMetricsConfig() MetricsConfig { PostgresqlWalAge: MetricConfig{ Enabled: true, }, + PostgresqlWalDelay: MetricConfig{ + Enabled: false, + }, PostgresqlWalLag: MetricConfig{ Enabled: true, }, diff --git a/receiver/postgresqlreceiver/internal/metadata/generated_config_test.go b/receiver/postgresqlreceiver/internal/metadata/generated_config_test.go index f4420d453fc1..959994b81a87 100644 --- a/receiver/postgresqlreceiver/internal/metadata/generated_config_test.go +++ b/receiver/postgresqlreceiver/internal/metadata/generated_config_test.go @@ -51,6 +51,7 @@ func TestMetricsBuilderConfig(t *testing.T) { PostgresqlTableVacuumCount: MetricConfig{Enabled: true}, PostgresqlTempFiles: MetricConfig{Enabled: true}, PostgresqlWalAge: MetricConfig{Enabled: true}, + PostgresqlWalDelay: MetricConfig{Enabled: true}, PostgresqlWalLag: MetricConfig{Enabled: true}, }, ResourceAttributes: ResourceAttributesConfig{ @@ -89,6 +90,7 @@ func TestMetricsBuilderConfig(t *testing.T) { PostgresqlTableVacuumCount: MetricConfig{Enabled: false}, PostgresqlTempFiles: MetricConfig{Enabled: false}, PostgresqlWalAge: MetricConfig{Enabled: false}, + PostgresqlWalDelay: MetricConfig{Enabled: false}, PostgresqlWalLag: MetricConfig{Enabled: false}, }, ResourceAttributes: ResourceAttributesConfig{ diff --git a/receiver/postgresqlreceiver/internal/metadata/generated_metrics.go b/receiver/postgresqlreceiver/internal/metadata/generated_metrics.go index 9a7d56df7e1c..8ac280a050c0 100644 --- a/receiver/postgresqlreceiver/internal/metadata/generated_metrics.go +++ b/receiver/postgresqlreceiver/internal/metadata/generated_metrics.go @@ -1520,6 +1520,58 @@ func newMetricPostgresqlWalAge(cfg MetricConfig) metricPostgresqlWalAge { return m } +type metricPostgresqlWalDelay struct { + data pmetric.Metric // data buffer for generated metric. + config MetricConfig // metric config provided by user. + capacity int // max observed number of data points added to the metric. +} + +// init fills postgresql.wal.delay metric with initial data. +func (m *metricPostgresqlWalDelay) init() { + m.data.SetName("postgresql.wal.delay") + m.data.SetDescription("Time between flushing recent WAL locally and receiving notification that the standby server has completed an operation with it.") + m.data.SetUnit("s") + m.data.SetEmptyGauge() + m.data.Gauge().DataPoints().EnsureCapacity(m.capacity) +} + +func (m *metricPostgresqlWalDelay) recordDataPoint(start pcommon.Timestamp, ts pcommon.Timestamp, val float64, walOperationLagAttributeValue string, replicationClientAttributeValue string) { + if !m.config.Enabled { + return + } + dp := m.data.Gauge().DataPoints().AppendEmpty() + dp.SetStartTimestamp(start) + dp.SetTimestamp(ts) + dp.SetDoubleValue(val) + dp.Attributes().PutStr("operation", walOperationLagAttributeValue) + dp.Attributes().PutStr("replication_client", replicationClientAttributeValue) +} + +// updateCapacity saves max length of data point slices that will be used for the slice capacity. +func (m *metricPostgresqlWalDelay) updateCapacity() { + if m.data.Gauge().DataPoints().Len() > m.capacity { + m.capacity = m.data.Gauge().DataPoints().Len() + } +} + +// emit appends recorded metric data to a metrics slice and prepares it for recording another set of data points. +func (m *metricPostgresqlWalDelay) emit(metrics pmetric.MetricSlice) { + if m.config.Enabled && m.data.Gauge().DataPoints().Len() > 0 { + m.updateCapacity() + m.data.MoveTo(metrics.AppendEmpty()) + m.init() + } +} + +func newMetricPostgresqlWalDelay(cfg MetricConfig) metricPostgresqlWalDelay { + m := metricPostgresqlWalDelay{config: cfg} + if cfg.Enabled { + m.data = pmetric.NewMetric() + m.init() + } + return m +} + type metricPostgresqlWalLag struct { data pmetric.Metric // data buffer for generated metric. config MetricConfig // metric config provided by user. @@ -1605,6 +1657,7 @@ type MetricsBuilder struct { metricPostgresqlTableVacuumCount metricPostgresqlTableVacuumCount metricPostgresqlTempFiles metricPostgresqlTempFiles metricPostgresqlWalAge metricPostgresqlWalAge + metricPostgresqlWalDelay metricPostgresqlWalDelay metricPostgresqlWalLag metricPostgresqlWalLag } @@ -1649,6 +1702,7 @@ func NewMetricsBuilder(mbc MetricsBuilderConfig, settings receiver.CreateSetting metricPostgresqlTableVacuumCount: newMetricPostgresqlTableVacuumCount(mbc.Metrics.PostgresqlTableVacuumCount), metricPostgresqlTempFiles: newMetricPostgresqlTempFiles(mbc.Metrics.PostgresqlTempFiles), metricPostgresqlWalAge: newMetricPostgresqlWalAge(mbc.Metrics.PostgresqlWalAge), + metricPostgresqlWalDelay: newMetricPostgresqlWalDelay(mbc.Metrics.PostgresqlWalDelay), metricPostgresqlWalLag: newMetricPostgresqlWalLag(mbc.Metrics.PostgresqlWalLag), } for _, op := range options { @@ -1736,6 +1790,7 @@ func (mb *MetricsBuilder) EmitForResource(rmo ...ResourceMetricsOption) { mb.metricPostgresqlTableVacuumCount.emit(ils.Metrics()) mb.metricPostgresqlTempFiles.emit(ils.Metrics()) mb.metricPostgresqlWalAge.emit(ils.Metrics()) + mb.metricPostgresqlWalDelay.emit(ils.Metrics()) mb.metricPostgresqlWalLag.emit(ils.Metrics()) for _, op := range rmo { @@ -1882,6 +1937,11 @@ func (mb *MetricsBuilder) RecordPostgresqlWalAgeDataPoint(ts pcommon.Timestamp, mb.metricPostgresqlWalAge.recordDataPoint(mb.startTime, ts, val) } +// RecordPostgresqlWalDelayDataPoint adds a data point to postgresql.wal.delay metric. +func (mb *MetricsBuilder) RecordPostgresqlWalDelayDataPoint(ts pcommon.Timestamp, val float64, walOperationLagAttributeValue AttributeWalOperationLag, replicationClientAttributeValue string) { + mb.metricPostgresqlWalDelay.recordDataPoint(mb.startTime, ts, val, walOperationLagAttributeValue.String(), replicationClientAttributeValue) +} + // RecordPostgresqlWalLagDataPoint adds a data point to postgresql.wal.lag metric. func (mb *MetricsBuilder) RecordPostgresqlWalLagDataPoint(ts pcommon.Timestamp, val int64, walOperationLagAttributeValue AttributeWalOperationLag, replicationClientAttributeValue string) { mb.metricPostgresqlWalLag.recordDataPoint(mb.startTime, ts, val, walOperationLagAttributeValue.String(), replicationClientAttributeValue) diff --git a/receiver/postgresqlreceiver/internal/metadata/generated_metrics_test.go b/receiver/postgresqlreceiver/internal/metadata/generated_metrics_test.go index 65a78cac0e48..edc82bc943d6 100644 --- a/receiver/postgresqlreceiver/internal/metadata/generated_metrics_test.go +++ b/receiver/postgresqlreceiver/internal/metadata/generated_metrics_test.go @@ -151,6 +151,9 @@ func TestMetricsBuilder(t *testing.T) { allMetricsCount++ mb.RecordPostgresqlWalAgeDataPoint(ts, 1) + allMetricsCount++ + mb.RecordPostgresqlWalDelayDataPoint(ts, 1, AttributeWalOperationLagFlush, "replication_client-val") + defaultMetricsCount++ allMetricsCount++ mb.RecordPostgresqlWalLagDataPoint(ts, 1, AttributeWalOperationLagFlush, "replication_client-val") @@ -551,6 +554,24 @@ func TestMetricsBuilder(t *testing.T) { assert.Equal(t, ts, dp.Timestamp()) assert.Equal(t, pmetric.NumberDataPointValueTypeInt, dp.ValueType()) assert.Equal(t, int64(1), dp.IntValue()) + case "postgresql.wal.delay": + assert.False(t, validatedMetrics["postgresql.wal.delay"], "Found a duplicate in the metrics slice: postgresql.wal.delay") + validatedMetrics["postgresql.wal.delay"] = true + assert.Equal(t, pmetric.MetricTypeGauge, ms.At(i).Type()) + assert.Equal(t, 1, ms.At(i).Gauge().DataPoints().Len()) + assert.Equal(t, "Time between flushing recent WAL locally and receiving notification that the standby server has completed an operation with it.", ms.At(i).Description()) + assert.Equal(t, "s", ms.At(i).Unit()) + dp := ms.At(i).Gauge().DataPoints().At(0) + assert.Equal(t, start, dp.StartTimestamp()) + assert.Equal(t, ts, dp.Timestamp()) + assert.Equal(t, pmetric.NumberDataPointValueTypeDouble, dp.ValueType()) + assert.Equal(t, float64(1), dp.DoubleValue()) + attrVal, ok := dp.Attributes().Get("operation") + assert.True(t, ok) + assert.EqualValues(t, "flush", attrVal.Str()) + attrVal, ok = dp.Attributes().Get("replication_client") + assert.True(t, ok) + assert.EqualValues(t, "replication_client-val", attrVal.Str()) case "postgresql.wal.lag": assert.False(t, validatedMetrics["postgresql.wal.lag"], "Found a duplicate in the metrics slice: postgresql.wal.lag") validatedMetrics["postgresql.wal.lag"] = true diff --git a/receiver/postgresqlreceiver/internal/metadata/testdata/config.yaml b/receiver/postgresqlreceiver/internal/metadata/testdata/config.yaml index e1bd2600c075..fb387887bce0 100644 --- a/receiver/postgresqlreceiver/internal/metadata/testdata/config.yaml +++ b/receiver/postgresqlreceiver/internal/metadata/testdata/config.yaml @@ -51,6 +51,8 @@ all_set: enabled: true postgresql.wal.age: enabled: true + postgresql.wal.delay: + enabled: true postgresql.wal.lag: enabled: true resource_attributes: @@ -112,6 +114,8 @@ none_set: enabled: false postgresql.wal.age: enabled: false + postgresql.wal.delay: + enabled: false postgresql.wal.lag: enabled: false resource_attributes: diff --git a/receiver/postgresqlreceiver/metadata.yaml b/receiver/postgresqlreceiver/metadata.yaml index 31441030e9a6..a30e0624e9ee 100644 --- a/receiver/postgresqlreceiver/metadata.yaml +++ b/receiver/postgresqlreceiver/metadata.yaml @@ -297,3 +297,13 @@ metrics: value_type: int extended_documentation: | This metric requires WAL to be enabled with at least one replica. + postgresql.wal.delay: + attributes: [wal_operation_lag, replication_client] + description: Time between flushing recent WAL locally and receiving notification that the standby server has completed an operation with it. + enabled: false + unit: s + gauge: + value_type: double + extended_documentation: | + This metric requires WAL to be enabled with at least one replica. + diff --git a/receiver/postgresqlreceiver/scraper.go b/receiver/postgresqlreceiver/scraper.go index 190b7468b26f..31c0eeb0ec6f 100644 --- a/receiver/postgresqlreceiver/scraper.go +++ b/receiver/postgresqlreceiver/scraper.go @@ -313,14 +313,26 @@ func (p *postgreSQLScraper) collectReplicationStats( if rs.pendingBytes >= 0 { p.mb.RecordPostgresqlReplicationDataDelayDataPoint(now, rs.pendingBytes, rs.clientAddr) } - if rs.writeLag >= 0 { - p.mb.RecordPostgresqlWalLagDataPoint(now, rs.writeLag, metadata.AttributeWalOperationLagWrite, rs.clientAddr) - } - if rs.replayLag >= 0 { - p.mb.RecordPostgresqlWalLagDataPoint(now, rs.replayLag, metadata.AttributeWalOperationLagReplay, rs.clientAddr) - } - if rs.flushLag >= 0 { - p.mb.RecordPostgresqlWalLagDataPoint(now, rs.flushLag, metadata.AttributeWalOperationLagFlush, rs.clientAddr) + if preciseLagMetricsFg.IsEnabled() { + if rs.writeLag >= 0 { + p.mb.RecordPostgresqlWalDelayDataPoint(now, rs.writeLag, metadata.AttributeWalOperationLagWrite, rs.clientAddr) + } + if rs.replayLag >= 0 { + p.mb.RecordPostgresqlWalDelayDataPoint(now, rs.replayLag, metadata.AttributeWalOperationLagReplay, rs.clientAddr) + } + if rs.flushLag >= 0 { + p.mb.RecordPostgresqlWalDelayDataPoint(now, rs.flushLag, metadata.AttributeWalOperationLagFlush, rs.clientAddr) + } + } else { + if rs.writeLagInt >= 0 { + p.mb.RecordPostgresqlWalLagDataPoint(now, rs.writeLagInt, metadata.AttributeWalOperationLagWrite, rs.clientAddr) + } + if rs.replayLagInt >= 0 { + p.mb.RecordPostgresqlWalLagDataPoint(now, rs.replayLagInt, metadata.AttributeWalOperationLagReplay, rs.clientAddr) + } + if rs.flushLagInt >= 0 { + p.mb.RecordPostgresqlWalLagDataPoint(now, rs.flushLagInt, metadata.AttributeWalOperationLagFlush, rs.clientAddr) + } } } } diff --git a/receiver/postgresqlreceiver/scraper_test.go b/receiver/postgresqlreceiver/scraper_test.go index c04359498c26..065e85e820d4 100644 --- a/receiver/postgresqlreceiver/scraper_test.go +++ b/receiver/postgresqlreceiver/scraper_test.go @@ -14,6 +14,7 @@ import ( "go.opentelemetry.io/collector/pdata/pmetric" "go.opentelemetry.io/collector/receiver/receivertest" + "github.com/open-telemetry/opentelemetry-collector-contrib/internal/common/testutil" "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/golden" "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest/pmetrictest" ) @@ -123,6 +124,37 @@ func TestScraperNoDatabaseMultiple(t *testing.T) { pmetrictest.IgnoreMetricDataPointsOrder(), pmetrictest.IgnoreStartTimestamp(), pmetrictest.IgnoreTimestamp())) } +func TestScraperNoDatabaseMultipleWithPreciseLagFeatureGate(t *testing.T) { + factory := mockClientFactory{} + factory.initMocks([]string{"otel", "open", "telemetry"}) + + cfg := createDefaultConfig().(*Config) + + testutil.SetFeatureGateForTest(t, preciseLagMetricsFg, true) + cfg.Metrics.PostgresqlWalDelay.Enabled = true + defer testutil.SetFeatureGateForTest(t, preciseLagMetricsFg, false) + + require.True(t, cfg.Metrics.PostgresqlDeadlocks.Enabled == false) + cfg.Metrics.PostgresqlDeadlocks.Enabled = true + require.True(t, cfg.Metrics.PostgresqlTempFiles.Enabled == false) + cfg.Metrics.PostgresqlTempFiles.Enabled = true + require.True(t, cfg.Metrics.PostgresqlSequentialScans.Enabled == false) + cfg.Metrics.PostgresqlSequentialScans.Enabled = true + require.True(t, cfg.Metrics.PostgresqlDatabaseLocks.Enabled == false) + cfg.Metrics.PostgresqlDatabaseLocks.Enabled = true + scraper := newPostgreSQLScraper(receivertest.NewNopCreateSettings(), cfg, &factory) + + actualMetrics, err := scraper.scrape(context.Background()) + require.NoError(t, err) + + expectedFile := filepath.Join("testdata", "scraper", "multiple", "expected_precise_lag.yaml") + expectedMetrics, err := golden.ReadMetrics(expectedFile) + require.NoError(t, err) + fmt.Println(actualMetrics.ResourceMetrics()) + require.NoError(t, pmetrictest.CompareMetrics(expectedMetrics, actualMetrics, pmetrictest.IgnoreResourceMetricsOrder(), + pmetrictest.IgnoreMetricDataPointsOrder(), pmetrictest.IgnoreStartTimestamp(), pmetrictest.IgnoreTimestamp())) +} + func TestScraperWithResourceAttributeFeatureGate(t *testing.T) { factory := mockClientFactory{} factory.initMocks([]string{"otel", "open", "telemetry"}) @@ -333,13 +365,19 @@ func (m *mockClient) initMocks(database string, databases []string, index int) { { clientAddr: "unix", pendingBytes: 1024, - flushLag: 600, - replayLag: 700, - writeLag: 800, + flushLagInt: 600, + replayLagInt: 700, + writeLagInt: 800, + flushLag: 600.400, + replayLag: 700.550, + writeLag: 800.660, }, { clientAddr: "nulls", pendingBytes: -1, + flushLagInt: -1, + replayLagInt: -1, + writeLagInt: -1, flushLag: -1, replayLag: -1, writeLag: -1, diff --git a/receiver/postgresqlreceiver/testdata/scraper/multiple/expected_precise_lag.yaml b/receiver/postgresqlreceiver/testdata/scraper/multiple/expected_precise_lag.yaml new file mode 100644 index 000000000000..3d2d92e10218 --- /dev/null +++ b/receiver/postgresqlreceiver/testdata/scraper/multiple/expected_precise_lag.yaml @@ -0,0 +1,1602 @@ +resourceMetrics: + - resource: {} + scopeMetrics: + - metrics: + - description: Number of buffers allocated. + name: postgresql.bgwriter.buffers.allocated + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "10" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{buffers}' + - description: Number of buffers written. + name: postgresql.bgwriter.buffers.writes + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "7" + attributes: + - key: source + value: + stringValue: backend + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "8" + attributes: + - key: source + value: + stringValue: backend_fsync + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "5" + attributes: + - key: source + value: + stringValue: bgwriter + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "9" + attributes: + - key: source + value: + stringValue: checkpoints + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{buffers}' + - description: The number of checkpoints performed. + name: postgresql.bgwriter.checkpoint.count + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "1" + attributes: + - key: type + value: + stringValue: requested + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "2" + attributes: + - key: type + value: + stringValue: scheduled + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{checkpoints}' + - description: Total time spent writing and syncing files to disk by checkpoints. + name: postgresql.bgwriter.duration + sum: + aggregationTemporality: 2 + dataPoints: + - asDouble: 4.23 + attributes: + - key: type + value: + stringValue: sync + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asDouble: 3.12 + attributes: + - key: type + value: + stringValue: write + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: ms + - description: Number of times the background writer stopped a cleaning scan because it had written too many buffers. + name: postgresql.bgwriter.maxwritten + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "11" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: Configured maximum number of client connections allowed + gauge: + dataPoints: + - asInt: "100" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + name: postgresql.connection.max + unit: '{connections}' + - description: Number of user databases. + name: postgresql.database.count + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "3" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: '{databases}' + - description: The number of database locks. + gauge: + dataPoints: + - asInt: "3600" + attributes: + - key: "relation" + value: + stringValue: "pg_locks" + - key: "mode" + value: + stringValue: "AccessShareLock" + - key: "lock_type" + value: + stringValue: "relation" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "5600" + attributes: + - key: "relation" + value: + stringValue: "pg_class" + - key: "mode" + value: + stringValue: "AccessShareLock" + - key: "lock_type" + value: + stringValue: "relation" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + name: postgresql.database.locks + unit: '{lock}' + - description: The amount of data delayed in replication. + gauge: + dataPoints: + - asInt: "1024" + attributes: + - key: replication_client + value: + stringValue: unix + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + name: postgresql.replication.data_delay + unit: By + - description: Age of the oldest WAL file. + gauge: + dataPoints: + - asInt: "3600" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + name: postgresql.wal.age + unit: s + - description: Time between flushing recent WAL locally and receiving notification that the standby server has completed an operation with it. + gauge: + dataPoints: + - asDouble: "600.400" + attributes: + - key: operation + value: + stringValue: flush + - key: replication_client + value: + stringValue: unix + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asDouble: "700.550" + attributes: + - key: operation + value: + stringValue: replay + - key: replication_client + value: + stringValue: unix + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asDouble: "800.660" + attributes: + - key: operation + value: + stringValue: write + - key: replication_client + value: + stringValue: unix + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + name: postgresql.wal.delay + unit: s + scope: + name: otelcol/postgresqlreceiver + version: latest + - resource: + attributes: + - key: postgresql.database.name + value: + stringValue: open + scopeMetrics: + - metrics: + - description: The number of backends. + name: postgresql.backends + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "4" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: "1" + - description: The number of commits. + name: postgresql.commits + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "2" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: The database disk usage. + name: postgresql.db_size + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "5" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: By + - description: The number of deadlocks. + name: postgresql.deadlocks + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "4" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{deadlock}' + - description: The number of rollbacks. + name: postgresql.rollbacks + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "3" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: Number of user tables in a database. + name: postgresql.table.count + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "2" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: '{table}' + - description: The number of temp files. + name: postgresql.temp_files + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "5" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{temp_file}' + scope: + name: otelcol/postgresqlreceiver + version: latest + - resource: + attributes: + - key: postgresql.database.name + value: + stringValue: otel + scopeMetrics: + - metrics: + - description: The number of backends. + name: postgresql.backends + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "3" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: "1" + - description: The number of commits. + name: postgresql.commits + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "1" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: The database disk usage. + name: postgresql.db_size + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "4" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: By + - description: The number of deadlocks. + name: postgresql.deadlocks + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "3" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{deadlock}' + - description: The number of rollbacks. + name: postgresql.rollbacks + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "2" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: Number of user tables in a database. + name: postgresql.table.count + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "2" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: '{table}' + - description: The number of temp files. + name: postgresql.temp_files + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "4" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{temp_file}' + scope: + name: otelcol/postgresqlreceiver + version: latest + - resource: + attributes: + - key: postgresql.database.name + value: + stringValue: telemetry + scopeMetrics: + - metrics: + - description: The number of backends. + name: postgresql.backends + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "5" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: "1" + - description: The number of commits. + name: postgresql.commits + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "3" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: The database disk usage. + name: postgresql.db_size + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "6" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: By + - description: The number of deadlocks. + name: postgresql.deadlocks + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "5" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{deadlock}' + - description: The number of rollbacks. + name: postgresql.rollbacks + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "4" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: Number of user tables in a database. + name: postgresql.table.count + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "2" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: '{table}' + - description: The number of temp files. + name: postgresql.temp_files + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "6" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{temp_file}' + scope: + name: otelcol/postgresqlreceiver + version: latest + - resource: + attributes: + - key: postgresql.database.name + value: + stringValue: open + - key: postgresql.table.name + value: + stringValue: public.table1 + scopeMetrics: + - metrics: + - description: The number of blocks read. + name: postgresql.blocks_read + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "21" + attributes: + - key: source + value: + stringValue: heap_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "20" + attributes: + - key: source + value: + stringValue: heap_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "23" + attributes: + - key: source + value: + stringValue: idx_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "22" + attributes: + - key: source + value: + stringValue: idx_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "27" + attributes: + - key: source + value: + stringValue: tidx_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "26" + attributes: + - key: source + value: + stringValue: tidx_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "25" + attributes: + - key: source + value: + stringValue: toast_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "24" + attributes: + - key: source + value: + stringValue: toast_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: The number of db row operations. + name: postgresql.operations + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "42" + attributes: + - key: operation + value: + stringValue: del + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "43" + attributes: + - key: operation + value: + stringValue: hot_upd + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "40" + attributes: + - key: operation + value: + stringValue: ins + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "41" + attributes: + - key: operation + value: + stringValue: upd + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: The number of rows in the database. + name: postgresql.rows + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "9" + attributes: + - key: state + value: + stringValue: dead + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "8" + attributes: + - key: state + value: + stringValue: live + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: "1" + - description: The number of sequential scans. + name: postgresql.sequential_scans + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "46" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "{sequential_scan}" + - description: Disk space used by a table. + name: postgresql.table.size + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "44" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: By + - description: Number of times a table has manually been vacuumed. + name: postgresql.table.vacuum.count + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "45" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{vacuums}' + scope: + name: otelcol/postgresqlreceiver + version: latest + - resource: + attributes: + - key: postgresql.database.name + value: + stringValue: open + - key: postgresql.table.name + value: + stringValue: public.table2 + scopeMetrics: + - metrics: + - description: The number of blocks read. + name: postgresql.blocks_read + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "29" + attributes: + - key: source + value: + stringValue: heap_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "28" + attributes: + - key: source + value: + stringValue: heap_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "31" + attributes: + - key: source + value: + stringValue: idx_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "30" + attributes: + - key: source + value: + stringValue: idx_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "35" + attributes: + - key: source + value: + stringValue: tidx_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "34" + attributes: + - key: source + value: + stringValue: tidx_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "33" + attributes: + - key: source + value: + stringValue: toast_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "32" + attributes: + - key: source + value: + stringValue: toast_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: The number of db row operations. + name: postgresql.operations + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "46" + attributes: + - key: operation + value: + stringValue: del + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "47" + attributes: + - key: operation + value: + stringValue: hot_upd + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "44" + attributes: + - key: operation + value: + stringValue: ins + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "45" + attributes: + - key: operation + value: + stringValue: upd + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: The number of rows in the database. + name: postgresql.rows + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "11" + attributes: + - key: state + value: + stringValue: dead + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "10" + attributes: + - key: state + value: + stringValue: live + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: "1" + - description: The number of sequential scans. + name: postgresql.sequential_scans + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "50" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "{sequential_scan}" + - description: Disk space used by a table. + name: postgresql.table.size + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "48" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: By + - description: Number of times a table has manually been vacuumed. + name: postgresql.table.vacuum.count + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "49" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{vacuums}' + scope: + name: otelcol/postgresqlreceiver + version: latest + - resource: + attributes: + - key: postgresql.database.name + value: + stringValue: otel + - key: postgresql.table.name + value: + stringValue: public.table1 + scopeMetrics: + - metrics: + - description: The number of blocks read. + name: postgresql.blocks_read + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "20" + attributes: + - key: source + value: + stringValue: heap_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "19" + attributes: + - key: source + value: + stringValue: heap_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "22" + attributes: + - key: source + value: + stringValue: idx_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "21" + attributes: + - key: source + value: + stringValue: idx_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "26" + attributes: + - key: source + value: + stringValue: tidx_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "25" + attributes: + - key: source + value: + stringValue: tidx_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "24" + attributes: + - key: source + value: + stringValue: toast_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "23" + attributes: + - key: source + value: + stringValue: toast_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: The number of db row operations. + name: postgresql.operations + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "41" + attributes: + - key: operation + value: + stringValue: del + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "42" + attributes: + - key: operation + value: + stringValue: hot_upd + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "39" + attributes: + - key: operation + value: + stringValue: ins + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "40" + attributes: + - key: operation + value: + stringValue: upd + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: The number of rows in the database. + name: postgresql.rows + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "8" + attributes: + - key: state + value: + stringValue: dead + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "7" + attributes: + - key: state + value: + stringValue: live + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: "1" + - description: The number of sequential scans. + name: postgresql.sequential_scans + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "45" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "{sequential_scan}" + - description: Disk space used by a table. + name: postgresql.table.size + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "43" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: By + - description: Number of times a table has manually been vacuumed. + name: postgresql.table.vacuum.count + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "44" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{vacuums}' + scope: + name: otelcol/postgresqlreceiver + version: latest + - resource: + attributes: + - key: postgresql.database.name + value: + stringValue: otel + - key: postgresql.table.name + value: + stringValue: public.table2 + scopeMetrics: + - metrics: + - description: The number of blocks read. + name: postgresql.blocks_read + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "28" + attributes: + - key: source + value: + stringValue: heap_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "27" + attributes: + - key: source + value: + stringValue: heap_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "30" + attributes: + - key: source + value: + stringValue: idx_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "29" + attributes: + - key: source + value: + stringValue: idx_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "34" + attributes: + - key: source + value: + stringValue: tidx_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "33" + attributes: + - key: source + value: + stringValue: tidx_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "32" + attributes: + - key: source + value: + stringValue: toast_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "31" + attributes: + - key: source + value: + stringValue: toast_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: The number of db row operations. + name: postgresql.operations + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "45" + attributes: + - key: operation + value: + stringValue: del + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "46" + attributes: + - key: operation + value: + stringValue: hot_upd + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "43" + attributes: + - key: operation + value: + stringValue: ins + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "44" + attributes: + - key: operation + value: + stringValue: upd + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: The number of rows in the database. + name: postgresql.rows + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "10" + attributes: + - key: state + value: + stringValue: dead + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "9" + attributes: + - key: state + value: + stringValue: live + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: "1" + - description: The number of sequential scans. + name: postgresql.sequential_scans + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "49" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "{sequential_scan}" + - description: Disk space used by a table. + name: postgresql.table.size + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "47" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: By + - description: Number of times a table has manually been vacuumed. + name: postgresql.table.vacuum.count + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "48" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{vacuums}' + scope: + name: otelcol/postgresqlreceiver + version: latest + - resource: + attributes: + - key: postgresql.database.name + value: + stringValue: telemetry + - key: postgresql.table.name + value: + stringValue: public.table1 + scopeMetrics: + - metrics: + - description: The number of blocks read. + name: postgresql.blocks_read + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "22" + attributes: + - key: source + value: + stringValue: heap_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "21" + attributes: + - key: source + value: + stringValue: heap_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "24" + attributes: + - key: source + value: + stringValue: idx_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "23" + attributes: + - key: source + value: + stringValue: idx_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "28" + attributes: + - key: source + value: + stringValue: tidx_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "27" + attributes: + - key: source + value: + stringValue: tidx_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "26" + attributes: + - key: source + value: + stringValue: toast_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "25" + attributes: + - key: source + value: + stringValue: toast_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: The number of db row operations. + name: postgresql.operations + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "43" + attributes: + - key: operation + value: + stringValue: del + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "44" + attributes: + - key: operation + value: + stringValue: hot_upd + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "41" + attributes: + - key: operation + value: + stringValue: ins + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "42" + attributes: + - key: operation + value: + stringValue: upd + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: The number of rows in the database. + name: postgresql.rows + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "10" + attributes: + - key: state + value: + stringValue: dead + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "9" + attributes: + - key: state + value: + stringValue: live + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: "1" + - description: The number of sequential scans. + name: postgresql.sequential_scans + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "47" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "{sequential_scan}" + - description: Disk space used by a table. + name: postgresql.table.size + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "45" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: By + - description: Number of times a table has manually been vacuumed. + name: postgresql.table.vacuum.count + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "46" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{vacuums}' + scope: + name: otelcol/postgresqlreceiver + version: latest + - resource: + attributes: + - key: postgresql.database.name + value: + stringValue: telemetry + - key: postgresql.table.name + value: + stringValue: public.table2 + scopeMetrics: + - metrics: + - description: The number of blocks read. + name: postgresql.blocks_read + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "30" + attributes: + - key: source + value: + stringValue: heap_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "29" + attributes: + - key: source + value: + stringValue: heap_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "32" + attributes: + - key: source + value: + stringValue: idx_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "31" + attributes: + - key: source + value: + stringValue: idx_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "36" + attributes: + - key: source + value: + stringValue: tidx_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "35" + attributes: + - key: source + value: + stringValue: tidx_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "34" + attributes: + - key: source + value: + stringValue: toast_hit + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "33" + attributes: + - key: source + value: + stringValue: toast_read + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: The number of db row operations. + name: postgresql.operations + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "47" + attributes: + - key: operation + value: + stringValue: del + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "48" + attributes: + - key: operation + value: + stringValue: hot_upd + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "45" + attributes: + - key: operation + value: + stringValue: ins + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "46" + attributes: + - key: operation + value: + stringValue: upd + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "1" + - description: The number of rows in the database. + name: postgresql.rows + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "12" + attributes: + - key: state + value: + stringValue: dead + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "11" + attributes: + - key: state + value: + stringValue: live + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: "1" + - description: The number of sequential scans. + name: postgresql.sequential_scans + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "51" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: "{sequential_scan}" + - description: Disk space used by a table. + name: postgresql.table.size + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "49" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: By + - description: Number of times a table has manually been vacuumed. + name: postgresql.table.vacuum.count + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "50" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{vacuums}' + scope: + name: otelcol/postgresqlreceiver + version: latest + - resource: + attributes: + - key: postgresql.database.name + value: + stringValue: open + - key: postgresql.index.name + value: + stringValue: open_test1_pkey + - key: postgresql.table.name + value: + stringValue: public.table1 + scopeMetrics: + - metrics: + - description: The number of index scans on a table. + name: postgresql.index.scans + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "36" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{scans}' + - description: The size of the index on disk. + gauge: + dataPoints: + - asInt: "37" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + name: postgresql.index.size + unit: By + scope: + name: otelcol/postgresqlreceiver + version: latest + - resource: + attributes: + - key: postgresql.database.name + value: + stringValue: open + - key: postgresql.index.name + value: + stringValue: open_test2_pkey + - key: postgresql.table.name + value: + stringValue: public.table2 + scopeMetrics: + - metrics: + - description: The number of index scans on a table. + name: postgresql.index.scans + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "38" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{scans}' + - description: The size of the index on disk. + gauge: + dataPoints: + - asInt: "39" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + name: postgresql.index.size + unit: By + scope: + name: otelcol/postgresqlreceiver + version: latest + - resource: + attributes: + - key: postgresql.database.name + value: + stringValue: otel + - key: postgresql.index.name + value: + stringValue: otel_test1_pkey + - key: postgresql.table.name + value: + stringValue: public.table1 + scopeMetrics: + - metrics: + - description: The number of index scans on a table. + name: postgresql.index.scans + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "35" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{scans}' + - description: The size of the index on disk. + gauge: + dataPoints: + - asInt: "36" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + name: postgresql.index.size + unit: By + scope: + name: otelcol/postgresqlreceiver + version: latest + - resource: + attributes: + - key: postgresql.database.name + value: + stringValue: otel + - key: postgresql.index.name + value: + stringValue: otel_test2_pkey + - key: postgresql.table.name + value: + stringValue: public.table2 + scopeMetrics: + - metrics: + - description: The number of index scans on a table. + name: postgresql.index.scans + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "37" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{scans}' + - description: The size of the index on disk. + gauge: + dataPoints: + - asInt: "38" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + name: postgresql.index.size + unit: By + scope: + name: otelcol/postgresqlreceiver + version: latest + - resource: + attributes: + - key: postgresql.database.name + value: + stringValue: telemetry + - key: postgresql.index.name + value: + stringValue: telemetry_test1_pkey + - key: postgresql.table.name + value: + stringValue: public.table1 + scopeMetrics: + - metrics: + - description: The number of index scans on a table. + name: postgresql.index.scans + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "37" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{scans}' + - description: The size of the index on disk. + gauge: + dataPoints: + - asInt: "38" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + name: postgresql.index.size + unit: By + scope: + name: otelcol/postgresqlreceiver + version: latest + - resource: + attributes: + - key: postgresql.database.name + value: + stringValue: telemetry + - key: postgresql.index.name + value: + stringValue: telemetry_test2_pkey + - key: postgresql.table.name + value: + stringValue: public.table2 + scopeMetrics: + - metrics: + - description: The number of index scans on a table. + name: postgresql.index.scans + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "39" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + isMonotonic: true + unit: '{scans}' + - description: The size of the index on disk. + gauge: + dataPoints: + - asInt: "40" + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + name: postgresql.index.size + unit: By + scope: + name: otelcol/postgresqlreceiver + version: latest