From a5cf26d405fba9fee6b7a4f607977097b06fec5b Mon Sep 17 00:00:00 2001 From: matt durham Date: Thu, 19 Sep 2024 13:25:19 -0400 Subject: [PATCH 01/44] Checkin the networking items. --- .../remote/queue/network/benchmark_test.go | 24 ++ .../prometheus/remote/queue/network/config.go | 24 ++ .../prometheus/remote/queue/network/loop.go | 374 ++++++++++++++++++ .../remote/queue/network/manager.go | 162 ++++++++ .../prometheus/remote/queue/network/stats.go | 110 ++++++ .../remote/queue/serialization/appender.go | 6 +- .../prometheus/remote/queue/types/network.go | 12 + .../remote/queue/types/serialization.go | 4 + .../prometheus/remote/queue/types/stats.go | 271 +++++++++++++ 9 files changed, 984 insertions(+), 3 deletions(-) create mode 100644 internal/component/prometheus/remote/queue/network/benchmark_test.go create mode 100644 internal/component/prometheus/remote/queue/network/config.go create mode 100644 internal/component/prometheus/remote/queue/network/loop.go create mode 100644 internal/component/prometheus/remote/queue/network/manager.go create mode 100644 internal/component/prometheus/remote/queue/network/stats.go create mode 100644 internal/component/prometheus/remote/queue/types/network.go diff --git a/internal/component/prometheus/remote/queue/network/benchmark_test.go b/internal/component/prometheus/remote/queue/network/benchmark_test.go new file mode 100644 index 0000000000..a1c22328cb --- /dev/null +++ b/internal/component/prometheus/remote/queue/network/benchmark_test.go @@ -0,0 +1,24 @@ +package network + +import ( + "context" + "testing" + + "github.com/vladopajic/go-actor/actor" +) + +func BenchmarkMailbox(b *testing.B) { + // This should be 260 ns roughly or 3m messages a second. + mbx := actor.NewMailbox[struct{}]() + mbx.Start() + defer mbx.Stop() + go func() { + for { + <-mbx.ReceiveC() + } + }() + ctx := context.Background() + for i := 0; i < b.N; i++ { + mbx.Send(ctx, struct{}{}) + } +} diff --git a/internal/component/prometheus/remote/queue/network/config.go b/internal/component/prometheus/remote/queue/network/config.go new file mode 100644 index 0000000000..bccd74becf --- /dev/null +++ b/internal/component/prometheus/remote/queue/network/config.go @@ -0,0 +1,24 @@ +package network + +import ( + "reflect" + "time" +) + +type ConnectionConfig struct { + URL string + Username string + Password string + UserAgent string + Timeout time.Duration + RetryBackoff time.Duration + MaxRetryBackoffAttempts time.Duration + BatchCount int + FlushFrequency time.Duration + ExternalLabels map[string]string + Connections uint64 +} + +func (cc ConnectionConfig) Equals(bb ConnectionConfig) bool { + return reflect.DeepEqual(cc, bb) +} diff --git a/internal/component/prometheus/remote/queue/network/loop.go b/internal/component/prometheus/remote/queue/network/loop.go new file mode 100644 index 0000000000..62293f4aa6 --- /dev/null +++ b/internal/component/prometheus/remote/queue/network/loop.go @@ -0,0 +1,374 @@ +package network + +import ( + "bufio" + "bytes" + "context" + "fmt" + "io" + "net/http" + "strconv" + "strings" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/golang/protobuf/proto" + "github.com/golang/snappy" + "github.com/grafana/alloy/internal/component/prometheus/remote/queue/types" + "github.com/prometheus/prometheus/prompb" + "github.com/vladopajic/go-actor/actor" + "go.uber.org/atomic" +) + +var _ actor.Worker = (*loop)(nil) + +// loop handles the low level sending of data. It conceptually a queue. +// loop makes no attempt to save or restore signals in the queue. +// loop config cannot be updated, it is easier to recreate. This does mean we lose any signals in the queue. +type loop struct { + isMeta bool + seriesMbx actor.Mailbox[*types.TimeSeriesBinary] + client *http.Client + batchCount int + flushFrequency time.Duration + cfg ConnectionConfig + log log.Logger + lastSend time.Time + statsFunc func(s types.NetworkStats) + stopCalled atomic.Bool + externalLabels map[string]string + series []*types.TimeSeriesBinary + self actor.Actor + ticker *time.Ticker + req *prompb.WriteRequest + buf *proto.Buffer + sendBuffer []byte +} + +func newLoop(cc ConnectionConfig, isMetaData bool, log log.Logger, stats func(s types.NetworkStats)) *loop { + // TODO @mattdurham add TLS support afer the initial push. + l := &loop{ + isMeta: isMetaData, + // In general we want a healthy queue of items, in this case we want to have 2x our maximum send sized ready. + seriesMbx: actor.NewMailbox[*types.TimeSeriesBinary](actor.OptCapacity(2 * cc.BatchCount)), + batchCount: cc.BatchCount, + flushFrequency: cc.FlushFrequency, + client: &http.Client{}, + cfg: cc, + log: log, + statsFunc: stats, + externalLabels: cc.ExternalLabels, + ticker: time.NewTicker(1 * time.Second), + buf: proto.NewBuffer(nil), + sendBuffer: make([]byte, 0), + } + l.req = &prompb.WriteRequest{ + // We know BatchCount is the most we will ever send. + Timeseries: make([]prompb.TimeSeries, 0, cc.BatchCount), + } + + return l +} + +func (l *loop) Start() { + l.self = actor.Combine(l.actors()...).Build() + l.self.Start() +} + +func (l *loop) Stop() { + l.stopCalled.Store(true) + l.self.Stop() +} + +func (l *loop) actors() []actor.Actor { + return []actor.Actor{ + actor.New(l), + l.seriesMbx, + } +} + +func (l *loop) DoWork(ctx actor.Context) actor.WorkerStatus { + // Main select loop + select { + case <-ctx.Done(): + l.stopCalled.Store(true) + return actor.WorkerEnd + // Ticker is to ensure the flush timer is called. + case <-l.ticker.C: + if len(l.series) == 0 { + return actor.WorkerContinue + } + if time.Since(l.lastSend) > l.flushFrequency { + l.trySend(ctx) + } + return actor.WorkerContinue + case series, ok := <-l.seriesMbx.ReceiveC(): + if !ok { + return actor.WorkerEnd + } + l.series = append(l.series, series) + if len(l.series) >= l.batchCount { + l.trySend(ctx) + } + return actor.WorkerContinue + } +} + +// trySend is the core functionality for sending data to a endpoint. It will attempt retries as defined in MaxRetryBackoffAttempts. +func (l *loop) trySend(ctx context.Context) { + attempts := 0 +attempt: + start := time.Now() + result := l.send(ctx, attempts) + duration := time.Since(start) + l.statsFunc(types.NetworkStats{ + SendDuration: duration, + }) + if result.successful { + l.sendingCleanup() + return + } + if !result.recoverableError { + l.sendingCleanup() + return + } + attempts++ + if attempts > int(l.cfg.MaxRetryBackoffAttempts) && l.cfg.MaxRetryBackoffAttempts > 0 { + level.Debug(l.log).Log("msg", "max retry attempts reached", "attempts", attempts) + l.sendingCleanup() + return + } + // This helps us short circuit the loop if we are stopping. + if l.stopCalled.Load() { + return + } + // Sleep between attempts. + time.Sleep(result.retryAfter) + goto attempt +} + +type sendResult struct { + err error + successful bool + recoverableError bool + retryAfter time.Duration + statusCode int + networkError bool +} + +func (l *loop) sendingCleanup() { + types.PutTimeSeriesSliceIntoPool(l.series) + l.sendBuffer = l.sendBuffer[:0] + l.series = make([]*types.TimeSeriesBinary, 0, l.batchCount) + l.lastSend = time.Now() +} + +// send is the main work loop of the loop. +func (l *loop) send(ctx context.Context, retryCount int) sendResult { + result := sendResult{} + defer func() { + recordStats(l.series, l.isMeta, l.statsFunc, result, len(l.sendBuffer)) + }() + var err error + // Check to see if this is a retry and we can reuse the buffer. + // I wonder if we should do this, its possible we are sending things that have exceeded the TTL. + if len(l.sendBuffer) == 0 { + var data []byte + var wrErr error + if l.isMeta { + data, wrErr = createWriteRequestMetadata(l.req, l.series, l.buf) + } else { + data, wrErr = createWriteRequest(l.req, l.series, l.externalLabels, l.buf) + } + if wrErr != nil { + result.err = wrErr + result.recoverableError = false + return result + } + l.sendBuffer = snappy.Encode(l.sendBuffer, data) + } + + httpReq, err := http.NewRequest("POST", l.cfg.URL, bytes.NewReader(l.sendBuffer)) + if err != nil { + result.err = err + result.recoverableError = true + result.networkError = true + return result + } + httpReq.Header.Add("Content-Encoding", "snappy") + httpReq.Header.Set("Content-Type", "application/x-protobuf") + httpReq.Header.Set("User-Agent", l.cfg.UserAgent) + httpReq.Header.Set("X-Prometheus-Remote-Write-Version", "0.1.0") + httpReq.SetBasicAuth(l.cfg.Username, l.cfg.Password) + + if retryCount > 0 { + httpReq.Header.Set("Retry-Attempt", strconv.Itoa(retryCount)) + } + ctx, cncl := context.WithTimeout(ctx, l.cfg.Timeout) + defer cncl() + resp, err := l.client.Do(httpReq.WithContext(ctx)) + // Network errors are recoverable. + if err != nil { + result.err = err + result.networkError = true + result.recoverableError = true + result.retryAfter = l.cfg.RetryBackoff + return result + } + result.statusCode = resp.StatusCode + defer resp.Body.Close() + // 500 errors are considered recoverable. + if resp.StatusCode/100 == 5 || resp.StatusCode == http.StatusTooManyRequests { + result.retryAfter = retryAfterDuration(l.cfg.RetryBackoff, resp.Header.Get("Retry-After")) + result.recoverableError = true + return result + } + // Status Codes that are not 500 or 200 are not recoverable and dropped. + if resp.StatusCode/100 != 2 { + scanner := bufio.NewScanner(io.LimitReader(resp.Body, 1_000)) + line := "" + if scanner.Scan() { + line = scanner.Text() + } + result.err = fmt.Errorf("server returned HTTP status %s: %s", resp.Status, line) + return result + } + + result.successful = true + return result +} + +func createWriteRequest(wr *prompb.WriteRequest, series []*types.TimeSeriesBinary, externalLabels map[string]string, data *proto.Buffer) ([]byte, error) { + if cap(wr.Timeseries) < len(series) { + wr.Timeseries = make([]prompb.TimeSeries, len(series)) + } + wr.Timeseries = wr.Timeseries[:len(series)] + + for i, tsBuf := range series { + ts := wr.Timeseries[i] + if cap(ts.Labels) < len(tsBuf.Labels) { + ts.Labels = make([]prompb.Label, 0, len(tsBuf.Labels)) + } + ts.Labels = ts.Labels[:len(tsBuf.Labels)] + for k, v := range tsBuf.Labels { + ts.Labels[k].Name = v.Name + ts.Labels[k].Value = v.Value + } + + // By default each sample only has a histogram, float histogram or sample. + if cap(ts.Histograms) == 0 { + ts.Histograms = make([]prompb.Histogram, 1) + } else { + ts.Histograms = ts.Histograms[:0] + } + if tsBuf.Histograms.Histogram != nil { + ts.Histograms = ts.Histograms[:1] + ts.Histograms[0] = tsBuf.Histograms.Histogram.ToPromHistogram() + } + if tsBuf.Histograms.FloatHistogram != nil { + ts.Histograms = ts.Histograms[:1] + ts.Histograms[0] = tsBuf.Histograms.FloatHistogram.ToPromFloatHistogram() + } + + if tsBuf.Histograms.Histogram == nil && tsBuf.Histograms.FloatHistogram == nil { + ts.Histograms = ts.Histograms[:0] + } + + // Encode the external labels inside if needed. + for k, v := range externalLabels { + found := false + for j, lbl := range ts.Labels { + if lbl.Name == k { + ts.Labels[j].Value = v + found = true + break + } + } + if !found { + ts.Labels = append(ts.Labels, prompb.Label{ + Name: k, + Value: v, + }) + } + } + // By default each TimeSeries only has one sample. + if len(ts.Samples) == 0 { + ts.Samples = make([]prompb.Sample, 1) + } + ts.Samples[0].Value = tsBuf.Value + ts.Samples[0].Timestamp = tsBuf.TS + wr.Timeseries[i] = ts + } + defer func() { + for i := 0; i < len(wr.Timeseries); i++ { + wr.Timeseries[i].Histograms = wr.Timeseries[i].Histograms[:0] + wr.Timeseries[i].Labels = wr.Timeseries[i].Labels[:0] + wr.Timeseries[i].Exemplars = wr.Timeseries[i].Exemplars[:0] + } + }() + // Reset the buffer for reuse. + data.Reset() + err := data.Marshal(wr) + return data.Bytes(), err +} + +func createWriteRequestMetadata(wr *prompb.WriteRequest, series []*types.TimeSeriesBinary, data *proto.Buffer) ([]byte, error) { + if cap(wr.Metadata) < len(series) { + wr.Metadata = make([]prompb.MetricMetadata, len(series)) + } else { + wr.Metadata = wr.Metadata[:len(series)] + } + + for i, ts := range series { + mt, valid := toMetadata(ts) + if !valid { + continue + } + wr.Metadata[i] = mt + } + data.Reset() + err := data.Marshal(wr) + return data.Bytes(), err +} + +func getMetadataCount(tss []*types.TimeSeriesBinary) int { + var cnt int + for _, ts := range tss { + if isMetadata(ts) { + cnt++ + } + } + return cnt +} + +func isMetadata(ts *types.TimeSeriesBinary) bool { + return ts.Labels.Has(types.MetaType) && + ts.Labels.Has(types.MetaUnit) && + ts.Labels.Has(types.MetaHelp) +} + +func toMetadata(ts *types.TimeSeriesBinary) (prompb.MetricMetadata, bool) { + if !isMetadata(ts) { + return prompb.MetricMetadata{}, false + } + return prompb.MetricMetadata{ + Type: prompb.MetricMetadata_MetricType(prompb.MetricMetadata_MetricType_value[strings.ToUpper(ts.Labels.Get(types.MetaType))]), + Help: ts.Labels.Get(types.MetaHelp), + Unit: ts.Labels.Get(types.MetaUnit), + MetricFamilyName: ts.Labels.Get("__name__"), + }, true +} + +func retryAfterDuration(defaultDuration time.Duration, t string) time.Duration { + parsedTime, err := time.Parse(http.TimeFormat, t) + if err == nil { + return time.Until(parsedTime) + } + // The duration can be in seconds. + d, err := strconv.Atoi(t) + if err != nil { + return defaultDuration + } + return time.Duration(d) * time.Second +} diff --git a/internal/component/prometheus/remote/queue/network/manager.go b/internal/component/prometheus/remote/queue/network/manager.go new file mode 100644 index 0000000000..d942d05a0e --- /dev/null +++ b/internal/component/prometheus/remote/queue/network/manager.go @@ -0,0 +1,162 @@ +package network + +import ( + "context" + + "github.com/grafana/alloy/internal/runtime/logging/level" + + "github.com/go-kit/log" + "github.com/grafana/alloy/internal/component/prometheus/remote/queue/types" + "github.com/vladopajic/go-actor/actor" +) + +// manager manages loops. Mostly it exists to control their lifecycle and send work to them. +type manager struct { + connectionCount uint64 + loops []*loop + metadata *loop + logger log.Logger + inbox actor.Mailbox[*types.TimeSeriesBinary] + metaInbox actor.Mailbox[*types.TimeSeriesBinary] + configInbox actor.Mailbox[ConnectionConfig] + self actor.Actor + cfg ConnectionConfig + stats func(types.NetworkStats) + metaStats func(types.NetworkStats) +} + +var _ types.NetworkClient = (*manager)(nil) + +var _ actor.Worker = (*manager)(nil) + +func New(cc ConnectionConfig, logger log.Logger, seriesStats, metadataStats func(types.NetworkStats)) (types.NetworkClient, error) { + s := &manager{ + connectionCount: cc.Connections, + loops: make([]*loop, 0), + logger: logger, + // This provides blocking to only handle one at a time, so that if a queue blocks + // it will stop the filequeue from feeding more. + inbox: actor.NewMailbox[*types.TimeSeriesBinary](actor.OptCapacity(1)), + metaInbox: actor.NewMailbox[*types.TimeSeriesBinary](actor.OptCapacity(1)), + configInbox: actor.NewMailbox[ConnectionConfig](), + stats: seriesStats, + } + + // start kicks off a number of concurrent connections. + var i uint64 + for ; i < s.connectionCount; i++ { + l := newLoop(cc, false, logger, seriesStats) + l.self = actor.New(l) + s.loops = append(s.loops, l) + } + + s.metadata = newLoop(cc, true, logger, metadataStats) + s.metadata.self = actor.New(s.metadata) + return s, nil +} + +func (s *manager) Start() { + actors := make([]actor.Actor, 0) + for _, l := range s.loops { + l.Start() + } + actors = append(actors, s.metadata.actors()...) + actors = append(actors, s.inbox) + actors = append(actors, s.metaInbox) + actors = append(actors, actor.New(s)) + actors = append(actors, s.configInbox) + s.self = actor.Combine(actors...).Build() + s.self.Start() +} + +func (s *manager) SendSeries(ctx context.Context, data *types.TimeSeriesBinary) error { + return s.inbox.Send(ctx, data) +} + +func (s *manager) SendMetadata(ctx context.Context, data *types.TimeSeriesBinary) error { + return s.metaInbox.Send(ctx, data) +} + +func (s *manager) UpdateConfig(ctx context.Context, cc ConnectionConfig) error { + return s.configInbox.Send(ctx, cc) +} + +func (s *manager) DoWork(ctx actor.Context) actor.WorkerStatus { + // This acts as a priority queue, always check for configuration changes first. + select { + case cfg, ok := <-s.configInbox.ReceiveC(): + if !ok { + return actor.WorkerEnd + } + s.updateConfig(cfg) + return actor.WorkerContinue + default: + } + select { + case <-ctx.Done(): + s.Stop() + return actor.WorkerEnd + case ts, ok := <-s.inbox.ReceiveC(): + if !ok { + return actor.WorkerEnd + } + s.queue(ctx, ts) + return actor.WorkerContinue + case ts, ok := <-s.metaInbox.ReceiveC(): + if !ok { + return actor.WorkerEnd + } + err := s.metadata.seriesMbx.Send(ctx, ts) + if err != nil { + level.Error(s.logger).Log("msg", "failed to send to metadata loop", "err", err) + } + return actor.WorkerContinue + } +} + +func (s *manager) updateConfig(cc ConnectionConfig) { + // No need to do anything if the configuration is the same. + if s.cfg.Equals(cc) { + return + } + // TODO @mattdurham make this smarter, at the moment any samples in the loops are lost. + // Ideally we would drain the queues and re add them but that is a future need. + // In practice this shouldn't change often so data loss should be minimal. + // For the moment we will stop all the items and recreate them. + for _, l := range s.loops { + l.Stop() + } + s.metadata.Stop() + + s.loops = make([]*loop, 0) + var i uint64 + for ; i < s.connectionCount; i++ { + l := newLoop(cc, false, s.logger, s.stats) + l.self = actor.New(l) + s.loops = append(s.loops, l) + } + + s.metadata = newLoop(cc, true, s.logger, s.metaStats) + s.metadata.self = actor.New(s.metadata) +} + +func (s *manager) Stop() { + level.Debug(s.logger).Log("msg", "stopping manager") + for _, l := range s.loops { + l.Stop() + l.stopCalled.Store(true) + } + s.metadata.stopCalled.Store(true) + s.self.Stop() +} + +// Queue adds anything thats not metadata to the queue. +func (s *manager) queue(ctx context.Context, ts *types.TimeSeriesBinary) { + // Based on a hash which is the label hash add to the queue. + queueNum := ts.Hash % s.connectionCount + // This will block if the queue is full. + err := s.loops[queueNum].seriesMbx.Send(ctx, ts) + if err != nil { + level.Error(s.logger).Log("msg", "failed to send to loop", "err", err) + } +} diff --git a/internal/component/prometheus/remote/queue/network/stats.go b/internal/component/prometheus/remote/queue/network/stats.go new file mode 100644 index 0000000000..ec24de5d0d --- /dev/null +++ b/internal/component/prometheus/remote/queue/network/stats.go @@ -0,0 +1,110 @@ +package network + +import ( + "net/http" + + "github.com/grafana/alloy/internal/component/prometheus/remote/queue/types" +) + +// recordStats determines what values to send to the stats function. This allows for any +// number of metrics/signals libraries to be used. Prometheus, OTel, and any other. +func recordStats(series []*types.TimeSeriesBinary, isMeta bool, stats func(s types.NetworkStats), r sendResult, bytesSent int) { + switch { + case r.networkError: + stats(types.NetworkStats{ + Series: types.CategoryStats{ + NetworkSamplesFailed: getSeriesCount(series), + }, + Histogram: types.CategoryStats{ + NetworkSamplesFailed: getHistogramCount(series), + }, + }) + case r.successful: + // Need to grab the newest series. + var newestTS int64 + for _, ts := range series { + if ts.TS > newestTS { + newestTS = ts.TS + } + } + var sampleBytesSent int + var metaBytesSent int + // Each loop is explicitly a normal signal or metadata sender. + if isMeta { + metaBytesSent = bytesSent + } else { + sampleBytesSent = bytesSent + } + stats(types.NetworkStats{ + Series: types.CategoryStats{ + SeriesSent: getSeriesCount(series), + }, + Histogram: types.CategoryStats{ + SeriesSent: getHistogramCount(series), + }, + MetadataBytes: metaBytesSent, + SeriesBytes: sampleBytesSent, + NewestTimestamp: newestTS, + }) + case r.statusCode == http.StatusTooManyRequests: + stats(types.NetworkStats{ + Series: types.CategoryStats{ + RetriedSamples: getSeriesCount(series), + RetriedSamples429: getSeriesCount(series), + }, + Histogram: types.CategoryStats{ + RetriedSamples: getHistogramCount(series), + RetriedSamples429: getHistogramCount(series), + }, + }) + case r.statusCode/100 == 5: + stats(types.NetworkStats{ + Series: types.CategoryStats{ + RetriedSamples5XX: getSeriesCount(series), + }, + Histogram: types.CategoryStats{ + RetriedSamples5XX: getHistogramCount(series), + }, + }) + case r.statusCode != 200: + stats(types.NetworkStats{ + Series: types.CategoryStats{ + FailedSamples: getSeriesCount(series), + }, + Histogram: types.CategoryStats{ + FailedSamples: getHistogramCount(series), + }, + Metadata: types.CategoryStats{ + FailedSamples: getMetadataCount(series), + }, + }) + } + +} + +func getSeriesCount(tss []*types.TimeSeriesBinary) int { + cnt := 0 + for _, ts := range tss { + // This is metadata + if isMetadata(ts) { + continue + } + if ts.Histograms.Histogram == nil && ts.Histograms.FloatHistogram == nil { + cnt++ + } + } + return cnt +} + +func getHistogramCount(tss []*types.TimeSeriesBinary) int { + cnt := 0 + for _, ts := range tss { + if isMetadata(ts) { + continue + } + if ts.Histograms.Histogram != nil || ts.Histograms.FloatHistogram != nil { + cnt++ + } + } + return cnt +} diff --git a/internal/component/prometheus/remote/queue/serialization/appender.go b/internal/component/prometheus/remote/queue/serialization/appender.go index b9248c00c6..e2cdd56272 100644 --- a/internal/component/prometheus/remote/queue/serialization/appender.go +++ b/internal/component/prometheus/remote/queue/serialization/appender.go @@ -104,15 +104,15 @@ func (a *appender) UpdateMetadata(ref storage.SeriesRef, l labels.Labels, m meta // to ensure its efficient it makes sense to encode metadata into it. combinedLabels := l.Copy() combinedLabels = append(combinedLabels, labels.Label{ - Name: "__alloy_metadata_type__", + Name: types.MetaType, Value: string(m.Type), }) combinedLabels = append(combinedLabels, labels.Label{ - Name: "__alloy_metadata_help__", + Name: types.MetaHelp, Value: m.Help, }) combinedLabels = append(combinedLabels, labels.Label{ - Name: "__alloy_metadata_unit__", + Name: types.MetaUnit, Value: m.Unit, }) ts.Labels = combinedLabels diff --git a/internal/component/prometheus/remote/queue/types/network.go b/internal/component/prometheus/remote/queue/types/network.go new file mode 100644 index 0000000000..44ee5e089d --- /dev/null +++ b/internal/component/prometheus/remote/queue/types/network.go @@ -0,0 +1,12 @@ +package types + +import ( + "context" +) + +type NetworkClient interface { + Start() + Stop() + SendSeries(ctx context.Context, d *TimeSeriesBinary) error + SendMetadata(ctx context.Context, d *TimeSeriesBinary) error +} diff --git a/internal/component/prometheus/remote/queue/types/serialization.go b/internal/component/prometheus/remote/queue/types/serialization.go index 7777055433..8064e8de3e 100644 --- a/internal/component/prometheus/remote/queue/types/serialization.go +++ b/internal/component/prometheus/remote/queue/types/serialization.go @@ -10,6 +10,10 @@ import ( "go.uber.org/atomic" ) +const MetaType = "__alloy_metadata_type__" +const MetaUnit = "__alloy_metadata_unit__" +const MetaHelp = "__alloy_metadata_help__" + // SeriesGroup is the holder for TimeSeries, Metadata, and the strings array. // When serialized the Labels Key,Value array will be transformed into // LabelNames and LabelsValues that point to the index in Strings. diff --git a/internal/component/prometheus/remote/queue/types/stats.go b/internal/component/prometheus/remote/queue/types/stats.go index c74f0953a4..06dcde57fb 100644 --- a/internal/component/prometheus/remote/queue/types/stats.go +++ b/internal/component/prometheus/remote/queue/types/stats.go @@ -1,8 +1,279 @@ package types +import ( + "time" + + "github.com/prometheus/client_golang/prometheus" +) + type SerializerStats struct { SeriesStored int MetadataStored int Errors int NewestTimestamp int64 } + +type PrometheusStats struct { + // Network Stats + NetworkSeriesSent prometheus.Counter + NetworkFailures prometheus.Counter + NetworkRetries prometheus.Counter + NetworkRetries429 prometheus.Counter + NetworkRetries5XX prometheus.Counter + NetworkSentDuration prometheus.Histogram + NetworkErrors prometheus.Counter + NetworkNewestOutTimeStampSeconds prometheus.Gauge + + // Filequeue Stats + FilequeueInSeries prometheus.Counter + FilequeueNewestInTimeStampSeconds prometheus.Gauge + FilequeueErrors prometheus.Counter + + // Backwards compatibility metrics + SamplesTotal prometheus.Counter + HistogramsTotal prometheus.Counter + MetadataTotal prometheus.Counter + + FailedSamplesTotal prometheus.Counter + FailedHistogramsTotal prometheus.Counter + FailedMetadataTotal prometheus.Counter + + RetriedSamplesTotal prometheus.Counter + RetriedHistogramsTotal prometheus.Counter + RetriedMetadataTotal prometheus.Counter + + EnqueueRetriesTotal prometheus.Counter + SentBatchDuration prometheus.Histogram + HighestSentTimestamp prometheus.Gauge + + SentBytesTotal prometheus.Counter + MetadataBytesTotal prometheus.Counter + RemoteStorageInTimestamp prometheus.Gauge + RemoteStorageOutTimestamp prometheus.Gauge + RemoteStorageDuration prometheus.Histogram +} + +func NewStats(namespace, subsystem string, registry prometheus.Registerer) *PrometheusStats { + s := &PrometheusStats{ + FilequeueInSeries: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "filequeue_incoming", + }), + FilequeueNewestInTimeStampSeconds: prometheus.NewGauge(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "filequeue_incoming_timestamp_seconds", + }), + FilequeueErrors: prometheus.NewGauge(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "filequeue_errors", + }), + NetworkNewestOutTimeStampSeconds: prometheus.NewGauge(prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "network_timestamp_seconds", + }), + RemoteStorageDuration: prometheus.NewHistogram(prometheus.HistogramOpts{ + Name: "prometheus_remote_storage_queue_duration_seconds", + }), + NetworkSeriesSent: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "network_sent", + }), + NetworkFailures: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "network_failed", + }), + NetworkRetries: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "network_retried", + }), + NetworkRetries429: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "network_retried_429", + }), + NetworkRetries5XX: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "network_retried_5xx", + }), + NetworkSentDuration: prometheus.NewHistogram(prometheus.HistogramOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "network_duration_seconds", + NativeHistogramBucketFactor: 1.1, + }), + NetworkErrors: prometheus.NewCounter(prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "network_errors", + }), + RemoteStorageOutTimestamp: prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_remote_storage_queue_highest_sent_timestamp_seconds", + }), + RemoteStorageInTimestamp: prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_remote_storage_highest_timestamp_in_seconds", + }), + SamplesTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_remote_storage_samples_total", + Help: "Total number of samples sent to remote storage.", + }), + HistogramsTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_remote_storage_histograms_total", + Help: "Total number of histograms sent to remote storage.", + }), + MetadataTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_remote_storage_metadata_total", + Help: "Total number of metadata sent to remote storage.", + }), + FailedSamplesTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_remote_storage_samples_failed_total", + Help: "Total number of samples which failed on send to remote storage, non-recoverable errors.", + }), + FailedHistogramsTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_remote_storage_histograms_failed_total", + Help: "Total number of histograms which failed on send to remote storage, non-recoverable errors.", + }), + FailedMetadataTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_remote_storage_metadata_failed_total", + Help: "Total number of metadata entries which failed on send to remote storage, non-recoverable errors.", + }), + + RetriedSamplesTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_remote_storage_samples_retried_total", + Help: "Total number of samples which failed on send to remote storage but were retried because the send error was recoverable.", + }), + RetriedHistogramsTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_remote_storage_histograms_retried_total", + Help: "Total number of histograms which failed on send to remote storage but were retried because the send error was recoverable.", + }), + RetriedMetadataTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_remote_storage_metadata_retried_total", + Help: "Total number of metadata entries which failed on send to remote storage but were retried because the send error was recoverable.", + }), + SentBytesTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_remote_storage_sent_bytes_total", + Help: "The total number of bytes of data (not metadata) sent by the queue after compression. Note that when exemplars over remote write is enabled the exemplars included in a remote write request count towards this metric.", + }), + MetadataBytesTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_remote_storage_metadata_bytes_total", + Help: "The total number of bytes of metadata sent by the queue after compression.", + }), + } + registry.MustRegister( + s.NetworkSentDuration, + s.NetworkRetries5XX, + s.NetworkRetries429, + s.NetworkRetries, + s.NetworkFailures, + s.NetworkSeriesSent, + s.NetworkErrors, + s.NetworkNewestOutTimeStampSeconds, + s.FilequeueInSeries, + s.FilequeueErrors, + s.FilequeueNewestInTimeStampSeconds, + ) + return s +} + +func (s *PrometheusStats) BackwardsCompatibility(registry prometheus.Registerer) { + registry.MustRegister( + s.RemoteStorageDuration, + s.RemoteStorageInTimestamp, + s.RemoteStorageOutTimestamp, + s.SamplesTotal, + s.HistogramsTotal, + s.MetadataTotal, + s.FailedSamplesTotal, + s.FailedHistogramsTotal, + s.FailedMetadataTotal, + s.RetriedSamplesTotal, + s.RetriedHistogramsTotal, + s.RetriedMetadataTotal, + s.SentBytesTotal, + s.MetadataBytesTotal, + ) +} + +func (s *PrometheusStats) UpdateNetwork(stats NetworkStats) { + s.NetworkSeriesSent.Add(float64(stats.TotalSent())) + s.NetworkRetries.Add(float64(stats.TotalRetried())) + s.NetworkFailures.Add(float64(stats.TotalFailed())) + s.NetworkRetries429.Add(float64(stats.Total429())) + s.NetworkRetries5XX.Add(float64(stats.Total5XX())) + s.NetworkSentDuration.Observe(stats.SendDuration.Seconds()) + s.RemoteStorageDuration.Observe(stats.SendDuration.Seconds()) + // The newest timestamp is no always sent. + if stats.NewestTimestamp != 0 { + s.RemoteStorageOutTimestamp.Set(float64(stats.NewestTimestamp)) + } + + s.SamplesTotal.Add(float64(stats.Series.SeriesSent)) + s.MetadataTotal.Add(float64(stats.Metadata.SeriesSent)) + s.HistogramsTotal.Add(float64(stats.Histogram.SeriesSent)) + + s.FailedSamplesTotal.Add(float64(stats.Series.FailedSamples)) + s.FailedMetadataTotal.Add(float64(stats.Metadata.FailedSamples)) + s.FailedHistogramsTotal.Add(float64(stats.Histogram.FailedSamples)) + + s.RetriedSamplesTotal.Add(float64(stats.Series.RetriedSamples)) + s.RetriedHistogramsTotal.Add(float64(stats.Histogram.RetriedSamples)) + s.RetriedMetadataTotal.Add(float64(stats.Metadata.RetriedSamples)) + + s.MetadataBytesTotal.Add(float64(stats.MetadataBytes)) + s.SentBytesTotal.Add(float64(stats.SeriesBytes)) +} + +func (s *PrometheusStats) UpdateFileQueue(stats FileQueueStats) { + s.FilequeueInSeries.Add(float64(stats.SeriesStored)) + s.FilequeueErrors.Add(float64(stats.Errors)) + if stats.NewestTimestamp != 0 { + s.FilequeueNewestInTimeStampSeconds.Set(float64(stats.NewestTimestamp)) + s.RemoteStorageInTimestamp.Set(float64(stats.NewestTimestamp)) + } +} + +type NetworkStats struct { + Series CategoryStats + Histogram CategoryStats + Metadata CategoryStats + SendDuration time.Duration + NewestTimestamp int64 + SeriesBytes int + MetadataBytes int +} + +func (ns NetworkStats) TotalSent() int { + return ns.Series.SeriesSent + ns.Histogram.SeriesSent + ns.Metadata.SeriesSent +} + +func (ns NetworkStats) TotalRetried() int { + return ns.Series.RetriedSamples + ns.Histogram.RetriedSamples + ns.Metadata.RetriedSamples +} + +func (ns NetworkStats) TotalFailed() int { + return ns.Series.FailedSamples + ns.Histogram.FailedSamples + ns.Metadata.FailedSamples +} + +func (ns NetworkStats) Total429() int { + return ns.Series.RetriedSamples429 + ns.Histogram.RetriedSamples429 + ns.Metadata.RetriedSamples429 +} + +func (ns NetworkStats) Total5XX() int { + return ns.Series.RetriedSamples5XX + ns.Histogram.RetriedSamples5XX + ns.Metadata.RetriedSamples5XX +} + +type CategoryStats struct { + RetriedSamples int + RetriedSamples429 int + RetriedSamples5XX int + SeriesSent int + FailedSamples int + NetworkSamplesFailed int +} From 9e84aee75496bcc6f6aeed289521e98ce5a09949 Mon Sep 17 00:00:00 2001 From: matt durham Date: Fri, 20 Sep 2024 09:36:08 -0400 Subject: [PATCH 02/44] Fix for config updating and tests. --- Makefile | 2 +- .../prometheus/remote/queue/network/config.go | 24 -- .../prometheus/remote/queue/network/loop.go | 4 +- .../remote/queue/network/manager.go | 61 ++-- .../remote/queue/network/manager_test.go | 313 ++++++++++++++++++ .../prometheus/remote/queue/types/network.go | 20 ++ .../prometheus/remote/queue/types/stats.go | 2 +- 7 files changed, 372 insertions(+), 54 deletions(-) delete mode 100644 internal/component/prometheus/remote/queue/network/config.go create mode 100644 internal/component/prometheus/remote/queue/network/manager_test.go diff --git a/Makefile b/Makefile index 3badbe3a5d..be8ec31531 100644 --- a/Makefile +++ b/Makefile @@ -141,7 +141,7 @@ lint: alloylint # final command runs tests for all other submodules. test: $(GO_ENV) go test $(GO_FLAGS) -race $(shell go list ./... | grep -v /integration-tests/) - $(GO_ENV) go test $(GO_FLAGS) ./internal/static/integrations/node_exporter ./internal/static/logs ./internal/component/otelcol/processor/tail_sampling ./internal/component/loki/source/file ./internal/component/loki/source/docker ./internal/component/prometheus/remote/queue/serialization + $(GO_ENV) go test $(GO_FLAGS) ./internal/static/integrations/node_exporter ./internal/static/logs ./internal/component/otelcol/processor/tail_sampling ./internal/component/loki/source/file ./internal/component/loki/source/docker ./internal/component/prometheus/remote/queue/serialization ./internal/component/prometheus/remote/queue/network $(GO_ENV) find . -name go.mod -not -path "./go.mod" -execdir go test -race ./... \; test-packages: diff --git a/internal/component/prometheus/remote/queue/network/config.go b/internal/component/prometheus/remote/queue/network/config.go deleted file mode 100644 index bccd74becf..0000000000 --- a/internal/component/prometheus/remote/queue/network/config.go +++ /dev/null @@ -1,24 +0,0 @@ -package network - -import ( - "reflect" - "time" -) - -type ConnectionConfig struct { - URL string - Username string - Password string - UserAgent string - Timeout time.Duration - RetryBackoff time.Duration - MaxRetryBackoffAttempts time.Duration - BatchCount int - FlushFrequency time.Duration - ExternalLabels map[string]string - Connections uint64 -} - -func (cc ConnectionConfig) Equals(bb ConnectionConfig) bool { - return reflect.DeepEqual(cc, bb) -} diff --git a/internal/component/prometheus/remote/queue/network/loop.go b/internal/component/prometheus/remote/queue/network/loop.go index 62293f4aa6..f80b3a4155 100644 --- a/internal/component/prometheus/remote/queue/network/loop.go +++ b/internal/component/prometheus/remote/queue/network/loop.go @@ -32,7 +32,7 @@ type loop struct { client *http.Client batchCount int flushFrequency time.Duration - cfg ConnectionConfig + cfg types.ConnectionConfig log log.Logger lastSend time.Time statsFunc func(s types.NetworkStats) @@ -46,7 +46,7 @@ type loop struct { sendBuffer []byte } -func newLoop(cc ConnectionConfig, isMetaData bool, log log.Logger, stats func(s types.NetworkStats)) *loop { +func newLoop(cc types.ConnectionConfig, isMetaData bool, log log.Logger, stats func(s types.NetworkStats)) *loop { // TODO @mattdurham add TLS support afer the initial push. l := &loop{ isMeta: isMetaData, diff --git a/internal/component/prometheus/remote/queue/network/manager.go b/internal/component/prometheus/remote/queue/network/manager.go index d942d05a0e..697f676fad 100644 --- a/internal/component/prometheus/remote/queue/network/manager.go +++ b/internal/component/prometheus/remote/queue/network/manager.go @@ -18,9 +18,9 @@ type manager struct { logger log.Logger inbox actor.Mailbox[*types.TimeSeriesBinary] metaInbox actor.Mailbox[*types.TimeSeriesBinary] - configInbox actor.Mailbox[ConnectionConfig] + configInbox actor.Mailbox[types.ConnectionConfig] self actor.Actor - cfg ConnectionConfig + cfg types.ConnectionConfig stats func(types.NetworkStats) metaStats func(types.NetworkStats) } @@ -29,17 +29,18 @@ var _ types.NetworkClient = (*manager)(nil) var _ actor.Worker = (*manager)(nil) -func New(cc ConnectionConfig, logger log.Logger, seriesStats, metadataStats func(types.NetworkStats)) (types.NetworkClient, error) { +func New(cc types.ConnectionConfig, logger log.Logger, seriesStats, metadataStats func(types.NetworkStats)) (types.NetworkClient, error) { s := &manager{ connectionCount: cc.Connections, - loops: make([]*loop, 0), + loops: make([]*loop, 0, cc.Connections), logger: logger, // This provides blocking to only handle one at a time, so that if a queue blocks // it will stop the filequeue from feeding more. inbox: actor.NewMailbox[*types.TimeSeriesBinary](actor.OptCapacity(1)), metaInbox: actor.NewMailbox[*types.TimeSeriesBinary](actor.OptCapacity(1)), - configInbox: actor.NewMailbox[ConnectionConfig](), + configInbox: actor.NewMailbox[types.ConnectionConfig](), stats: seriesStats, + cfg: cc, } // start kicks off a number of concurrent connections. @@ -56,16 +57,11 @@ func New(cc ConnectionConfig, logger log.Logger, seriesStats, metadataStats func } func (s *manager) Start() { - actors := make([]actor.Actor, 0) - for _, l := range s.loops { - l.Start() - } - actors = append(actors, s.metadata.actors()...) - actors = append(actors, s.inbox) - actors = append(actors, s.metaInbox) - actors = append(actors, actor.New(s)) - actors = append(actors, s.configInbox) - s.self = actor.Combine(actors...).Build() + s.startLoops() + s.configInbox.Start() + s.metaInbox.Start() + s.inbox.Start() + s.self = actor.New(s) s.self.Start() } @@ -77,7 +73,7 @@ func (s *manager) SendMetadata(ctx context.Context, data *types.TimeSeriesBinary return s.metaInbox.Send(ctx, data) } -func (s *manager) UpdateConfig(ctx context.Context, cc ConnectionConfig) error { +func (s *manager) UpdateConfig(ctx context.Context, cc types.ConnectionConfig) error { return s.configInbox.Send(ctx, cc) } @@ -114,40 +110,53 @@ func (s *manager) DoWork(ctx actor.Context) actor.WorkerStatus { } } -func (s *manager) updateConfig(cc ConnectionConfig) { +func (s *manager) updateConfig(cc types.ConnectionConfig) { // No need to do anything if the configuration is the same. if s.cfg.Equals(cc) { return } + s.cfg = cc // TODO @mattdurham make this smarter, at the moment any samples in the loops are lost. // Ideally we would drain the queues and re add them but that is a future need. // In practice this shouldn't change often so data loss should be minimal. // For the moment we will stop all the items and recreate them. - for _, l := range s.loops { - l.Stop() - } - s.metadata.Stop() - - s.loops = make([]*loop, 0) + s.stopLoops() + s.loops = make([]*loop, 0, s.connectionCount) var i uint64 for ; i < s.connectionCount; i++ { l := newLoop(cc, false, s.logger, s.stats) l.self = actor.New(l) + s.loops = append(s.loops, l) } s.metadata = newLoop(cc, true, s.logger, s.metaStats) s.metadata.self = actor.New(s.metadata) + s.startLoops() } func (s *manager) Stop() { - level.Debug(s.logger).Log("msg", "stopping manager") + s.stopLoops() + s.configInbox.Stop() + s.metaInbox.Stop() + s.inbox.Stop() + s.self.Stop() +} + +func (s *manager) stopLoops() { for _, l := range s.loops { - l.Stop() l.stopCalled.Store(true) + l.Stop() } s.metadata.stopCalled.Store(true) - s.self.Stop() + s.metadata.Stop() +} + +func (s *manager) startLoops() { + for _, l := range s.loops { + l.Start() + } + s.metadata.Start() } // Queue adds anything thats not metadata to the queue. diff --git a/internal/component/prometheus/remote/queue/network/manager_test.go b/internal/component/prometheus/remote/queue/network/manager_test.go new file mode 100644 index 0000000000..013f2b9196 --- /dev/null +++ b/internal/component/prometheus/remote/queue/network/manager_test.go @@ -0,0 +1,313 @@ +//go:build !race + +package network + +import ( + "context" + "io" + "math/rand" + "net/http" + "net/http/httptest" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/golang/snappy" + "github.com/grafana/alloy/internal/component/prometheus/remote/queue/types" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/prompb" + "github.com/stretchr/testify/require" + "go.uber.org/atomic" + "go.uber.org/goleak" +) + +func TestSending(t *testing.T) { + defer goleak.VerifyNone(t) + + recordsFound := atomic.Uint32{} + svr := httptest.NewServer(handler(t, http.StatusOK, func(wr *prompb.WriteRequest) { + recordsFound.Add(uint32(len(wr.Timeseries))) + })) + + defer svr.Close() + ctx := context.Background() + ctx, cncl := context.WithCancel(ctx) + defer cncl() + + cc := types.ConnectionConfig{ + URL: svr.URL, + Timeout: 1 * time.Second, + BatchCount: 10, + FlushFrequency: 1 * time.Second, + Connections: 4, + } + + logger := log.NewNopLogger() + wr, err := New(cc, logger, func(s types.NetworkStats) {}, func(s types.NetworkStats) {}) + wr.Start() + defer wr.Stop() + require.NoError(t, err) + for i := 0; i < 1_000; i++ { + send(t, wr, ctx) + } + require.Eventually(t, func() bool { + return recordsFound.Load() == 1_000 + }, 10*time.Second, 100*time.Millisecond) +} + +func TestUpdatingConfig(t *testing.T) { + defer goleak.VerifyNone(t) + + recordsFound := atomic.Uint32{} + svr := httptest.NewServer(handler(t, http.StatusOK, func(wr *prompb.WriteRequest) { + recordsFound.Add(uint32(len(wr.Timeseries))) + })) + + defer svr.Close() + ctx := context.Background() + ctx, cncl := context.WithCancel(ctx) + defer cncl() + + cc := types.ConnectionConfig{ + URL: svr.URL, + Timeout: 1 * time.Second, + BatchCount: 10, + FlushFrequency: 1 * time.Second, + Connections: 4, + } + + logger := log.NewNopLogger() + wr, err := New(cc, logger, func(s types.NetworkStats) {}, func(s types.NetworkStats) {}) + wr.Start() + defer wr.Stop() + + cc2 := types.ConnectionConfig{ + URL: svr.URL, + Timeout: 5 * time.Second, + BatchCount: 100, + FlushFrequency: 1 * time.Second, + Connections: 4, + } + + err = wr.UpdateConfig(context.Background(), cc2) + require.NoError(t, err) + for i := 0; i < 1_000; i++ { + send(t, wr, ctx) + } + require.Eventuallyf(t, func() bool { + return recordsFound.Load() == 1_000 + }, 10*time.Second, 1*time.Second, "record count should be 1000 but is %d", recordsFound.Load()) + + require.Truef(t, wr.(*manager).cfg.BatchCount == 100, "batch_count should be 100 but is %d", wr.(*manager).cfg.BatchCount) + +} + +func TestRetry(t *testing.T) { + defer goleak.VerifyNone(t) + + retries := atomic.Uint32{} + var previous *prompb.WriteRequest + svr := httptest.NewServer(handler(t, http.StatusTooManyRequests, func(wr *prompb.WriteRequest) { + retries.Add(1) + // Check that we are getting the same sample back. + if previous == nil { + previous = wr + } else { + require.True(t, previous.Timeseries[0].Labels[0].Value == wr.Timeseries[0].Labels[0].Value) + } + })) + defer svr.Close() + ctx := context.Background() + ctx, cncl := context.WithCancel(ctx) + defer cncl() + + cc := types.ConnectionConfig{ + URL: svr.URL, + Timeout: 1 * time.Second, + BatchCount: 1, + FlushFrequency: 1 * time.Second, + RetryBackoff: 100 * time.Millisecond, + Connections: 1, + } + + logger := log.NewNopLogger() + wr, err := New(cc, logger, func(s types.NetworkStats) {}, func(s types.NetworkStats) {}) + require.NoError(t, err) + wr.Start() + defer wr.Stop() + + for i := 0; i < 10; i++ { + send(t, wr, ctx) + } + require.Eventually(t, func() bool { + done := retries.Load() > 5 + return done + }, 10*time.Second, 1*time.Second) +} + +func TestRetryBounded(t *testing.T) { + defer goleak.VerifyNone(t) + + sends := atomic.Uint32{} + svr := httptest.NewServer(handler(t, http.StatusTooManyRequests, func(wr *prompb.WriteRequest) { + sends.Add(1) + })) + + defer svr.Close() + ctx := context.Background() + ctx, cncl := context.WithCancel(ctx) + defer cncl() + + cc := types.ConnectionConfig{ + URL: svr.URL, + Timeout: 1 * time.Second, + BatchCount: 1, + FlushFrequency: 1 * time.Second, + RetryBackoff: 100 * time.Millisecond, + MaxRetryBackoffAttempts: 1, + Connections: 1, + } + + logger := log.NewNopLogger() + wr, err := New(cc, logger, func(s types.NetworkStats) {}, func(s types.NetworkStats) {}) + wr.Start() + defer wr.Stop() + require.NoError(t, err) + for i := 0; i < 10; i++ { + send(t, wr, ctx) + } + require.Eventually(t, func() bool { + // We send 10 but each one gets retried once so 20 total. + return sends.Load() == 10*2 + }, 2*time.Second, 100*time.Millisecond) + time.Sleep(2 * time.Second) + // Ensure we dont get any more. + require.True(t, sends.Load() == 10*2) +} + +func TestRecoverable(t *testing.T) { + defer goleak.VerifyNone(t) + + recoverable := atomic.Uint32{} + svr := httptest.NewServer(handler(t, http.StatusInternalServerError, func(wr *prompb.WriteRequest) { + })) + defer svr.Close() + ctx := context.Background() + ctx, cncl := context.WithCancel(ctx) + defer cncl() + + cc := types.ConnectionConfig{ + URL: svr.URL, + Timeout: 1 * time.Second, + BatchCount: 1, + FlushFrequency: 1 * time.Second, + RetryBackoff: 100 * time.Millisecond, + MaxRetryBackoffAttempts: 1, + Connections: 1, + } + + logger := log.NewNopLogger() + wr, err := New(cc, logger, func(s types.NetworkStats) { + recoverable.Add(uint32(s.Total5XX())) + }, func(s types.NetworkStats) {}) + require.NoError(t, err) + wr.Start() + defer wr.Stop() + for i := 0; i < 10; i++ { + send(t, wr, ctx) + } + require.Eventually(t, func() bool { + // We send 10 but each one gets retried once so 20 total. + return recoverable.Load() == 10*2 + }, 2*time.Second, 100*time.Millisecond) + time.Sleep(2 * time.Second) + // Ensure we dont get any more. + require.True(t, recoverable.Load() == 10*2) +} + +func TestNonRecoverable(t *testing.T) { + defer goleak.VerifyNone(t) + + nonRecoverable := atomic.Uint32{} + svr := httptest.NewServer(handler(t, http.StatusBadRequest, func(wr *prompb.WriteRequest) { + })) + + defer svr.Close() + ctx := context.Background() + ctx, cncl := context.WithCancel(ctx) + defer cncl() + + cc := types.ConnectionConfig{ + URL: svr.URL, + Timeout: 1 * time.Second, + BatchCount: 1, + FlushFrequency: 1 * time.Second, + RetryBackoff: 100 * time.Millisecond, + MaxRetryBackoffAttempts: 1, + Connections: 1, + } + + logger := log.NewNopLogger() + wr, err := New(cc, logger, func(s types.NetworkStats) { + nonRecoverable.Add(uint32(s.TotalFailed())) + }, func(s types.NetworkStats) {}) + wr.Start() + defer wr.Stop() + require.NoError(t, err) + for i := 0; i < 10; i++ { + send(t, wr, ctx) + } + require.Eventually(t, func() bool { + return nonRecoverable.Load() == 10 + }, 2*time.Second, 100*time.Millisecond) + time.Sleep(2 * time.Second) + // Ensure we dont get any more. + require.True(t, nonRecoverable.Load() == 10) +} + +func send(t *testing.T, wr types.NetworkClient, ctx context.Context) { + ts := createSeries(t) + // The actual hash is only used for queueing into different buckets. + err := wr.SendSeries(ctx, ts) + require.NoError(t, err) +} + +func handler(t *testing.T, code int, callback func(wr *prompb.WriteRequest)) http.HandlerFunc { + return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + buf, err := io.ReadAll(r.Body) + require.NoError(t, err) + defer r.Body.Close() + decoded, err := snappy.Decode(nil, buf) + require.NoError(t, err) + + wr := &prompb.WriteRequest{} + err = wr.Unmarshal(decoded) + require.NoError(t, err) + callback(wr) + w.WriteHeader(code) + }) +} + +func createSeries(_ *testing.T) *types.TimeSeriesBinary { + ts := &types.TimeSeriesBinary{ + TS: time.Now().Unix(), + Value: 1, + Labels: []labels.Label{ + { + Name: "__name__", + Value: randSeq(10), + }, + }, + } + return ts +} + +var letters = []rune("abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ") + +func randSeq(n int) string { + b := make([]rune, n) + for i := range b { + b[i] = letters[rand.Intn(len(letters))] + } + return string(b) +} diff --git a/internal/component/prometheus/remote/queue/types/network.go b/internal/component/prometheus/remote/queue/types/network.go index 44ee5e089d..024f8a3122 100644 --- a/internal/component/prometheus/remote/queue/types/network.go +++ b/internal/component/prometheus/remote/queue/types/network.go @@ -2,6 +2,8 @@ package types import ( "context" + "reflect" + "time" ) type NetworkClient interface { @@ -9,4 +11,22 @@ type NetworkClient interface { Stop() SendSeries(ctx context.Context, d *TimeSeriesBinary) error SendMetadata(ctx context.Context, d *TimeSeriesBinary) error + UpdateConfig(ctx context.Context, cfg ConnectionConfig) error +} +type ConnectionConfig struct { + URL string + Username string + Password string + UserAgent string + Timeout time.Duration + RetryBackoff time.Duration + MaxRetryBackoffAttempts time.Duration + BatchCount int + FlushFrequency time.Duration + ExternalLabels map[string]string + Connections uint64 +} + +func (cc ConnectionConfig) Equals(bb ConnectionConfig) bool { + return reflect.DeepEqual(cc, bb) } diff --git a/internal/component/prometheus/remote/queue/types/stats.go b/internal/component/prometheus/remote/queue/types/stats.go index 06dcde57fb..4107d8089f 100644 --- a/internal/component/prometheus/remote/queue/types/stats.go +++ b/internal/component/prometheus/remote/queue/types/stats.go @@ -230,7 +230,7 @@ func (s *PrometheusStats) UpdateNetwork(stats NetworkStats) { s.SentBytesTotal.Add(float64(stats.SeriesBytes)) } -func (s *PrometheusStats) UpdateFileQueue(stats FileQueueStats) { +func (s *PrometheusStats) UpdateFileQueue(stats SerializerStats) { s.FilequeueInSeries.Add(float64(stats.SeriesStored)) s.FilequeueErrors.Add(float64(stats.Errors)) if stats.NewestTimestamp != 0 { From 1cf1e7a8d40e040c9981c6f5605a005e9df9d76e Mon Sep 17 00:00:00 2001 From: mattdurham Date: Tue, 1 Oct 2024 14:00:17 -0400 Subject: [PATCH 03/44] Update internal/component/prometheus/remote/queue/network/loop.go Co-authored-by: William Dumont --- internal/component/prometheus/remote/queue/network/loop.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/component/prometheus/remote/queue/network/loop.go b/internal/component/prometheus/remote/queue/network/loop.go index f80b3a4155..8135afac46 100644 --- a/internal/component/prometheus/remote/queue/network/loop.go +++ b/internal/component/prometheus/remote/queue/network/loop.go @@ -23,7 +23,7 @@ import ( var _ actor.Worker = (*loop)(nil) -// loop handles the low level sending of data. It conceptually a queue. +// loop handles the low level sending of data. It's conceptually a queue. // loop makes no attempt to save or restore signals in the queue. // loop config cannot be updated, it is easier to recreate. This does mean we lose any signals in the queue. type loop struct { From 5c49e9ef8b07b3a5a6f2360195623cfa34a55bfa Mon Sep 17 00:00:00 2001 From: mattdurham Date: Tue, 1 Oct 2024 14:29:19 -0400 Subject: [PATCH 04/44] Update internal/component/prometheus/remote/queue/network/loop.go Co-authored-by: Piotr <17101802+thampiotr@users.noreply.github.com> --- internal/component/prometheus/remote/queue/network/loop.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/internal/component/prometheus/remote/queue/network/loop.go b/internal/component/prometheus/remote/queue/network/loop.go index 8135afac46..e579b76e72 100644 --- a/internal/component/prometheus/remote/queue/network/loop.go +++ b/internal/component/prometheus/remote/queue/network/loop.go @@ -361,8 +361,7 @@ func toMetadata(ts *types.TimeSeriesBinary) (prompb.MetricMetadata, bool) { } func retryAfterDuration(defaultDuration time.Duration, t string) time.Duration { - parsedTime, err := time.Parse(http.TimeFormat, t) - if err == nil { + if parsedTime, err := time.Parse(http.TimeFormat, t); err == nil { return time.Until(parsedTime) } // The duration can be in seconds. From 02a41e0e1259e6028ca9f55e94e87f143f13f342 Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 1 Oct 2024 14:35:20 -0400 Subject: [PATCH 05/44] pr feedback --- .../prometheus/remote/queue/network/loop.go | 86 +++++++++---------- .../remote/queue/network/manager.go | 38 ++++---- 2 files changed, 58 insertions(+), 66 deletions(-) diff --git a/internal/component/prometheus/remote/queue/network/loop.go b/internal/component/prometheus/remote/queue/network/loop.go index 8135afac46..0f5bf3d583 100644 --- a/internal/component/prometheus/remote/queue/network/loop.go +++ b/internal/component/prometheus/remote/queue/network/loop.go @@ -30,8 +30,6 @@ type loop struct { isMeta bool seriesMbx actor.Mailbox[*types.TimeSeriesBinary] client *http.Client - batchCount int - flushFrequency time.Duration cfg types.ConnectionConfig log log.Logger lastSend time.Time @@ -48,12 +46,10 @@ type loop struct { func newLoop(cc types.ConnectionConfig, isMetaData bool, log log.Logger, stats func(s types.NetworkStats)) *loop { // TODO @mattdurham add TLS support afer the initial push. - l := &loop{ + return &loop{ isMeta: isMetaData, // In general we want a healthy queue of items, in this case we want to have 2x our maximum send sized ready. seriesMbx: actor.NewMailbox[*types.TimeSeriesBinary](actor.OptCapacity(2 * cc.BatchCount)), - batchCount: cc.BatchCount, - flushFrequency: cc.FlushFrequency, client: &http.Client{}, cfg: cc, log: log, @@ -62,13 +58,11 @@ func newLoop(cc types.ConnectionConfig, isMetaData bool, log log.Logger, stats f ticker: time.NewTicker(1 * time.Second), buf: proto.NewBuffer(nil), sendBuffer: make([]byte, 0), + req: &prompb.WriteRequest{ + // We know BatchCount is the most we will ever send. + Timeseries: make([]prompb.TimeSeries, 0, cc.BatchCount), + }, } - l.req = &prompb.WriteRequest{ - // We know BatchCount is the most we will ever send. - Timeseries: make([]prompb.TimeSeries, 0, cc.BatchCount), - } - - return l } func (l *loop) Start() { @@ -92,14 +86,13 @@ func (l *loop) DoWork(ctx actor.Context) actor.WorkerStatus { // Main select loop select { case <-ctx.Done(): - l.stopCalled.Store(true) return actor.WorkerEnd // Ticker is to ensure the flush timer is called. case <-l.ticker.C: if len(l.series) == 0 { return actor.WorkerContinue } - if time.Since(l.lastSend) > l.flushFrequency { + if time.Since(l.lastSend) > l.cfg.FlushFrequency { l.trySend(ctx) } return actor.WorkerContinue @@ -108,7 +101,7 @@ func (l *loop) DoWork(ctx actor.Context) actor.WorkerStatus { return actor.WorkerEnd } l.series = append(l.series, series) - if len(l.series) >= l.batchCount { + if len(l.series) >= l.cfg.BatchCount { l.trySend(ctx) } return actor.WorkerContinue @@ -118,34 +111,37 @@ func (l *loop) DoWork(ctx actor.Context) actor.WorkerStatus { // trySend is the core functionality for sending data to a endpoint. It will attempt retries as defined in MaxRetryBackoffAttempts. func (l *loop) trySend(ctx context.Context) { attempts := 0 -attempt: - start := time.Now() - result := l.send(ctx, attempts) - duration := time.Since(start) - l.statsFunc(types.NetworkStats{ - SendDuration: duration, - }) - if result.successful { - l.sendingCleanup() - return - } - if !result.recoverableError { - l.sendingCleanup() - return - } - attempts++ - if attempts > int(l.cfg.MaxRetryBackoffAttempts) && l.cfg.MaxRetryBackoffAttempts > 0 { - level.Debug(l.log).Log("msg", "max retry attempts reached", "attempts", attempts) - l.sendingCleanup() - return - } - // This helps us short circuit the loop if we are stopping. - if l.stopCalled.Load() { - return + for { + start := time.Now() + result := l.send(ctx, attempts) + duration := time.Since(start) + l.statsFunc(types.NetworkStats{ + SendDuration: duration, + }) + if result.err != nil { + level.Error(l.log).Log("msg", "error in sending telemetry", "err", result.err.Error()) + } + if result.successful { + l.sendingCleanup() + return + } + if !result.recoverableError { + l.sendingCleanup() + return + } + attempts++ + if attempts > int(l.cfg.MaxRetryBackoffAttempts) && l.cfg.MaxRetryBackoffAttempts > 0 { + level.Debug(l.log).Log("msg", "max retry attempts reached", "attempts", attempts) + l.sendingCleanup() + return + } + // This helps us short circuit the loop if we are stopping. + if l.stopCalled.Load() { + return + } + // Sleep between attempts. + time.Sleep(result.retryAfter) } - // Sleep between attempts. - time.Sleep(result.retryAfter) - goto attempt } type sendResult struct { @@ -160,7 +156,7 @@ type sendResult struct { func (l *loop) sendingCleanup() { types.PutTimeSeriesSliceIntoPool(l.series) l.sendBuffer = l.sendBuffer[:0] - l.series = make([]*types.TimeSeriesBinary, 0, l.batchCount) + l.series = make([]*types.TimeSeriesBinary, 0, l.cfg.BatchCount) l.lastSend = time.Now() } @@ -170,14 +166,13 @@ func (l *loop) send(ctx context.Context, retryCount int) sendResult { defer func() { recordStats(l.series, l.isMeta, l.statsFunc, result, len(l.sendBuffer)) }() - var err error // Check to see if this is a retry and we can reuse the buffer. // I wonder if we should do this, its possible we are sending things that have exceeded the TTL. if len(l.sendBuffer) == 0 { var data []byte var wrErr error if l.isMeta { - data, wrErr = createWriteRequestMetadata(l.req, l.series, l.buf) + data, wrErr = createWriteRequestMetadata(l.log, l.req, l.series, l.buf) } else { data, wrErr = createWriteRequest(l.req, l.series, l.externalLabels, l.buf) } @@ -313,7 +308,7 @@ func createWriteRequest(wr *prompb.WriteRequest, series []*types.TimeSeriesBinar return data.Bytes(), err } -func createWriteRequestMetadata(wr *prompb.WriteRequest, series []*types.TimeSeriesBinary, data *proto.Buffer) ([]byte, error) { +func createWriteRequestMetadata(l log.Logger, wr *prompb.WriteRequest, series []*types.TimeSeriesBinary, data *proto.Buffer) ([]byte, error) { if cap(wr.Metadata) < len(series) { wr.Metadata = make([]prompb.MetricMetadata, len(series)) } else { @@ -323,6 +318,7 @@ func createWriteRequestMetadata(wr *prompb.WriteRequest, series []*types.TimeSer for i, ts := range series { mt, valid := toMetadata(ts) if !valid { + level.Error(l).Log("msg", "invalid metadata was found", "labels", ts.Labels.String()) continue } wr.Metadata[i] = mt diff --git a/internal/component/prometheus/remote/queue/network/manager.go b/internal/component/prometheus/remote/queue/network/manager.go index 697f676fad..19e4f5b961 100644 --- a/internal/component/prometheus/remote/queue/network/manager.go +++ b/internal/component/prometheus/remote/queue/network/manager.go @@ -12,17 +12,16 @@ import ( // manager manages loops. Mostly it exists to control their lifecycle and send work to them. type manager struct { - connectionCount uint64 - loops []*loop - metadata *loop - logger log.Logger - inbox actor.Mailbox[*types.TimeSeriesBinary] - metaInbox actor.Mailbox[*types.TimeSeriesBinary] - configInbox actor.Mailbox[types.ConnectionConfig] - self actor.Actor - cfg types.ConnectionConfig - stats func(types.NetworkStats) - metaStats func(types.NetworkStats) + loops []*loop + metadata *loop + logger log.Logger + inbox actor.Mailbox[*types.TimeSeriesBinary] + metaInbox actor.Mailbox[*types.TimeSeriesBinary] + configInbox actor.Mailbox[types.ConnectionConfig] + self actor.Actor + cfg types.ConnectionConfig + stats func(types.NetworkStats) + metaStats func(types.NetworkStats) } var _ types.NetworkClient = (*manager)(nil) @@ -31,9 +30,8 @@ var _ actor.Worker = (*manager)(nil) func New(cc types.ConnectionConfig, logger log.Logger, seriesStats, metadataStats func(types.NetworkStats)) (types.NetworkClient, error) { s := &manager{ - connectionCount: cc.Connections, - loops: make([]*loop, 0, cc.Connections), - logger: logger, + loops: make([]*loop, 0, cc.Connections), + logger: logger, // This provides blocking to only handle one at a time, so that if a queue blocks // it will stop the filequeue from feeding more. inbox: actor.NewMailbox[*types.TimeSeriesBinary](actor.OptCapacity(1)), @@ -44,8 +42,7 @@ func New(cc types.ConnectionConfig, logger log.Logger, seriesStats, metadataStat } // start kicks off a number of concurrent connections. - var i uint64 - for ; i < s.connectionCount; i++ { + for i := uint64(0); i < s.cfg.Connections; i++ { l := newLoop(cc, false, logger, seriesStats) l.self = actor.New(l) s.loops = append(s.loops, l) @@ -120,10 +117,11 @@ func (s *manager) updateConfig(cc types.ConnectionConfig) { // Ideally we would drain the queues and re add them but that is a future need. // In practice this shouldn't change often so data loss should be minimal. // For the moment we will stop all the items and recreate them. + level.Debug(s.logger).Log("msg", "dropping all series in loops and creating queue due to config change") s.stopLoops() - s.loops = make([]*loop, 0, s.connectionCount) + s.loops = make([]*loop, 0, s.cfg.Connections) var i uint64 - for ; i < s.connectionCount; i++ { + for ; i < s.cfg.Connections; i++ { l := newLoop(cc, false, s.logger, s.stats) l.self = actor.New(l) @@ -145,10 +143,8 @@ func (s *manager) Stop() { func (s *manager) stopLoops() { for _, l := range s.loops { - l.stopCalled.Store(true) l.Stop() } - s.metadata.stopCalled.Store(true) s.metadata.Stop() } @@ -162,7 +158,7 @@ func (s *manager) startLoops() { // Queue adds anything thats not metadata to the queue. func (s *manager) queue(ctx context.Context, ts *types.TimeSeriesBinary) { // Based on a hash which is the label hash add to the queue. - queueNum := ts.Hash % s.connectionCount + queueNum := ts.Hash % s.cfg.Connections // This will block if the queue is full. err := s.loops[queueNum].seriesMbx.Send(ctx, ts) if err != nil { From a638c1aa83d9c2a97e784f6a4d78cc402523eac3 Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 1 Oct 2024 14:43:24 -0400 Subject: [PATCH 06/44] pr feedback --- internal/component/prometheus/remote/queue/network/loop.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/internal/component/prometheus/remote/queue/network/loop.go b/internal/component/prometheus/remote/queue/network/loop.go index f300749be1..cbe94f2e41 100644 --- a/internal/component/prometheus/remote/queue/network/loop.go +++ b/internal/component/prometheus/remote/queue/network/loop.go @@ -44,7 +44,7 @@ type loop struct { sendBuffer []byte } -func newLoop(cc types.ConnectionConfig, isMetaData bool, log log.Logger, stats func(s types.NetworkStats)) *loop { +func newLoop(cc types.ConnectionConfig, isMetaData bool, l log.Logger, stats func(s types.NetworkStats)) *loop { // TODO @mattdurham add TLS support afer the initial push. return &loop{ isMeta: isMetaData, @@ -52,7 +52,7 @@ func newLoop(cc types.ConnectionConfig, isMetaData bool, log log.Logger, stats f seriesMbx: actor.NewMailbox[*types.TimeSeriesBinary](actor.OptCapacity(2 * cc.BatchCount)), client: &http.Client{}, cfg: cc, - log: log, + log: log.With(l, "name", "loop", "url", cc.URL), statsFunc: stats, externalLabels: cc.ExternalLabels, ticker: time.NewTicker(1 * time.Second), @@ -215,6 +215,7 @@ func (l *loop) send(ctx context.Context, retryCount int) sendResult { defer resp.Body.Close() // 500 errors are considered recoverable. if resp.StatusCode/100 == 5 || resp.StatusCode == http.StatusTooManyRequests { + result.err = fmt.Errorf("server responded with status code %d", resp.StatusCode) result.retryAfter = retryAfterDuration(l.cfg.RetryBackoff, resp.Header.Get("Retry-After")) result.recoverableError = true return result From 5abe2716db70a662ea110d39c3d27eb36b5f443c Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 1 Oct 2024 14:59:30 -0400 Subject: [PATCH 07/44] simplify stats --- .../prometheus/remote/queue/network/stats.go | 42 +++++++++++++------ 1 file changed, 29 insertions(+), 13 deletions(-) diff --git a/internal/component/prometheus/remote/queue/network/stats.go b/internal/component/prometheus/remote/queue/network/stats.go index ec24de5d0d..4b384f51fb 100644 --- a/internal/component/prometheus/remote/queue/network/stats.go +++ b/internal/component/prometheus/remote/queue/network/stats.go @@ -9,14 +9,20 @@ import ( // recordStats determines what values to send to the stats function. This allows for any // number of metrics/signals libraries to be used. Prometheus, OTel, and any other. func recordStats(series []*types.TimeSeriesBinary, isMeta bool, stats func(s types.NetworkStats), r sendResult, bytesSent int) { + seriesCount := getSeriesCount(series) + histogramCount := getHistogramCount(series) + metadataCount := getMetadataCount(series) switch { case r.networkError: stats(types.NetworkStats{ Series: types.CategoryStats{ - NetworkSamplesFailed: getSeriesCount(series), + NetworkSamplesFailed: seriesCount, }, Histogram: types.CategoryStats{ - NetworkSamplesFailed: getHistogramCount(series), + NetworkSamplesFailed: histogramCount, + }, + Metadata: types.CategoryStats{ + NetworkSamplesFailed: metadataCount, }, }) case r.successful: @@ -37,10 +43,13 @@ func recordStats(series []*types.TimeSeriesBinary, isMeta bool, stats func(s typ } stats(types.NetworkStats{ Series: types.CategoryStats{ - SeriesSent: getSeriesCount(series), + SeriesSent: seriesCount, }, Histogram: types.CategoryStats{ - SeriesSent: getHistogramCount(series), + SeriesSent: histogramCount, + }, + Metadata: types.CategoryStats{ + SeriesSent: metadataCount, }, MetadataBytes: metaBytesSent, SeriesBytes: sampleBytesSent, @@ -49,33 +58,40 @@ func recordStats(series []*types.TimeSeriesBinary, isMeta bool, stats func(s typ case r.statusCode == http.StatusTooManyRequests: stats(types.NetworkStats{ Series: types.CategoryStats{ - RetriedSamples: getSeriesCount(series), - RetriedSamples429: getSeriesCount(series), + RetriedSamples: seriesCount, + RetriedSamples429: seriesCount, }, Histogram: types.CategoryStats{ - RetriedSamples: getHistogramCount(series), - RetriedSamples429: getHistogramCount(series), + RetriedSamples: histogramCount, + RetriedSamples429: histogramCount, + }, + Metadata: types.CategoryStats{ + RetriedSamples: metadataCount, + RetriedSamples429: metadataCount, }, }) case r.statusCode/100 == 5: stats(types.NetworkStats{ Series: types.CategoryStats{ - RetriedSamples5XX: getSeriesCount(series), + RetriedSamples5XX: seriesCount, }, Histogram: types.CategoryStats{ - RetriedSamples5XX: getHistogramCount(series), + RetriedSamples5XX: histogramCount, + }, + Metadata: types.CategoryStats{ + RetriedSamples: metadataCount, }, }) case r.statusCode != 200: stats(types.NetworkStats{ Series: types.CategoryStats{ - FailedSamples: getSeriesCount(series), + FailedSamples: seriesCount, }, Histogram: types.CategoryStats{ - FailedSamples: getHistogramCount(series), + FailedSamples: histogramCount, }, Metadata: types.CategoryStats{ - FailedSamples: getMetadataCount(series), + FailedSamples: metadataCount, }, }) } From 2d0eb0045c065a5c09f693644d2869efef9756db Mon Sep 17 00:00:00 2001 From: matt durham Date: Wed, 2 Oct 2024 15:04:22 -0400 Subject: [PATCH 08/44] simplify stats --- .../prometheus/remote/queue/component.go | 165 +++++ .../prometheus/remote/queue/config.go | 1 + .../prometheus/remote/queue/e2e_bench_test.go | 129 ++++ .../prometheus/remote/queue/e2e_stats_test.go | 568 ++++++++++++++++++ .../prometheus/remote/queue/e2e_test.go | 365 +++++++++++ .../prometheus/remote/queue/endpoint.go | 136 +++++ .../prometheus/remote/queue/fanout.go | 85 +++ .../prometheus/remote/queue/types.go | 112 ++++ .../prometheus/remote/queue/types/network.go | 12 +- .../remote/queue/types/storage_test.go | 25 + 10 files changed, 1594 insertions(+), 4 deletions(-) create mode 100644 internal/component/prometheus/remote/queue/component.go create mode 100644 internal/component/prometheus/remote/queue/config.go create mode 100644 internal/component/prometheus/remote/queue/e2e_bench_test.go create mode 100644 internal/component/prometheus/remote/queue/e2e_stats_test.go create mode 100644 internal/component/prometheus/remote/queue/e2e_test.go create mode 100644 internal/component/prometheus/remote/queue/endpoint.go create mode 100644 internal/component/prometheus/remote/queue/fanout.go create mode 100644 internal/component/prometheus/remote/queue/types.go create mode 100644 internal/component/prometheus/remote/queue/types/storage_test.go diff --git a/internal/component/prometheus/remote/queue/component.go b/internal/component/prometheus/remote/queue/component.go new file mode 100644 index 0000000000..1e0c005c3a --- /dev/null +++ b/internal/component/prometheus/remote/queue/component.go @@ -0,0 +1,165 @@ +package queue + +import ( + "context" + "path/filepath" + "reflect" + "sync" + "time" + + "github.com/grafana/alloy/internal/component/prometheus/remote/queue/filequeue" + "github.com/grafana/alloy/internal/component/prometheus/remote/queue/network" + "github.com/prometheus/client_golang/prometheus" + + "github.com/go-kit/log" + "github.com/grafana/alloy/internal/component" + "github.com/grafana/alloy/internal/component/prometheus/remote/queue/serialization" + "github.com/grafana/alloy/internal/component/prometheus/remote/queue/types" + "github.com/grafana/alloy/internal/featuregate" + "github.com/prometheus/prometheus/storage" +) + +func init() { + component.Register(component.Registration{ + Name: "prometheus.remote.queue", + Args: Arguments{}, + Exports: Exports{}, + Stability: featuregate.StabilityExperimental, + Build: func(opts component.Options, args component.Arguments) (component.Component, error) { + return NewComponent(opts, args.(Arguments)) + }, + }) +} + +func NewComponent(opts component.Options, args Arguments) (*Queue, error) { + s := &Queue{ + opts: opts, + args: args, + log: opts.Logger, + endpoints: map[string]*endpoint{}, + } + s.opts.OnStateChange(Exports{Receiver: s}) + err := s.createEndpoints() + if err != nil { + return nil, err + } + return s, nil +} + +// Queue is a queue based WAL used to send data to a remote_write endpoint. Queue supports replaying +// and TTLs. +type Queue struct { + mut sync.RWMutex + args Arguments + opts component.Options + log log.Logger + endpoints map[string]*endpoint +} + +// Run starts the component, blocking until ctx is canceled or the component +// suffers a fatal error. Run is guaranteed to be called exactly once per +// Component. +func (s *Queue) Run(ctx context.Context) error { + defer func() { + s.mut.Lock() + defer s.mut.Unlock() + + for _, ep := range s.endpoints { + ep.Stop() + } + }() + + <-ctx.Done() + return nil +} + +// Update provides a new Config to the component. The type of newConfig will +// always match the struct type which the component registers. +// +// Update will be called concurrently with Run. The component must be able to +// gracefully handle updating its config while still running. +// +// An error may be returned if the provided config is invalid. +func (s *Queue) Update(args component.Arguments) error { + s.mut.Lock() + defer s.mut.Unlock() + + newArgs := args.(Arguments) + sync.OnceFunc(func() { + s.opts.OnStateChange(Exports{Receiver: s}) + }) + // If they are the same do nothing. + if reflect.DeepEqual(newArgs, s.args) { + return nil + } + s.args = newArgs + // TODO @mattdurham need to cycle through the endpoints figuring out what changed instead of this global stop and start.. + if len(s.endpoints) > 0 { + for _, ep := range s.endpoints { + ep.Stop() + } + s.endpoints = map[string]*endpoint{} + } + return s.createEndpoints() +} + +func (s *Queue) createEndpoints() error { + for _, ep := range s.args.Connections { + reg := prometheus.WrapRegistererWith(prometheus.Labels{"endpoint": ep.Name}, s.opts.Registerer) + stats := types.NewStats("alloy", "queue_series", reg) + stats.BackwardsCompatibility(reg) + meta := types.NewStats("alloy", "queue_metadata", reg) + client, err := network.New(network.ConnectionConfig{ + URL: ep.URL, + Username: ep.BasicAuth.Username, + Password: string(ep.BasicAuth.Password), + BatchCount: ep.BatchCount, + // Functionally this cannot go below 1s + FlushFrequency: ep.FlushDuration, + Timeout: ep.Timeout, + UserAgent: "alloy", + ExternalLabels: s.args.ExternalLabels, + Connections: uint64(ep.QueueCount), + }, s.log, stats.UpdateNetwork, meta.UpdateNetwork) + + if err != nil { + return err + } + // Serializer is set after + end := NewEndpoint(client, nil, stats, meta, s.args.TTL, s.opts.Logger) + // This wait group is to ensure we are started before we send on the mailbox. + fq, err := filequeue.NewQueue(filepath.Join(s.opts.DataPath, ep.Name, "wal"), func(ctx context.Context, dh types.DataHandle) { + _ = end.incoming.Send(ctx, dh) + }, s.opts.Logger) + if err != nil { + return err + } + serial, err := serialization.NewSerializer(types.SerializerConfig{ + MaxSignalsInBatch: 10_000, + FlushFrequency: 1 * time.Second, + }, fq, s.opts.Logger) + if err != nil { + return err + } + end.serializer = serial + s.endpoints[ep.Name] = end + // endpoint is responsible for starting all the children, this way they spin up + // together and are town down together. Or at least handled internally. + end.Start() + } + return nil +} + +// Appender returns a new appender for the storage. The implementation +// can choose whether or not to use the context, for deadlines or to check +// for errors. +func (c *Queue) Appender(ctx context.Context) storage.Appender { + c.mut.RLock() + defer c.mut.RUnlock() + + children := make([]storage.Appender, 0) + for _, ep := range c.endpoints { + children = append(children, serialization.NewAppender(ctx, c.args.TTL, ep.serializer, c.args.AppenderBatchSize, ep.stat.UpdateFileQueue, c.opts.Logger)) + } + return &fanout{children: children} +} diff --git a/internal/component/prometheus/remote/queue/config.go b/internal/component/prometheus/remote/queue/config.go new file mode 100644 index 0000000000..c969bad24a --- /dev/null +++ b/internal/component/prometheus/remote/queue/config.go @@ -0,0 +1 @@ +package queue diff --git a/internal/component/prometheus/remote/queue/e2e_bench_test.go b/internal/component/prometheus/remote/queue/e2e_bench_test.go new file mode 100644 index 0000000000..612f835657 --- /dev/null +++ b/internal/component/prometheus/remote/queue/e2e_bench_test.go @@ -0,0 +1,129 @@ +package queue + +import ( + "context" + "net/http" + "net/http/httptest" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/grafana/alloy/internal/component" + "github.com/grafana/alloy/internal/component/prometheus/remote/queue/types" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/prompb" + "github.com/prometheus/prometheus/storage" + "github.com/stretchr/testify/require" +) + +func BenchmarkE2E(b *testing.B) { + // Around 120k ops if you look at profile roughly 20k are actual implementation with the rest being benchmark + // setup. + type e2eTest struct { + name string + maker func(index int, app storage.Appender) + tester func(samples []prompb.TimeSeries) + } + tests := []e2eTest{ + { + name: "normal", + maker: func(index int, app storage.Appender) { + ts, v, lbls := makeSeries(index) + _, _ = app.Append(0, lbls, ts, v) + }, + tester: func(samples []prompb.TimeSeries) { + b.Helper() + for _, s := range samples { + require.True(b, len(s.Samples) == 1) + } + }, + }, + } + for _, test := range tests { + b.Run(test.name, func(t *testing.B) { + runBenchmark(t, test.maker, test.tester) + }) + } +} + +func runBenchmark(t *testing.B, add func(index int, appendable storage.Appender), _ func(samples []prompb.TimeSeries)) { + t.ReportAllocs() + l := log.NewNopLogger() + done := make(chan struct{}) + srv := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + })) + expCh := make(chan Exports, 1) + c, err := newComponentBenchmark(t, l, srv.URL, expCh) + require.NoError(t, err) + ctx := context.Background() + ctx, cancel := context.WithCancel(ctx) + go func() { + runErr := c.Run(ctx) + require.NoError(t, runErr) + }() + // Wait for export to spin up. + exp := <-expCh + + index := 0 + for i := 0; i < t.N; i++ { + go func() { + app := exp.Receiver.Appender(ctx) + for j := 0; j < items; j++ { + index++ + add(index, app) + } + require.NoError(t, app.Commit()) + }() + } + tm := time.NewTimer(10 * time.Second) + select { + case <-done: + case <-tm.C: + } + cancel() + +} + +func newComponentBenchmark(t *testing.B, l log.Logger, url string, exp chan Exports) (*Queue, error) { + return NewComponent(component.Options{ + ID: "test", + Logger: l, + DataPath: t.TempDir(), + OnStateChange: func(e component.Exports) { + exp <- e.(Exports) + }, + Registerer: fakeRegistry{}, + Tracer: nil, + }, Arguments{ + TTL: 2 * time.Hour, + MaxFlushSize: 100_000, + FlushDuration: 1 * time.Second, + Connections: []types.ConnectionConfig{{ + Name: "test", + URL: url, + Timeout: 10 * time.Second, + RetryBackoff: 1 * time.Second, + MaxRetryBackoffAttempts: 0, + BatchCount: 50, + FlushDuration: 1 * time.Second, + QueueCount: 1, + }}, + AppenderBatchSize: 1_000, + ExternalLabels: nil, + }) +} + +var _ (prometheus.Registerer) = (*fakeRegistry)(nil) + +type fakeRegistry struct{} + +func (f fakeRegistry) Register(collector prometheus.Collector) error { + return nil +} + +func (f fakeRegistry) MustRegister(collector ...prometheus.Collector) { +} + +func (f fakeRegistry) Unregister(collector prometheus.Collector) bool { + return true +} diff --git a/internal/component/prometheus/remote/queue/e2e_stats_test.go b/internal/component/prometheus/remote/queue/e2e_stats_test.go new file mode 100644 index 0000000000..1fe7601184 --- /dev/null +++ b/internal/component/prometheus/remote/queue/e2e_stats_test.go @@ -0,0 +1,568 @@ +package queue + +import ( + "context" + "net/http" + "net/http/httptest" + "testing" + "time" + + "github.com/grafana/alloy/internal/util" + "github.com/prometheus/client_golang/prometheus" + dto "github.com/prometheus/client_model/go" + "github.com/stretchr/testify/require" +) + +const remoteSamples = "prometheus_remote_storage_samples_total" +const remoteHistograms = "prometheus_remote_storage_histograms_total" +const remoteMetadata = "prometheus_remote_storage_metadata_total" + +const sentBytes = "prometheus_remote_storage_sent_bytes_total" +const sentMetadataBytes = "prometheus_remote_storage_metadata_bytes_total" + +const outTimestamp = "prometheus_remote_storage_queue_highest_sent_timestamp_seconds" +const inTimestamp = "prometheus_remote_storage_highest_timestamp_in_seconds" + +const failedSample = "prometheus_remote_storage_samples_failed_total" +const failedHistogram = "prometheus_remote_storage_histograms_failed_total" +const failedMetadata = "prometheus_remote_storage_metadata_failed_total" + +const retriedSamples = "prometheus_remote_storage_samples_retried_total" +const retriedHistogram = "prometheus_remote_storage_histograms_retried_total" +const retriedMetadata = "prometheus_remote_storage_metadata_retried_total" + +const prometheusDuration = "prometheus_remote_storage_queue_duration_seconds" + +const filequeueIncoming = "alloy_queue_series_filequeue_incoming" +const alloySent = "alloy_queue_series_network_sent" +const alloyFileQueueIncoming = "alloy_queue_series_filequeue_incoming_timestamp_seconds" +const alloyNetworkDuration = "alloy_queue_series_network_duration_seconds" +const alloyFailures = "alloy_queue_series_network_failed" +const alloyRetries = "alloy_queue_series_network_retried" +const alloy429 = "alloy_queue_series_network_retried_429" + +// TestMetrics is the large end to end testing for the queue based wal. +func TestMetrics(t *testing.T) { + // Check assumes you are checking for any value that is not 0. + // The test at the end will see if there are any values that were not 0. + tests := []statsTest{ + // Sample Tests + { + name: "sample success", + returnStatusCode: http.StatusOK, + dtype: Sample, + checks: []check{ + { + name: filequeueIncoming, + value: 10, + }, + { + name: remoteSamples, + value: 10, + }, + { + name: alloySent, + value: 10, + }, + { + name: prometheusDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyNetworkDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyFileQueueIncoming, + valueFunc: isReasonableTimeStamp, + }, + { + name: sentBytes, + valueFunc: greaterThenZero, + }, + { + name: outTimestamp, + valueFunc: isReasonableTimeStamp, + }, + { + name: inTimestamp, + valueFunc: isReasonableTimeStamp, + }, + }, + }, + { + name: "sample failure", + returnStatusCode: http.StatusBadRequest, + dtype: Sample, + checks: []check{ + { + name: alloyFailures, + value: 10, + }, + { + name: filequeueIncoming, + value: 10, + }, + { + name: failedSample, + value: 10, + }, + { + name: prometheusDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyNetworkDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyFileQueueIncoming, + valueFunc: isReasonableTimeStamp, + }, + { + name: inTimestamp, + valueFunc: isReasonableTimeStamp, + }, + }, + }, + { + name: "sample retry", + returnStatusCode: http.StatusTooManyRequests, + dtype: Sample, + checks: []check{ + { + name: filequeueIncoming, + value: 10, + }, + { + name: retriedSamples, + // This will be more than 10 since it retries in a loop. + valueFunc: greaterThenZero, + }, + { + name: alloyRetries, + // This will be more than 10 since it retries in a loop. + valueFunc: greaterThenZero, + }, + { + name: alloy429, + // This will be more than 10 since it retries in a loop. + valueFunc: greaterThenZero, + }, + { + name: prometheusDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyNetworkDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyFileQueueIncoming, + valueFunc: isReasonableTimeStamp, + }, + { + name: inTimestamp, + valueFunc: isReasonableTimeStamp, + }, + }, + }, + // histograms + { + name: "histogram success", + returnStatusCode: http.StatusOK, + dtype: Histogram, + checks: []check{ + { + name: filequeueIncoming, + value: 10, + }, + { + name: remoteHistograms, + value: 10, + }, + { + name: alloySent, + value: 10, + }, + { + name: prometheusDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyNetworkDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyFileQueueIncoming, + valueFunc: isReasonableTimeStamp, + }, + { + name: sentBytes, + valueFunc: greaterThenZero, + }, + { + name: outTimestamp, + valueFunc: isReasonableTimeStamp, + }, + { + name: inTimestamp, + valueFunc: isReasonableTimeStamp, + }, + }, + }, + { + name: "histogram failure", + returnStatusCode: http.StatusBadRequest, + dtype: Histogram, + checks: []check{ + { + name: alloyFailures, + value: 10, + }, + { + name: filequeueIncoming, + value: 10, + }, + { + name: failedHistogram, + value: 10, + }, + { + name: prometheusDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyNetworkDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyFileQueueIncoming, + valueFunc: isReasonableTimeStamp, + }, + { + name: inTimestamp, + valueFunc: isReasonableTimeStamp, + }, + }, + }, + { + name: "histogram retry", + returnStatusCode: http.StatusTooManyRequests, + dtype: Histogram, + checks: []check{ + { + name: filequeueIncoming, + value: 10, + }, + { + name: retriedHistogram, + // This will be more than 10 since it retries in a loop. + valueFunc: greaterThenZero, + }, + { + name: alloyRetries, + // This will be more than 10 since it retries in a loop. + valueFunc: greaterThenZero, + }, + { + name: alloy429, + // This will be more than 10 since it retries in a loop. + valueFunc: greaterThenZero, + }, + { + name: prometheusDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyNetworkDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyFileQueueIncoming, + valueFunc: isReasonableTimeStamp, + }, + { + name: inTimestamp, + valueFunc: isReasonableTimeStamp, + }, + }, + }, + // exemplar, note that once it hits the appender exemplars are treated the same as series. + { + name: "exemplar success", + returnStatusCode: http.StatusOK, + dtype: Exemplar, + checks: []check{ + { + name: filequeueIncoming, + value: 10, + }, + { + name: remoteSamples, + value: 10, + }, + { + name: alloySent, + value: 10, + }, + { + name: prometheusDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyNetworkDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyFileQueueIncoming, + valueFunc: isReasonableTimeStamp, + }, + { + name: sentBytes, + valueFunc: greaterThenZero, + }, + { + name: outTimestamp, + valueFunc: isReasonableTimeStamp, + }, + { + name: inTimestamp, + valueFunc: isReasonableTimeStamp, + }, + }, + }, + { + name: "exemplar failure", + returnStatusCode: http.StatusBadRequest, + dtype: Exemplar, + checks: []check{ + { + name: alloyFailures, + value: 10, + }, + { + name: filequeueIncoming, + value: 10, + }, + { + name: failedSample, + value: 10, + }, + { + name: prometheusDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyNetworkDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyFileQueueIncoming, + valueFunc: isReasonableTimeStamp, + }, + { + name: inTimestamp, + valueFunc: isReasonableTimeStamp, + }, + }, + }, + { + name: "exemplar retry", + returnStatusCode: http.StatusTooManyRequests, + dtype: Exemplar, + checks: []check{ + { + name: filequeueIncoming, + value: 10, + }, + { + name: retriedSamples, + // This will be more than 10 since it retries in a loop. + valueFunc: greaterThenZero, + }, + { + name: alloyRetries, + // This will be more than 10 since it retries in a loop. + valueFunc: greaterThenZero, + }, + { + name: alloy429, + // This will be more than 10 since it retries in a loop. + valueFunc: greaterThenZero, + }, + { + name: prometheusDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyNetworkDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyFileQueueIncoming, + valueFunc: isReasonableTimeStamp, + }, + { + name: inTimestamp, + valueFunc: isReasonableTimeStamp, + }, + }, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + runE2eStats(t, test) + }) + } + +} + +func greaterThenZero(v float64) bool { + return v > 0 +} + +func isReasonableTimeStamp(v float64) bool { + if v < 0 { + return false + } + unixTime := time.Unix(int64(v), 0) + + return time.Since(unixTime) < 10*time.Second +} + +type dataType int + +const ( + Sample dataType = iota + Histogram + Exemplar + Metadata +) + +type check struct { + name string + value float64 + valueFunc func(v float64) bool +} +type statsTest struct { + name string + returnStatusCode int + // Only check for non zero values, once all checks are ran it will automatically ensure all remaining metrics are 0. + checks []check + dtype dataType +} + +func runE2eStats(t *testing.T, test statsTest) { + l := util.TestAlloyLogger(t) + srv := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + w.WriteHeader(test.returnStatusCode) + })) + expCh := make(chan Exports, 1) + + reg := prometheus.NewRegistry() + c, err := newComponent(t, l, srv.URL, expCh, reg) + require.NoError(t, err) + ctx := context.Background() + ctx, cancel := context.WithCancel(ctx) + go func() { + runErr := c.Run(ctx) + require.NoError(t, runErr) + }() + // Wait for export to spin up. + exp := <-expCh + + index := 0 + + go func() { + app := exp.Receiver.Appender(ctx) + for j := 0; j < 10; j++ { + index++ + switch test.dtype { + case Sample: + ts, v, lbls := makeSeries(index) + _, errApp := app.Append(0, lbls, ts, v) + require.NoError(t, errApp) + case Histogram: + ts, lbls, h := makeHistogram(index) + _, errApp := app.AppendHistogram(0, lbls, ts, h, nil) + require.NoError(t, errApp) + case Exemplar: + ex := makeExemplar(index) + _, errApp := app.AppendExemplar(0, nil, ex) + require.NoError(t, errApp) + default: + require.True(t, false) + } + } + require.NoError(t, app.Commit()) + }() + tm := time.NewTimer(8 * time.Second) + <-tm.C + cancel() + + require.Eventually(t, func() bool { + dtos, gatherErr := reg.Gather() + require.NoError(t, gatherErr) + for _, d := range dtos { + if getValue(d) > 0 { + return true + } + } + return false + }, 10*time.Second, 1*time.Second) + metrics := make(map[string]float64) + dtos, err := reg.Gather() + require.NoError(t, err) + for _, d := range dtos { + metrics[*d.Name] = getValue(d) + } + + // Check for the metrics that matter. + for _, valChk := range test.checks { + if valChk.valueFunc != nil { + metrics = checkValueCondition(t, valChk.name, valChk.valueFunc, metrics) + } else { + metrics = checkValue(t, valChk.name, valChk.value, metrics) + } + } + // all other metrics should be zero. + for k, v := range metrics { + require.Zerof(t, v, "%s should be zero", k) + } +} + +func getValue(d *dto.MetricFamily) float64 { + switch *d.Type { + case dto.MetricType_COUNTER: + return d.Metric[0].Counter.GetValue() + case dto.MetricType_GAUGE: + return d.Metric[0].Gauge.GetValue() + case dto.MetricType_SUMMARY: + return d.Metric[0].Summary.GetSampleSum() + case dto.MetricType_UNTYPED: + return d.Metric[0].Untyped.GetValue() + case dto.MetricType_HISTOGRAM: + return d.Metric[0].Histogram.GetSampleSum() + case dto.MetricType_GAUGE_HISTOGRAM: + return d.Metric[0].Histogram.GetSampleSum() + default: + panic("unknown type " + d.Type.String()) + } +} + +func checkValue(t *testing.T, name string, value float64, metrics map[string]float64) map[string]float64 { + v, ok := metrics[name] + require.Truef(t, ok, "invalid metric name %s", name) + require.Equalf(t, value, v, "%s should be %f", name, value) + delete(metrics, name) + return metrics +} + +func checkValueCondition(t *testing.T, name string, chk func(float64) bool, metrics map[string]float64) map[string]float64 { + v, ok := metrics[name] + require.True(t, ok) + require.True(t, chk(v)) + delete(metrics, name) + return metrics +} diff --git a/internal/component/prometheus/remote/queue/e2e_test.go b/internal/component/prometheus/remote/queue/e2e_test.go new file mode 100644 index 0000000000..63b4820312 --- /dev/null +++ b/internal/component/prometheus/remote/queue/e2e_test.go @@ -0,0 +1,365 @@ +package queue + +import ( + "context" + "fmt" + "io" + "net/http" + "net/http/httptest" + "reflect" + "strings" + "sync" + "testing" + "time" + + "github.com/golang/snappy" + "github.com/grafana/alloy/internal/component" + "github.com/grafana/alloy/internal/component/prometheus/remote/queue/types" + "github.com/grafana/alloy/internal/runtime/logging" + "github.com/grafana/alloy/internal/util" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/metadata" + "github.com/prometheus/prometheus/prompb" + "github.com/prometheus/prometheus/storage" + "github.com/stretchr/testify/require" + "go.uber.org/atomic" +) + +func TestE2E(t *testing.T) { + type e2eTest struct { + name string + maker func(index int, app storage.Appender) (float64, labels.Labels) + tester func(samples []prompb.TimeSeries) + testMeta func(samples []prompb.MetricMetadata) + } + tests := []e2eTest{ + { + name: "normal", + maker: func(index int, app storage.Appender) (float64, labels.Labels) { + ts, v, lbls := makeSeries(index) + _, errApp := app.Append(0, lbls, ts, v) + require.NoError(t, errApp) + return v, lbls + }, + tester: func(samples []prompb.TimeSeries) { + t.Helper() + for _, s := range samples { + require.True(t, len(s.Samples) == 1) + require.True(t, s.Samples[0].Timestamp > 0) + require.True(t, s.Samples[0].Value > 0) + require.True(t, len(s.Labels) == 1) + require.Truef(t, s.Labels[0].Name == fmt.Sprintf("name_%d", int(s.Samples[0].Value)), "%d name %s", int(s.Samples[0].Value), s.Labels[0].Name) + require.True(t, s.Labels[0].Value == fmt.Sprintf("value_%d", int(s.Samples[0].Value))) + } + }, + }, + { + name: "metadata", + maker: func(index int, app storage.Appender) (float64, labels.Labels) { + meta, lbls := makeMetadata(index) + _, errApp := app.UpdateMetadata(0, lbls, meta) + require.NoError(t, errApp) + return 0, lbls + }, + testMeta: func(samples []prompb.MetricMetadata) { + for _, s := range samples { + require.True(t, s.GetUnit() == "seconds") + require.True(t, s.Help == "metadata help") + require.True(t, s.Unit == "seconds") + require.True(t, s.Type == prompb.MetricMetadata_COUNTER) + require.True(t, strings.HasPrefix(s.MetricFamilyName, "name_")) + + } + }, + }, + + { + name: "histogram", + maker: func(index int, app storage.Appender) (float64, labels.Labels) { + ts, lbls, h := makeHistogram(index) + _, errApp := app.AppendHistogram(0, lbls, ts, h, nil) + require.NoError(t, errApp) + return h.Sum, lbls + }, + tester: func(samples []prompb.TimeSeries) { + t.Helper() + for _, s := range samples { + require.True(t, len(s.Samples) == 1) + require.True(t, s.Samples[0].Timestamp > 0) + require.True(t, s.Samples[0].Value == 0) + require.True(t, len(s.Labels) == 1) + histSame(t, hist(int(s.Histograms[0].Sum)), s.Histograms[0]) + } + }, + }, + { + name: "float histogram", + maker: func(index int, app storage.Appender) (float64, labels.Labels) { + ts, lbls, h := makeFloatHistogram(index) + _, errApp := app.AppendHistogram(0, lbls, ts, nil, h) + require.NoError(t, errApp) + return h.Sum, lbls + }, + tester: func(samples []prompb.TimeSeries) { + t.Helper() + for _, s := range samples { + require.True(t, len(s.Samples) == 1) + require.True(t, s.Samples[0].Timestamp > 0) + require.True(t, s.Samples[0].Value == 0) + require.True(t, len(s.Labels) == 1) + histFloatSame(t, histFloat(int(s.Histograms[0].Sum)), s.Histograms[0]) + } + }, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + runTest(t, test.maker, test.tester, test.testMeta) + }) + } +} + +const iterations = 100 +const items = 10_000 + +func runTest(t *testing.T, add func(index int, appendable storage.Appender) (float64, labels.Labels), test func(samples []prompb.TimeSeries), metaTest func(meta []prompb.MetricMetadata)) { + l := util.TestAlloyLogger(t) + done := make(chan struct{}) + var series atomic.Int32 + var meta atomic.Int32 + samples := make([]prompb.TimeSeries, 0) + metaSamples := make([]prompb.MetricMetadata, 0) + srv := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + newSamples, newMetadata := handlePost(t, w, r) + series.Add(int32(len(newSamples))) + meta.Add(int32(len(newMetadata))) + samples = append(samples, newSamples...) + metaSamples = append(metaSamples, newMetadata...) + if series.Load() == iterations*items { + done <- struct{}{} + } + if meta.Load() == iterations*items { + done <- struct{}{} + } + })) + expCh := make(chan Exports, 1) + c, err := newComponent(t, l, srv.URL, expCh, prometheus.NewRegistry()) + require.NoError(t, err) + ctx := context.Background() + ctx, cancel := context.WithCancel(ctx) + go func() { + runErr := c.Run(ctx) + require.NoError(t, runErr) + }() + // Wait for export to spin up. + exp := <-expCh + + index := 0 + results := make(map[float64]labels.Labels) + mut := sync.Mutex{} + + for i := 0; i < iterations; i++ { + go func() { + app := exp.Receiver.Appender(ctx) + for j := 0; j < items; j++ { + index++ + v, lbl := add(index, app) + mut.Lock() + results[v] = lbl + mut.Unlock() + } + require.NoError(t, app.Commit()) + }() + + } + // This is a weird use case to handle eventually. + tm := time.NewTimer(15 * time.Second) + select { + case <-done: + case <-tm.C: + } + cancel() + for _, s := range samples { + if len(s.Histograms) == 1 { + lbls, ok := results[s.Histograms[0].Sum] + require.True(t, ok) + for i, sLbl := range s.Labels { + require.True(t, lbls[i].Name == sLbl.Name) + require.True(t, lbls[i].Value == sLbl.Value) + } + } else { + lbls, ok := results[s.Samples[0].Value] + require.True(t, ok) + for i, sLbl := range s.Labels { + require.True(t, lbls[i].Name == sLbl.Name) + require.True(t, lbls[i].Value == sLbl.Value) + } + } + + } + if test != nil { + test(samples) + } else { + metaTest(metaSamples) + } + require.True(t, types.OutStandingTimeSeriesBinary.Load() == 0) +} + +func handlePost(t *testing.T, _ http.ResponseWriter, r *http.Request) ([]prompb.TimeSeries, []prompb.MetricMetadata) { + defer r.Body.Close() + data, err := io.ReadAll(r.Body) + require.NoError(t, err) + + data, err = snappy.Decode(nil, data) + require.NoError(t, err) + + var req prompb.WriteRequest + err = req.Unmarshal(data) + require.NoError(t, err) + return req.GetTimeseries(), req.Metadata + +} + +func makeSeries(index int) (int64, float64, labels.Labels) { + return time.Now().UTC().Unix(), float64(index), labels.FromStrings(fmt.Sprintf("name_%d", index), fmt.Sprintf("value_%d", index)) +} + +func makeMetadata(index int) (metadata.Metadata, labels.Labels) { + return metadata.Metadata{ + Type: "counter", + Unit: "seconds", + Help: "metadata help", + }, labels.FromStrings("__name__", fmt.Sprintf("name_%d", index)) +} + +func makeHistogram(index int) (int64, labels.Labels, *histogram.Histogram) { + return time.Now().UTC().Unix(), labels.FromStrings(fmt.Sprintf("name_%d", index), fmt.Sprintf("value_%d", index)), hist(index) +} +func makeExemplar(index int) exemplar.Exemplar { + return exemplar.Exemplar{ + Labels: labels.FromStrings(fmt.Sprintf("name_%d", index), fmt.Sprintf("value_%d", index)), + Ts: time.Now().Unix(), + HasTs: true, + Value: float64(index), + } +} + +func hist(i int) *histogram.Histogram { + return &histogram.Histogram{ + CounterResetHint: 1, + Schema: 2, + ZeroThreshold: 3, + ZeroCount: 4, + Count: 5, + Sum: float64(i), + PositiveSpans: []histogram.Span{ + { + Offset: 1, + Length: 2, + }, + }, + NegativeSpans: []histogram.Span{ + { + Offset: 3, + Length: 4, + }, + }, + PositiveBuckets: []int64{1, 2, 3}, + NegativeBuckets: []int64{1, 2, 3}, + } +} + +func histSame(t *testing.T, h *histogram.Histogram, pb prompb.Histogram) { + require.True(t, h.Sum == pb.Sum) + require.True(t, h.ZeroCount == pb.ZeroCount.(*prompb.Histogram_ZeroCountInt).ZeroCountInt) + require.True(t, h.Schema == pb.Schema) + require.True(t, h.Count == pb.Count.(*prompb.Histogram_CountInt).CountInt) + require.True(t, h.ZeroThreshold == pb.ZeroThreshold) + require.True(t, int32(h.CounterResetHint) == int32(pb.ResetHint)) + require.True(t, reflect.DeepEqual(h.PositiveBuckets, pb.PositiveDeltas)) + require.True(t, reflect.DeepEqual(h.NegativeBuckets, pb.NegativeDeltas)) + histSpanSame(t, h.PositiveSpans, pb.PositiveSpans) + histSpanSame(t, h.NegativeSpans, pb.NegativeSpans) +} + +func histSpanSame(t *testing.T, h []histogram.Span, pb []prompb.BucketSpan) { + require.True(t, len(h) == len(pb)) + for i := range h { + require.True(t, h[i].Length == pb[i].Length) + require.True(t, h[i].Offset == pb[i].Offset) + } +} + +func makeFloatHistogram(index int) (int64, labels.Labels, *histogram.FloatHistogram) { + return time.Now().UTC().Unix(), labels.FromStrings(fmt.Sprintf("name_%d", index), fmt.Sprintf("value_%d", index)), histFloat(index) +} + +func histFloat(i int) *histogram.FloatHistogram { + return &histogram.FloatHistogram{ + CounterResetHint: 1, + Schema: 2, + ZeroThreshold: 3, + ZeroCount: 4, + Count: 5, + Sum: float64(i), + PositiveSpans: []histogram.Span{ + { + Offset: 1, + Length: 2, + }, + }, + NegativeSpans: []histogram.Span{ + { + Offset: 3, + Length: 4, + }, + }, + PositiveBuckets: []float64{1.1, 2.2, 3.3}, + NegativeBuckets: []float64{1.2, 2.3, 3.4}, + } +} + +func histFloatSame(t *testing.T, h *histogram.FloatHistogram, pb prompb.Histogram) { + require.True(t, h.Sum == pb.Sum) + require.True(t, h.ZeroCount == pb.ZeroCount.(*prompb.Histogram_ZeroCountFloat).ZeroCountFloat) + require.True(t, h.Schema == pb.Schema) + require.True(t, h.Count == pb.Count.(*prompb.Histogram_CountFloat).CountFloat) + require.True(t, h.ZeroThreshold == pb.ZeroThreshold) + require.True(t, int32(h.CounterResetHint) == int32(pb.ResetHint)) + require.True(t, reflect.DeepEqual(h.PositiveBuckets, pb.PositiveCounts)) + require.True(t, reflect.DeepEqual(h.NegativeBuckets, pb.NegativeCounts)) + histSpanSame(t, h.PositiveSpans, pb.PositiveSpans) + histSpanSame(t, h.NegativeSpans, pb.NegativeSpans) +} + +func newComponent(t *testing.T, l *logging.Logger, url string, exp chan Exports, reg prometheus.Registerer) (*Queue, error) { + return NewComponent(component.Options{ + ID: "test", + Logger: l, + DataPath: t.TempDir(), + OnStateChange: func(e component.Exports) { + exp <- e.(Exports) + }, + Registerer: reg, + Tracer: nil, + }, Arguments{ + TTL: 2 * time.Hour, + MaxFlushSize: 10_000, + FlushDuration: 1 * time.Second, + Connections: []types.ConnectionConfig{{ + Name: "test", + URL: url, + Timeout: 20 * time.Second, + RetryBackoff: 5 * time.Second, + MaxRetryBackoffAttempts: 1, + BatchCount: 50, + FlushDuration: 1 * time.Second, + QueueCount: 1, + }}, + AppenderBatchSize: 1_000, + ExternalLabels: nil, + }) +} diff --git a/internal/component/prometheus/remote/queue/endpoint.go b/internal/component/prometheus/remote/queue/endpoint.go new file mode 100644 index 0000000000..5cbca5e5e4 --- /dev/null +++ b/internal/component/prometheus/remote/queue/endpoint.go @@ -0,0 +1,136 @@ +package queue + +import ( + "context" + "strconv" + "time" + + snappy "github.com/eapache/go-xerial-snappy" + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/alloy/internal/component/prometheus/remote/queue/types" + "github.com/vladopajic/go-actor/actor" +) + +var _ actor.Worker = (*endpoint)(nil) + +// endpoint handles communication between the serializer, filequeue and network. +type endpoint struct { + network types.NetworkClient + serializer types.Serializer + stat *types.PrometheusStats + metaStats *types.PrometheusStats + log log.Logger + ttl time.Duration + incoming actor.Mailbox[types.DataHandle] + buf []byte + self actor.Actor +} + +func NewEndpoint(client types.NetworkClient, serializer types.Serializer, stats, metatStats *types.PrometheusStats, ttl time.Duration, logger log.Logger) *endpoint { + return &endpoint{ + network: client, + serializer: serializer, + stat: stats, + metaStats: metatStats, + log: logger, + ttl: ttl, + incoming: actor.NewMailbox[types.DataHandle](actor.OptCapacity(1)), + buf: make([]byte, 0, 1024), + } +} + +func (ep *endpoint) Start() { + ep.self = actor.Combine(actor.New(ep), ep.incoming).Build() + ep.self.Start() + ep.serializer.Start() + ep.network.Start() +} + +func (ep *endpoint) Stop() { + ep.serializer.Stop() + ep.network.Stop() + ep.network.Stop() + ep.self.Stop() +} + +func (ep *endpoint) DoWork(ctx actor.Context) actor.WorkerStatus { + select { + case <-ctx.Done(): + return actor.WorkerEnd + case file, ok := <-ep.incoming.ReceiveC(): + if !ok { + return actor.WorkerEnd + } + meta, buf, err := file.Get() + if err != nil { + level.Error(ep.log).Log("msg", "unable to get file contents", "name", file.Name, "err", err) + return actor.WorkerContinue + } + ep.deserializeAndSend(ctx, meta, buf) + return actor.WorkerContinue + } +} + +func (ep *endpoint) deserializeAndSend(ctx context.Context, meta map[string]string, buf []byte) { + var err error + ep.buf, err = snappy.Decode(buf) + if err != nil { + level.Debug(ep.log).Log("msg", "error snappy decoding", "err", err) + return + } + // The version of each file is in the metadata. Right now there is only one version + // supported but in the future the ability to support more. Along with different + // compression. + version, ok := meta["version"] + if !ok { + level.Error(ep.log).Log("msg", "version not found for deserialization") + return + } + if version != "alloy.metrics.queue.v1" { + level.Error(ep.log).Log("msg", "invalid version found for deserialization", "version", version) + return + } + // Grab the amounts of each type and we can go ahead and alloc the space. + seriesCount, _ := strconv.Atoi(meta["series_count"]) + metaCount, _ := strconv.Atoi(meta["meta_count"]) + stringsCount, _ := strconv.Atoi(meta["strings_count"]) + sg := &types.SeriesGroup{ + Series: make([]*types.TimeSeriesBinary, seriesCount), + Metadata: make([]*types.TimeSeriesBinary, metaCount), + Strings: make([]string, stringsCount), + } + for i := 0; i < seriesCount; i++ { + sg.Series[i] = types.GetTimeSeriesBinary() + } + for i := 0; i < metaCount; i++ { + sg.Metadata[i] = types.GetTimeSeriesBinary() + } + sg, ep.buf, err = types.DeserializeToSeriesGroup(sg, ep.buf) + + if err != nil { + level.Debug(ep.log).Log("msg", "error deserializing", "err", err) + return + } + + for _, series := range sg.Series { + // One last chance to check the TTL. Writing to the filequeue will check it but + // in a situation where the network is down and writing backs up we dont want to send + // data that will get rejected. + seriesAge := time.Since(time.Unix(series.TS, 0)) + if seriesAge > ep.ttl { + continue + } + sendErr := ep.network.SendSeries(ctx, series) + if sendErr != nil { + level.Error(ep.log).Log("msg", "error sending to write client", "err", sendErr) + } + } + + for _, md := range sg.Metadata { + sendErr := ep.network.SendMetadata(ctx, md) + if sendErr != nil { + level.Error(ep.log).Log("msg", "error sending metadata to write client", "err", sendErr) + } + } +} diff --git a/internal/component/prometheus/remote/queue/fanout.go b/internal/component/prometheus/remote/queue/fanout.go new file mode 100644 index 0000000000..09a7fb97ed --- /dev/null +++ b/internal/component/prometheus/remote/queue/fanout.go @@ -0,0 +1,85 @@ +package queue + +import ( + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/metadata" + "github.com/prometheus/prometheus/storage" +) + +var _ storage.Appender = (*fanout)(nil) + +type fanout struct { + children []storage.Appender +} + +func (f fanout) Append(ref storage.SeriesRef, l labels.Labels, t int64, v float64) (storage.SeriesRef, error) { + for _, child := range f.children { + _, err := child.Append(ref, l, t, v) + if err != nil { + return ref, err + } + } + return ref, nil +} + +func (f fanout) Commit() error { + for _, child := range f.children { + err := child.Commit() + if err != nil { + return err + } + } + return nil +} + +func (f fanout) Rollback() error { + for _, child := range f.children { + err := child.Rollback() + if err != nil { + return err + } + } + return nil +} + +func (f fanout) AppendExemplar(ref storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { + for _, child := range f.children { + _, err := child.AppendExemplar(ref, l, e) + if err != nil { + return ref, err + } + } + return ref, nil +} + +func (f fanout) AppendHistogram(ref storage.SeriesRef, l labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { + for _, child := range f.children { + _, err := child.AppendHistogram(ref, l, t, h, fh) + if err != nil { + return ref, err + } + } + return ref, nil +} + +func (f fanout) UpdateMetadata(ref storage.SeriesRef, l labels.Labels, m metadata.Metadata) (storage.SeriesRef, error) { + for _, child := range f.children { + _, err := child.UpdateMetadata(ref, l, m) + if err != nil { + return ref, err + } + } + return ref, nil +} + +func (f fanout) AppendCTZeroSample(ref storage.SeriesRef, l labels.Labels, t, ct int64) (storage.SeriesRef, error) { + for _, child := range f.children { + _, err := child.AppendCTZeroSample(ref, l, t, ct) + if err != nil { + return ref, err + } + } + return ref, nil +} diff --git a/internal/component/prometheus/remote/queue/types.go b/internal/component/prometheus/remote/queue/types.go new file mode 100644 index 0000000000..57f46ca3ac --- /dev/null +++ b/internal/component/prometheus/remote/queue/types.go @@ -0,0 +1,112 @@ +package queue + +import ( + "fmt" + "github.com/grafana/alloy/internal/component/prometheus/remote/queue/types" + "time" + + "github.com/grafana/alloy/syntax/alloytypes" + "github.com/prometheus/prometheus/storage" +) + +func defaultArgs() Arguments { + return Arguments{ + TTL: 2 * time.Hour, + MaxFlushSize: 10_000, + FlushDuration: 5 * time.Second, + AppenderBatchSize: 1_000, + } +} + +type Arguments struct { + // TTL is how old a series can be. + TTL time.Duration `alloy:"ttl,attr,optional"` + // The batch size to persist to the file queue. + MaxFlushSize int `alloy:"max_flush_size,attr,optional"` + // How often to flush to the file queue if BatchSizeBytes isn't met. + FlushDuration time.Duration `alloy:"flush_duration,attr,optional"` + Connections []types.ConnectionConfig `alloy:"endpoint,block"` + // AppenderBatchSize determines how often to flush the appender batch size. + AppenderBatchSize int `alloy:"appender_batch_size,attr,optional"` + ExternalLabels map[string]string `alloy:"external_labels,attr,optional"` +} + +type Exports struct { + Receiver storage.Appendable `alloy:"receiver,attr"` +} + +// SetToDefault sets the default +func (rc *Arguments) SetToDefault() { + *rc = defaultArgs() +} +func defaultCC() ConnectionConfig { + return ConnectionConfig{ + Timeout: 15 * time.Second, + RetryBackoff: 1 * time.Second, + MaxRetryBackoffAttempts: 0, + BatchCount: 1_000, + FlushFrequency: 1 * time.Second, + Connections: 4, + } +} +func (cc *ConnectionConfig) SetToDefault() { + *cc = defaultCC() +} + +func (r *Arguments) Validate() error { + if r.AppenderBatchSize == 0 { + return fmt.Errorf("appender_batch_size must be greater than zero") + } + for _, conn := range r.Connections { + if conn.BatchCount <= 0 { + return fmt.Errorf("batch_count must be greater than 0") + } + } + return nil +} + +type ConnectionConfig struct { + Name string `alloy:",label"` + URL string `alloy:"url,attr"` + BasicAuth *BasicAuth `alloy:"basic_auth,block,optional"` + Timeout time.Duration `alloy:"write_timeout,attr,optional"` + // How long to wait between retries. + RetryBackoff time.Duration `alloy:"retry_backoff,attr,optional"` + // Maximum number of retries. + MaxRetryBackoffAttempts uint `alloy:"max_retry_backoff,attr,optional"` + // How many series to write at a time. + BatchCount int `alloy:"batch_count,attr,optional"` + // How long to wait before sending regardless of batch count. + FlushFrequency time.Duration `alloy:"flush_duration,attr,optional"` + // How many concurrent queues to have. + Connections uint `alloy:"queue_count,attr,optional"` + + ExternalLabels map[string]string `alloy:"external_labels,attr,optional"` +} + +func (cc ConnectionConfig) ToTypesCC() types.ConnectionConfig { + tcc := types.ConnectionConfig{ + URL: cc.URL, + UserAgent: "alloy", + Timeout: cc.Timeout, + RetryBackoff: cc.RetryBackoff, + MaxRetryBackoffAttempts: cc.MaxRetryBackoffAttempts, + BatchCount: cc.BatchCount, + FlushFrequency: cc.FlushFrequency, + ExternalLabels: cc.ExternalLabels, + Connections: cc.Connections, + } + if cc.BasicAuth != nil { + tcc.BasicAuth = &types.BasicAuth{ + Username: cc.BasicAuth.Username, + Password: string(cc.BasicAuth.Password), + } + } + return tcc + +} + +type BasicAuth struct { + Username string `alloy:"username,attr,optional"` + Password alloytypes.Secret `alloy:"password,attr,optional"` +} diff --git a/internal/component/prometheus/remote/queue/types/network.go b/internal/component/prometheus/remote/queue/types/network.go index 024f8a3122..e83a816f87 100644 --- a/internal/component/prometheus/remote/queue/types/network.go +++ b/internal/component/prometheus/remote/queue/types/network.go @@ -15,16 +15,20 @@ type NetworkClient interface { } type ConnectionConfig struct { URL string - Username string - Password string + BasicAuth *BasicAuth UserAgent string Timeout time.Duration RetryBackoff time.Duration - MaxRetryBackoffAttempts time.Duration + MaxRetryBackoffAttempts uint BatchCount int FlushFrequency time.Duration ExternalLabels map[string]string - Connections uint64 + Connections uint +} + +type BasicAuth struct { + Username string + Password string } func (cc ConnectionConfig) Equals(bb ConnectionConfig) bool { diff --git a/internal/component/prometheus/remote/queue/types/storage_test.go b/internal/component/prometheus/remote/queue/types/storage_test.go new file mode 100644 index 0000000000..bfd0d8b67a --- /dev/null +++ b/internal/component/prometheus/remote/queue/types/storage_test.go @@ -0,0 +1,25 @@ +package types + +import ( + "testing" + + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/require" +) + +func TestStorage(t *testing.T) { + ts := GetTimeSeriesBinary() + ts.Labels = labels.FromStrings("one", "two") + ts.LabelsValues = make([]uint32, 1) + ts.LabelsNames = make([]uint32, 1) + ts.LabelsValues[0] = 1 + ts.LabelsNames[0] = 2 + + PutTimeSeriesBinary(ts) + ts = GetTimeSeriesBinary() + require.Len(t, ts.Labels, 0) + require.True(t, cap(ts.LabelsValues) == 1) + require.True(t, cap(ts.LabelsNames) == 1) + require.Len(t, ts.LabelsValues, 0) + require.Len(t, ts.LabelsNames, 0) +} From 46e1764db7d525112277aee59d14199aec900746 Mon Sep 17 00:00:00 2001 From: Matt Durham Date: Thu, 3 Oct 2024 14:33:51 -0400 Subject: [PATCH 09/44] Initial push. --- .../prometheus/remote/queue/component.go | 21 +++++++++------- .../prometheus/remote/queue/e2e_bench_test.go | 24 +++++++++---------- .../prometheus/remote/queue/e2e_test.go | 9 ++++--- .../prometheus/remote/queue/endpoint.go | 6 ++--- .../prometheus/remote/queue/network/loop.go | 4 +++- .../remote/queue/network/manager.go | 7 +++--- .../prometheus/remote/queue/types.go | 6 ++--- 7 files changed, 40 insertions(+), 37 deletions(-) diff --git a/internal/component/prometheus/remote/queue/component.go b/internal/component/prometheus/remote/queue/component.go index 1e0c005c3a..a4185b4f3e 100644 --- a/internal/component/prometheus/remote/queue/component.go +++ b/internal/component/prometheus/remote/queue/component.go @@ -109,18 +109,23 @@ func (s *Queue) createEndpoints() error { stats := types.NewStats("alloy", "queue_series", reg) stats.BackwardsCompatibility(reg) meta := types.NewStats("alloy", "queue_metadata", reg) - client, err := network.New(network.ConnectionConfig{ + cfg := types.ConnectionConfig{ URL: ep.URL, - Username: ep.BasicAuth.Username, - Password: string(ep.BasicAuth.Password), BatchCount: ep.BatchCount, // Functionally this cannot go below 1s - FlushFrequency: ep.FlushDuration, + FlushFrequency: ep.FlushFrequency, Timeout: ep.Timeout, UserAgent: "alloy", ExternalLabels: s.args.ExternalLabels, - Connections: uint64(ep.QueueCount), - }, s.log, stats.UpdateNetwork, meta.UpdateNetwork) + Connections: ep.Connections, + } + if ep.BasicAuth != nil { + cfg.BasicAuth = &types.BasicAuth{ + Username: ep.BasicAuth.Username, + Password: string(ep.BasicAuth.Password), + } + } + client, err := network.New(cfg, s.log, stats.UpdateNetwork, meta.UpdateNetwork) if err != nil { return err @@ -137,7 +142,7 @@ func (s *Queue) createEndpoints() error { serial, err := serialization.NewSerializer(types.SerializerConfig{ MaxSignalsInBatch: 10_000, FlushFrequency: 1 * time.Second, - }, fq, s.opts.Logger) + }, fq, stats.UpdateFileQueue, s.opts.Logger) if err != nil { return err } @@ -159,7 +164,7 @@ func (c *Queue) Appender(ctx context.Context) storage.Appender { children := make([]storage.Appender, 0) for _, ep := range c.endpoints { - children = append(children, serialization.NewAppender(ctx, c.args.TTL, ep.serializer, c.args.AppenderBatchSize, ep.stat.UpdateFileQueue, c.opts.Logger)) + children = append(children, serialization.NewAppender(ctx, c.args.TTL, ep.serializer, c.opts.Logger)) } return &fanout{children: children} } diff --git a/internal/component/prometheus/remote/queue/e2e_bench_test.go b/internal/component/prometheus/remote/queue/e2e_bench_test.go index 612f835657..94212a25ce 100644 --- a/internal/component/prometheus/remote/queue/e2e_bench_test.go +++ b/internal/component/prometheus/remote/queue/e2e_bench_test.go @@ -9,7 +9,6 @@ import ( "github.com/go-kit/log" "github.com/grafana/alloy/internal/component" - "github.com/grafana/alloy/internal/component/prometheus/remote/queue/types" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/prompb" "github.com/prometheus/prometheus/storage" @@ -26,6 +25,7 @@ func BenchmarkE2E(b *testing.B) { } tests := []e2eTest{ { + // This should be ~1200 allocs an op name: "normal", maker: func(index int, app storage.Appender) { ts, v, lbls := makeSeries(index) @@ -65,16 +65,14 @@ func runBenchmark(t *testing.B, add func(index int, appendable storage.Appender) exp := <-expCh index := 0 + app := exp.Receiver.Appender(ctx) + for i := 0; i < t.N; i++ { - go func() { - app := exp.Receiver.Appender(ctx) - for j := 0; j < items; j++ { - index++ - add(index, app) - } - require.NoError(t, app.Commit()) - }() + index++ + add(index, app) } + require.NoError(t, app.Commit()) + tm := time.NewTimer(10 * time.Second) select { case <-done: @@ -98,22 +96,22 @@ func newComponentBenchmark(t *testing.B, l log.Logger, url string, exp chan Expo TTL: 2 * time.Hour, MaxFlushSize: 100_000, FlushDuration: 1 * time.Second, - Connections: []types.ConnectionConfig{{ + Connections: []ConnectionConfig{{ Name: "test", URL: url, Timeout: 10 * time.Second, RetryBackoff: 1 * time.Second, MaxRetryBackoffAttempts: 0, BatchCount: 50, - FlushDuration: 1 * time.Second, - QueueCount: 1, + FlushFrequency: 1 * time.Second, + Connections: 1, }}, AppenderBatchSize: 1_000, ExternalLabels: nil, }) } -var _ (prometheus.Registerer) = (*fakeRegistry)(nil) +var _ prometheus.Registerer = (*fakeRegistry)(nil) type fakeRegistry struct{} diff --git a/internal/component/prometheus/remote/queue/e2e_test.go b/internal/component/prometheus/remote/queue/e2e_test.go index 63b4820312..b4a581f8f9 100644 --- a/internal/component/prometheus/remote/queue/e2e_test.go +++ b/internal/component/prometheus/remote/queue/e2e_test.go @@ -71,7 +71,6 @@ func TestE2E(t *testing.T) { require.True(t, s.Unit == "seconds") require.True(t, s.Type == prompb.MetricMetadata_COUNTER) require.True(t, strings.HasPrefix(s.MetricFamilyName, "name_")) - } }, }, @@ -205,7 +204,7 @@ func runTest(t *testing.T, add func(index int, appendable storage.Appender) (flo } else { metaTest(metaSamples) } - require.True(t, types.OutStandingTimeSeriesBinary.Load() == 0) + require.Truef(t, types.OutStandingTimeSeriesBinary.Load() == 0, "there are %d time series not collected", types.OutStandingTimeSeriesBinary.Load()) } func handlePost(t *testing.T, _ http.ResponseWriter, r *http.Request) ([]prompb.TimeSeries, []prompb.MetricMetadata) { @@ -349,15 +348,15 @@ func newComponent(t *testing.T, l *logging.Logger, url string, exp chan Exports, TTL: 2 * time.Hour, MaxFlushSize: 10_000, FlushDuration: 1 * time.Second, - Connections: []types.ConnectionConfig{{ + Connections: []ConnectionConfig{{ Name: "test", URL: url, Timeout: 20 * time.Second, RetryBackoff: 5 * time.Second, MaxRetryBackoffAttempts: 1, BatchCount: 50, - FlushDuration: 1 * time.Second, - QueueCount: 1, + FlushFrequency: 1 * time.Second, + Connections: 1, }}, AppenderBatchSize: 1_000, ExternalLabels: nil, diff --git a/internal/component/prometheus/remote/queue/endpoint.go b/internal/component/prometheus/remote/queue/endpoint.go index 5cbca5e5e4..2653eb43b6 100644 --- a/internal/component/prometheus/remote/queue/endpoint.go +++ b/internal/component/prometheus/remote/queue/endpoint.go @@ -62,7 +62,7 @@ func (ep *endpoint) DoWork(ctx actor.Context) actor.WorkerStatus { if !ok { return actor.WorkerEnd } - meta, buf, err := file.Get() + meta, buf, err := file.Pop() if err != nil { level.Error(ep.log).Log("msg", "unable to get file contents", "name", file.Name, "err", err) return actor.WorkerContinue @@ -101,10 +101,10 @@ func (ep *endpoint) deserializeAndSend(ctx context.Context, meta map[string]stri Strings: make([]string, stringsCount), } for i := 0; i < seriesCount; i++ { - sg.Series[i] = types.GetTimeSeriesBinary() + sg.Series[i] = types.GetTimeSeriesFromPool() } for i := 0; i < metaCount; i++ { - sg.Metadata[i] = types.GetTimeSeriesBinary() + sg.Metadata[i] = types.GetTimeSeriesFromPool() } sg, ep.buf, err = types.DeserializeToSeriesGroup(sg, ep.buf) diff --git a/internal/component/prometheus/remote/queue/network/loop.go b/internal/component/prometheus/remote/queue/network/loop.go index cbe94f2e41..72fb17ac39 100644 --- a/internal/component/prometheus/remote/queue/network/loop.go +++ b/internal/component/prometheus/remote/queue/network/loop.go @@ -195,7 +195,9 @@ func (l *loop) send(ctx context.Context, retryCount int) sendResult { httpReq.Header.Set("Content-Type", "application/x-protobuf") httpReq.Header.Set("User-Agent", l.cfg.UserAgent) httpReq.Header.Set("X-Prometheus-Remote-Write-Version", "0.1.0") - httpReq.SetBasicAuth(l.cfg.Username, l.cfg.Password) + if l.cfg.BasicAuth != nil { + httpReq.SetBasicAuth(l.cfg.BasicAuth.Username, l.cfg.BasicAuth.Password) + } if retryCount > 0 { httpReq.Header.Set("Retry-Attempt", strconv.Itoa(retryCount)) diff --git a/internal/component/prometheus/remote/queue/network/manager.go b/internal/component/prometheus/remote/queue/network/manager.go index 19e4f5b961..6281536838 100644 --- a/internal/component/prometheus/remote/queue/network/manager.go +++ b/internal/component/prometheus/remote/queue/network/manager.go @@ -42,7 +42,7 @@ func New(cc types.ConnectionConfig, logger log.Logger, seriesStats, metadataStat } // start kicks off a number of concurrent connections. - for i := uint64(0); i < s.cfg.Connections; i++ { + for i := uint(0); i < s.cfg.Connections; i++ { l := newLoop(cc, false, logger, seriesStats) l.self = actor.New(l) s.loops = append(s.loops, l) @@ -120,8 +120,7 @@ func (s *manager) updateConfig(cc types.ConnectionConfig) { level.Debug(s.logger).Log("msg", "dropping all series in loops and creating queue due to config change") s.stopLoops() s.loops = make([]*loop, 0, s.cfg.Connections) - var i uint64 - for ; i < s.cfg.Connections; i++ { + for i := uint(0); i < s.cfg.Connections; i++ { l := newLoop(cc, false, s.logger, s.stats) l.self = actor.New(l) @@ -158,7 +157,7 @@ func (s *manager) startLoops() { // Queue adds anything thats not metadata to the queue. func (s *manager) queue(ctx context.Context, ts *types.TimeSeriesBinary) { // Based on a hash which is the label hash add to the queue. - queueNum := ts.Hash % s.cfg.Connections + queueNum := ts.Hash % uint64(s.cfg.Connections) // This will block if the queue is full. err := s.loops[queueNum].seriesMbx.Send(ctx, ts) if err != nil { diff --git a/internal/component/prometheus/remote/queue/types.go b/internal/component/prometheus/remote/queue/types.go index 57f46ca3ac..1754549ad3 100644 --- a/internal/component/prometheus/remote/queue/types.go +++ b/internal/component/prometheus/remote/queue/types.go @@ -24,8 +24,8 @@ type Arguments struct { // The batch size to persist to the file queue. MaxFlushSize int `alloy:"max_flush_size,attr,optional"` // How often to flush to the file queue if BatchSizeBytes isn't met. - FlushDuration time.Duration `alloy:"flush_duration,attr,optional"` - Connections []types.ConnectionConfig `alloy:"endpoint,block"` + FlushDuration time.Duration `alloy:"flush_duration,attr,optional"` + Connections []ConnectionConfig `alloy:"endpoint,block"` // AppenderBatchSize determines how often to flush the appender batch size. AppenderBatchSize int `alloy:"appender_batch_size,attr,optional"` ExternalLabels map[string]string `alloy:"external_labels,attr,optional"` @@ -84,7 +84,7 @@ type ConnectionConfig struct { ExternalLabels map[string]string `alloy:"external_labels,attr,optional"` } -func (cc ConnectionConfig) ToTypesCC() types.ConnectionConfig { +func (cc ConnectionConfig) ToNativeType() types.ConnectionConfig { tcc := types.ConnectionConfig{ URL: cc.URL, UserAgent: "alloy", From 254dc4ce9b5c320a83b26cfafffe33ff6d91b956 Mon Sep 17 00:00:00 2001 From: Matt Durham Date: Thu, 3 Oct 2024 15:44:34 -0400 Subject: [PATCH 10/44] docs and some renaming --- .../prometheus/prometheus.remote.queue.md | 237 ++++++++++++++++++ .../prometheus/remote/queue/component.go | 37 +-- .../prometheus/remote/queue/config.go | 1 - .../prometheus/remote/queue/e2e_bench_test.go | 12 +- .../prometheus/remote/queue/e2e_test.go | 21 +- .../prometheus/remote/queue/endpoint.go | 5 +- .../prometheus/remote/queue/types.go | 48 ++-- .../prometheus/remote/queue/types/stats.go | 32 +-- .../remote/queue/types/storage_test.go | 7 +- 9 files changed, 320 insertions(+), 80 deletions(-) create mode 100644 docs/sources/reference/components/prometheus/prometheus.remote.queue.md delete mode 100644 internal/component/prometheus/remote/queue/config.go diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md new file mode 100644 index 0000000000..cdde61e751 --- /dev/null +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -0,0 +1,237 @@ +--- +canonical: https://grafana.com/docs/alloy/latest/reference/components/prometheus/prometheus.remote.queue/ +aliases: + - ../prometheus.remote.queue/ # /docs/alloy/latest/reference/components/prometheus.remote.queue/ +description: Learn about prometheus.remote.queue +title: prometheus.remote.queue +--- + +# prometheus.remote.queue + +`prometheus.remote.queue` collects metrics sent from other components into a +Write-Ahead Log (WAL) and forwards them over the network to a series of +user-supplied endpoints. Metrics are sent over the network using the +[Prometheus Remote Write protocol][remote_write-spec]. + +Multiple `prometheus.remote.queue` components can be specified by giving them +different labels. + +[remote_write-spec]: https://docs.google.com/document/d/1LPhVRSFkGNSuU1fBd81ulhsCPR4hkSZyyBj1SZ8fWOM/edit + +## Usage + +```alloy +prometheus.remote.queue "LABEL" { + endpoint { + url = REMOTE_WRITE_URL + + ... + } + + ... +} +``` + +## Arguments + +The following arguments are supported: + +Name | Type | Description | Default | Required +---- | ---- | ----------- | ------- | -------- +`ttl` | `time` | `duration` | How long the timestamp of a signal is valid for, before the signal is discarded. | `2h` | no +`max_signals_to_batch` | `uint` | The maximum number of signals before they are batched to disk. | `10,000` | no +`batch_frequency` | `duration` | How often to batch signals to disk if `max_signals_to_batch` is not reached. | no + + +## Blocks + +The following blocks are supported inside the definition of +`prometheus.remote.queue`: + +Hierarchy | Block | Description | Required +--------- | ----- | ----------- | -------- +endpoint | [endpoint][] | Location to send metrics to. | no +endpoint > basic_auth | [basic_auth][] | Configure basic_auth for authenticating to the endpoint. | no + +The `>` symbol indicates deeper levels of nesting. For example, `endpoint > +basic_auth` refers to a `basic_auth` block defined inside an +`endpoint` block. + +[endpoint]: #endpoint-block +[basic_auth]: #basic_auth-block + +### endpoint block + +The `endpoint` block describes a single location to send metrics to. Multiple +`endpoint` blocks can be provided to send metrics to multiple locations. + +The following arguments are supported: + +Name | Type | Description | Default | Required +---- | ---- | ----------- | ------- | -------- +`url` | `string` | Full URL to send metrics to. | | yes +`name` | `string` | Optional name to identify the endpoint in metrics. | | no +`write_timeout` | `duration` | Timeout for requests made to the URL. | `"30s"` | no +`retry_backoff` | `duration` | How often to wait between retries. | `1s` | no +`max_retry_backoff_attempts` | Maximum number of retries before dropping the batch. | `1s` | no +`batch_count` | `uint` | How many series to queue in each queue. | `1,000` | no +`flush_frequency` | `duration` | How often to wait until sending if `batch_count` is not trigger. | `1s` | no +`queue_count` | `uint` | How many concurrent batches to write. | 10 | no +`external_labels` | `map(string)` | Labels to add to metrics sent over the network. | | no + +### basic_auth block + +{{< docs/shared lookup="reference/components/basic-auth-block.md" source="alloy" version="" >}} + + +## Exported fields + +The following fields are exported and can be referenced by other components: + +Name | Type | Description +---- | ---- | ----------- +`receiver` | `MetricsReceiver` | A value which other components can use to send metrics to. + +## Component health + +`prometheus.remote.queue` is only reported as unhealthy if given an invalid +configuration. In those cases, exported fields are kept at their last healthy +values. + +## Debug information + +`prometheus.remote_write` does not expose any component-specific debug +information. + +## Debug metrics + +The below metrics are provided for backwards compatibility, they behave generally the same but there are likely +edge cases where they differ. + +* `prometheus_remote_write_wal_storage_created_series_total` (counter): Total number of created + series appended to the WAL. +* `prometheus_remote_write_wal_storage_removed_series_total` (counter): Total number of series + removed from the WAL. +* `prometheus_remote_write_wal_samples_appended_total` (counter): Total number of samples + appended to the WAL. +* `prometheus_remote_write_wal_exemplars_appended_total` (counter): Total number of exemplars + appended to the WAL. +* `prometheus_remote_storage_samples_total` (counter): Total number of samples + sent to remote storage. +* `prometheus_remote_storage_exemplars_total` (counter): Total number of + exemplars sent to remote storage. +* `prometheus_remote_storage_metadata_total` (counter): Total number of + metadata entries sent to remote storage. +* `prometheus_remote_storage_samples_failed_total` (counter): Total number of + samples that failed to send to remote storage due to non-recoverable errors. +* `prometheus_remote_storage_exemplars_failed_total` (counter): Total number of + exemplars that failed to send to remote storage due to non-recoverable errors. +* `prometheus_remote_storage_metadata_failed_total` (counter): Total number of + metadata entries that failed to send to remote storage due to + non-recoverable errors. +* `prometheus_remote_storage_samples_retries_total` (counter): Total number of + samples that failed to send to remote storage but were retried due to + recoverable errors. +* `prometheus_remote_storage_exemplars_retried_total` (counter): Total number of + exemplars that failed to send to remote storage but were retried due to + recoverable errors. +* `prometheus_remote_storage_metadata_retried_total` (counter): Total number of + metadata entries that failed to send to remote storage but were retried due + to recoverable errors. +* `prometheus_remote_storage_samples_dropped_total` (counter): Total number of + samples which were dropped after being read from the WAL before being sent to + remote_write because of an unknown reference ID. +* `prometheus_remote_storage_exemplars_dropped_total` (counter): Total number + of exemplars which were dropped after being read from the WAL before being + sent to remote_write because of an unknown reference ID. +* `prometheus_remote_storage_enqueue_retries_total` (counter): Total number of + times enqueue has failed because a shard's queue was full. +* `prometheus_remote_storage_sent_batch_duration_seconds` (histogram): Duration + of send calls to remote storage. +* `prometheus_remote_storage_queue_highest_sent_timestamp_seconds` (gauge): + Unix timestamp of the latest WAL sample successfully sent by a queue. +* `prometheus_remote_storage_samples_pending` (gauge): The number of samples + pending in shards to be sent to remote storage. +* `prometheus_remote_storage_exemplars_pending` (gauge): The number of + exemplars pending in shards to be sent to remote storage. +* `prometheus_remote_storage_samples_in_total` (counter): Samples read into + remote storage. +* `prometheus_remote_storage_exemplars_in_total` (counter): Exemplars read into + remote storage. + +TODO document new metrics. + +## Examples + +The following examples show you how to create `prometheus.remote_write` components that send metrics to different destinations. + +### Send metrics to a local Mimir instance + +You can create a `prometheus.remote.queue` component that sends your metrics to a local Mimir instance: + +```alloy +prometheus.remote.queue "staging" { + // Send metrics to a locally running Mimir. + endpoint "mimir" { + url = "http://mimir:9009/api/v1/push" + + basic_auth { + username = "example-user" + password = "example-password" + } + } +} + +// Configure a prometheus.scrape component to send metrics to +// prometheus.remote_write component. +prometheus.scrape "demo" { + targets = [ + // Collect metrics from the default HTTP listen address. + {"__address__" = "127.0.0.1:12345"}, + ] + forward_to = [prometheus.remote.queue.staging.receiver] +} + +``` + +## TODO Metadata settings + +## Technical details + +`prometheus.remote.queue` uses [snappy][] for compression. +`prometheus.remote.queue` sends native histograms by default. +Any labels that start with `__` will be removed before sending to the endpoint. + +### Data retention + +Data is written to disk in blocks utilizing [snappy][] compression. These blocks are read on startup and resent if they are still within the TTL. +Any data that has not been written to disk, or that is in the network queues is lost if Alloy is restarted. + +### Retries + +Network errors will be retried. 429 errors will be retried. 5XX errors will retry. Any other non-2XX return codes will not be tried. + +### Memory + +`prometheus.remote.queue` is meant to be memory efficient. By adjusting the `max_signals_to_batch`, `queue_count`, and `batch_size` the amount of memory +can be controlled. A higher `max_signals_to_batch` allows for more efficient disk compression. A higher `queue_count` allows more concurrent writes and `batch_size` +allows more data sent at one time. This can allow greater throughput, at the cost of more memory on both Alloy and the endpoint. The defaults are good for most +common usages. + +## Compatible components + +`prometheus.remote.queue` has exports that can be consumed by the following components: + +- Components that consume [Prometheus `MetricsReceiver`](../../../compatibility/#prometheus-metricsreceiver-consumers) + +{{< admonition type="note" >}} +Connecting some components may not be sensible or components may require further configuration to make the connection work correctly. +Refer to the linked documentation for more details. +{{< /admonition >}} + + + +[snappy]: https://en.wikipedia.org/wiki/Snappy_(compression) +[WAL block]: #wal-block +[Stop]: ../../../../set-up/run/ +[run]: ../../../cli/run/ diff --git a/internal/component/prometheus/remote/queue/component.go b/internal/component/prometheus/remote/queue/component.go index a4185b4f3e..e82ca88a23 100644 --- a/internal/component/prometheus/remote/queue/component.go +++ b/internal/component/prometheus/remote/queue/component.go @@ -5,17 +5,15 @@ import ( "path/filepath" "reflect" "sync" - "time" - - "github.com/grafana/alloy/internal/component/prometheus/remote/queue/filequeue" - "github.com/grafana/alloy/internal/component/prometheus/remote/queue/network" - "github.com/prometheus/client_golang/prometheus" "github.com/go-kit/log" "github.com/grafana/alloy/internal/component" + "github.com/grafana/alloy/internal/component/prometheus/remote/queue/filequeue" + "github.com/grafana/alloy/internal/component/prometheus/remote/queue/network" "github.com/grafana/alloy/internal/component/prometheus/remote/queue/serialization" "github.com/grafana/alloy/internal/component/prometheus/remote/queue/types" "github.com/grafana/alloy/internal/featuregate" + "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/storage" ) @@ -60,6 +58,9 @@ type Queue struct { // suffers a fatal error. Run is guaranteed to be called exactly once per // Component. func (s *Queue) Run(ctx context.Context) error { + for _, ep := range s.endpoints { + ep.Start() + } defer func() { s.mut.Lock() defer s.mut.Unlock() @@ -100,7 +101,14 @@ func (s *Queue) Update(args component.Arguments) error { } s.endpoints = map[string]*endpoint{} } - return s.createEndpoints() + err := s.createEndpoints() + if err != nil { + return err + } + for _, ep := range s.endpoints { + ep.Start() + } + return nil } func (s *Queue) createEndpoints() error { @@ -110,14 +118,13 @@ func (s *Queue) createEndpoints() error { stats.BackwardsCompatibility(reg) meta := types.NewStats("alloy", "queue_metadata", reg) cfg := types.ConnectionConfig{ - URL: ep.URL, - BatchCount: ep.BatchCount, - // Functionally this cannot go below 1s + URL: ep.URL, + BatchCount: ep.BatchCount, FlushFrequency: ep.FlushFrequency, Timeout: ep.Timeout, UserAgent: "alloy", ExternalLabels: s.args.ExternalLabels, - Connections: ep.Connections, + Connections: ep.QueueCount, } if ep.BasicAuth != nil { cfg.BasicAuth = &types.BasicAuth{ @@ -126,13 +133,10 @@ func (s *Queue) createEndpoints() error { } } client, err := network.New(cfg, s.log, stats.UpdateNetwork, meta.UpdateNetwork) - if err != nil { return err } - // Serializer is set after end := NewEndpoint(client, nil, stats, meta, s.args.TTL, s.opts.Logger) - // This wait group is to ensure we are started before we send on the mailbox. fq, err := filequeue.NewQueue(filepath.Join(s.opts.DataPath, ep.Name, "wal"), func(ctx context.Context, dh types.DataHandle) { _ = end.incoming.Send(ctx, dh) }, s.opts.Logger) @@ -140,17 +144,14 @@ func (s *Queue) createEndpoints() error { return err } serial, err := serialization.NewSerializer(types.SerializerConfig{ - MaxSignalsInBatch: 10_000, - FlushFrequency: 1 * time.Second, + MaxSignalsInBatch: uint32(s.args.MaxSignalsToBatch), + FlushFrequency: s.args.BatchFrequency, }, fq, stats.UpdateFileQueue, s.opts.Logger) if err != nil { return err } end.serializer = serial s.endpoints[ep.Name] = end - // endpoint is responsible for starting all the children, this way they spin up - // together and are town down together. Or at least handled internally. - end.Start() } return nil } diff --git a/internal/component/prometheus/remote/queue/config.go b/internal/component/prometheus/remote/queue/config.go deleted file mode 100644 index c969bad24a..0000000000 --- a/internal/component/prometheus/remote/queue/config.go +++ /dev/null @@ -1 +0,0 @@ -package queue diff --git a/internal/component/prometheus/remote/queue/e2e_bench_test.go b/internal/component/prometheus/remote/queue/e2e_bench_test.go index 94212a25ce..1de030d39c 100644 --- a/internal/component/prometheus/remote/queue/e2e_bench_test.go +++ b/internal/component/prometheus/remote/queue/e2e_bench_test.go @@ -79,7 +79,6 @@ func runBenchmark(t *testing.B, add func(index int, appendable storage.Appender) case <-tm.C: } cancel() - } func newComponentBenchmark(t *testing.B, l log.Logger, url string, exp chan Exports) (*Queue, error) { @@ -93,9 +92,9 @@ func newComponentBenchmark(t *testing.B, l log.Logger, url string, exp chan Expo Registerer: fakeRegistry{}, Tracer: nil, }, Arguments{ - TTL: 2 * time.Hour, - MaxFlushSize: 100_000, - FlushDuration: 1 * time.Second, + TTL: 2 * time.Hour, + MaxSignalsToBatch: 100_000, + BatchFrequency: 1 * time.Second, Connections: []ConnectionConfig{{ Name: "test", URL: url, @@ -104,10 +103,9 @@ func newComponentBenchmark(t *testing.B, l log.Logger, url string, exp chan Expo MaxRetryBackoffAttempts: 0, BatchCount: 50, FlushFrequency: 1 * time.Second, - Connections: 1, + QueueCount: 1, }}, - AppenderBatchSize: 1_000, - ExternalLabels: nil, + ExternalLabels: nil, }) } diff --git a/internal/component/prometheus/remote/queue/e2e_test.go b/internal/component/prometheus/remote/queue/e2e_test.go index b4a581f8f9..58e5597d61 100644 --- a/internal/component/prometheus/remote/queue/e2e_test.go +++ b/internal/component/prometheus/remote/queue/e2e_test.go @@ -121,8 +121,10 @@ func TestE2E(t *testing.T) { } } -const iterations = 100 -const items = 10_000 +const ( + iterations = 100 + items = 10_000 +) func runTest(t *testing.T, add func(index int, appendable storage.Appender) (float64, labels.Labels), test func(samples []prompb.TimeSeries), metaTest func(meta []prompb.MetricMetadata)) { l := util.TestAlloyLogger(t) @@ -172,7 +174,6 @@ func runTest(t *testing.T, add func(index int, appendable storage.Appender) (flo } require.NoError(t, app.Commit()) }() - } // This is a weird use case to handle eventually. tm := time.NewTimer(15 * time.Second) @@ -197,7 +198,6 @@ func runTest(t *testing.T, add func(index int, appendable storage.Appender) (flo require.True(t, lbls[i].Value == sLbl.Value) } } - } if test != nil { test(samples) @@ -219,7 +219,6 @@ func handlePost(t *testing.T, _ http.ResponseWriter, r *http.Request) ([]prompb. err = req.Unmarshal(data) require.NoError(t, err) return req.GetTimeseries(), req.Metadata - } func makeSeries(index int) (int64, float64, labels.Labels) { @@ -237,6 +236,7 @@ func makeMetadata(index int) (metadata.Metadata, labels.Labels) { func makeHistogram(index int) (int64, labels.Labels, *histogram.Histogram) { return time.Now().UTC().Unix(), labels.FromStrings(fmt.Sprintf("name_%d", index), fmt.Sprintf("value_%d", index)), hist(index) } + func makeExemplar(index int) exemplar.Exemplar { return exemplar.Exemplar{ Labels: labels.FromStrings(fmt.Sprintf("name_%d", index), fmt.Sprintf("value_%d", index)), @@ -345,9 +345,9 @@ func newComponent(t *testing.T, l *logging.Logger, url string, exp chan Exports, Registerer: reg, Tracer: nil, }, Arguments{ - TTL: 2 * time.Hour, - MaxFlushSize: 10_000, - FlushDuration: 1 * time.Second, + TTL: 2 * time.Hour, + MaxSignalsToBatch: 10_000, + BatchFrequency: 1 * time.Second, Connections: []ConnectionConfig{{ Name: "test", URL: url, @@ -356,9 +356,8 @@ func newComponent(t *testing.T, l *logging.Logger, url string, exp chan Exports, MaxRetryBackoffAttempts: 1, BatchCount: 50, FlushFrequency: 1 * time.Second, - Connections: 1, + QueueCount: 1, }}, - AppenderBatchSize: 1_000, - ExternalLabels: nil, + ExternalLabels: nil, }) } diff --git a/internal/component/prometheus/remote/queue/endpoint.go b/internal/component/prometheus/remote/queue/endpoint.go index 2653eb43b6..3f80528aff 100644 --- a/internal/component/prometheus/remote/queue/endpoint.go +++ b/internal/component/prometheus/remote/queue/endpoint.go @@ -74,7 +74,7 @@ func (ep *endpoint) DoWork(ctx actor.Context) actor.WorkerStatus { func (ep *endpoint) deserializeAndSend(ctx context.Context, meta map[string]string, buf []byte) { var err error - ep.buf, err = snappy.Decode(buf) + ep.buf, err = snappy.DecodeInto(ep.buf, buf) if err != nil { level.Debug(ep.log).Log("msg", "error snappy decoding", "err", err) return @@ -100,6 +100,7 @@ func (ep *endpoint) deserializeAndSend(ctx context.Context, meta map[string]stri Metadata: make([]*types.TimeSeriesBinary, metaCount), Strings: make([]string, stringsCount), } + // Prefill our series with items from the pool to limit allocs. for i := 0; i < seriesCount; i++ { sg.Series[i] = types.GetTimeSeriesFromPool() } @@ -107,7 +108,6 @@ func (ep *endpoint) deserializeAndSend(ctx context.Context, meta map[string]stri sg.Metadata[i] = types.GetTimeSeriesFromPool() } sg, ep.buf, err = types.DeserializeToSeriesGroup(sg, ep.buf) - if err != nil { level.Debug(ep.log).Log("msg", "error deserializing", "err", err) return @@ -119,6 +119,7 @@ func (ep *endpoint) deserializeAndSend(ctx context.Context, meta map[string]stri // data that will get rejected. seriesAge := time.Since(time.Unix(series.TS, 0)) if seriesAge > ep.ttl { + // TODO @mattdurham add metric here for ttl expired. continue } sendErr := ep.network.SendSeries(ctx, series) diff --git a/internal/component/prometheus/remote/queue/types.go b/internal/component/prometheus/remote/queue/types.go index 1754549ad3..7e424a62d9 100644 --- a/internal/component/prometheus/remote/queue/types.go +++ b/internal/component/prometheus/remote/queue/types.go @@ -2,9 +2,9 @@ package queue import ( "fmt" - "github.com/grafana/alloy/internal/component/prometheus/remote/queue/types" "time" + "github.com/grafana/alloy/internal/component/prometheus/remote/queue/types" "github.com/grafana/alloy/syntax/alloytypes" "github.com/prometheus/prometheus/storage" ) @@ -12,9 +12,8 @@ import ( func defaultArgs() Arguments { return Arguments{ TTL: 2 * time.Hour, - MaxFlushSize: 10_000, - FlushDuration: 5 * time.Second, - AppenderBatchSize: 1_000, + MaxSignalsToBatch: 10_000, + BatchFrequency: 5 * time.Second, } } @@ -22,13 +21,11 @@ type Arguments struct { // TTL is how old a series can be. TTL time.Duration `alloy:"ttl,attr,optional"` // The batch size to persist to the file queue. - MaxFlushSize int `alloy:"max_flush_size,attr,optional"` - // How often to flush to the file queue if BatchSizeBytes isn't met. - FlushDuration time.Duration `alloy:"flush_duration,attr,optional"` - Connections []ConnectionConfig `alloy:"endpoint,block"` - // AppenderBatchSize determines how often to flush the appender batch size. - AppenderBatchSize int `alloy:"appender_batch_size,attr,optional"` - ExternalLabels map[string]string `alloy:"external_labels,attr,optional"` + MaxSignalsToBatch int `alloy:"max_signals_to_batch,attr,optional"` + // How often to flush to the file queue if BatchSize isn't met. + // TODO @mattdurham this may need to go into a specific block for the serializer. + BatchFrequency time.Duration `alloy:"batch_frequency,attr,optional"` + Connections []ConnectionConfig `alloy:"endpoint,block"` } type Exports struct { @@ -39,32 +36,39 @@ type Exports struct { func (rc *Arguments) SetToDefault() { *rc = defaultArgs() } + func defaultCC() ConnectionConfig { return ConnectionConfig{ - Timeout: 15 * time.Second, + Timeout: 30 * time.Second, RetryBackoff: 1 * time.Second, MaxRetryBackoffAttempts: 0, BatchCount: 1_000, FlushFrequency: 1 * time.Second, - Connections: 4, + QueueCount: 4, } } + func (cc *ConnectionConfig) SetToDefault() { *cc = defaultCC() } func (r *Arguments) Validate() error { - if r.AppenderBatchSize == 0 { - return fmt.Errorf("appender_batch_size must be greater than zero") - } for _, conn := range r.Connections { if conn.BatchCount <= 0 { return fmt.Errorf("batch_count must be greater than 0") } + if conn.FlushFrequency < 1*time.Second { + return fmt.Errorf("flush_frequency must be greater or equal to 1s, the internal timers resolution is 1s") + } } + return nil } +// ConnectionConfig is the alloy specific version of ConnectionConfig. This looks odd, the idea +// +// is that once this code is tested that the bulk of the underlying code will be used elsewhere. +// this means we need a very generic interface for that code, and a specific alloy implementation here. type ConnectionConfig struct { Name string `alloy:",label"` URL string `alloy:"url,attr"` @@ -73,20 +77,21 @@ type ConnectionConfig struct { // How long to wait between retries. RetryBackoff time.Duration `alloy:"retry_backoff,attr,optional"` // Maximum number of retries. - MaxRetryBackoffAttempts uint `alloy:"max_retry_backoff,attr,optional"` + MaxRetryBackoffAttempts uint `alloy:"max_retry_backoff_attempts,attr,optional"` // How many series to write at a time. BatchCount int `alloy:"batch_count,attr,optional"` // How long to wait before sending regardless of batch count. - FlushFrequency time.Duration `alloy:"flush_duration,attr,optional"` + FlushFrequency time.Duration `alloy:"flush_frequency,attr,optional"` // How many concurrent queues to have. - Connections uint `alloy:"queue_count,attr,optional"` + QueueCount uint `alloy:"queue_count,attr,optional"` ExternalLabels map[string]string `alloy:"external_labels,attr,optional"` } func (cc ConnectionConfig) ToNativeType() types.ConnectionConfig { tcc := types.ConnectionConfig{ - URL: cc.URL, + URL: cc.URL, + // TODO @mattdurham generate this with build information. UserAgent: "alloy", Timeout: cc.Timeout, RetryBackoff: cc.RetryBackoff, @@ -94,7 +99,7 @@ func (cc ConnectionConfig) ToNativeType() types.ConnectionConfig { BatchCount: cc.BatchCount, FlushFrequency: cc.FlushFrequency, ExternalLabels: cc.ExternalLabels, - Connections: cc.Connections, + Connections: cc.QueueCount, } if cc.BasicAuth != nil { tcc.BasicAuth = &types.BasicAuth{ @@ -103,7 +108,6 @@ func (cc ConnectionConfig) ToNativeType() types.ConnectionConfig { } } return tcc - } type BasicAuth struct { diff --git a/internal/component/prometheus/remote/queue/types/stats.go b/internal/component/prometheus/remote/queue/types/stats.go index 4107d8089f..f68600076e 100644 --- a/internal/component/prometheus/remote/queue/types/stats.go +++ b/internal/component/prometheus/remote/queue/types/stats.go @@ -24,10 +24,10 @@ type PrometheusStats struct { NetworkErrors prometheus.Counter NetworkNewestOutTimeStampSeconds prometheus.Gauge - // Filequeue Stats - FilequeueInSeries prometheus.Counter - FilequeueNewestInTimeStampSeconds prometheus.Gauge - FilequeueErrors prometheus.Counter + // Serializer Stats + SerializerInSeries prometheus.Counter + SerializerNewestInTimeStampSeconds prometheus.Gauge + SerializerErrors prometheus.Counter // Backwards compatibility metrics SamplesTotal prometheus.Counter @@ -55,20 +55,20 @@ type PrometheusStats struct { func NewStats(namespace, subsystem string, registry prometheus.Registerer) *PrometheusStats { s := &PrometheusStats{ - FilequeueInSeries: prometheus.NewCounter(prometheus.CounterOpts{ + SerializerInSeries: prometheus.NewCounter(prometheus.CounterOpts{ Namespace: namespace, Subsystem: subsystem, - Name: "filequeue_incoming", + Name: "serializer_incoming_signals", }), - FilequeueNewestInTimeStampSeconds: prometheus.NewGauge(prometheus.GaugeOpts{ + SerializerNewestInTimeStampSeconds: prometheus.NewGauge(prometheus.GaugeOpts{ Namespace: namespace, Subsystem: subsystem, - Name: "filequeue_incoming_timestamp_seconds", + Name: "serializer_incoming_timestamp_seconds", }), - FilequeueErrors: prometheus.NewGauge(prometheus.GaugeOpts{ + SerializerErrors: prometheus.NewGauge(prometheus.GaugeOpts{ Namespace: namespace, Subsystem: subsystem, - Name: "filequeue_errors", + Name: "serializer_errors", }), NetworkNewestOutTimeStampSeconds: prometheus.NewGauge(prometheus.GaugeOpts{ Namespace: namespace, @@ -175,9 +175,9 @@ func NewStats(namespace, subsystem string, registry prometheus.Registerer) *Prom s.NetworkSeriesSent, s.NetworkErrors, s.NetworkNewestOutTimeStampSeconds, - s.FilequeueInSeries, - s.FilequeueErrors, - s.FilequeueNewestInTimeStampSeconds, + s.SerializerInSeries, + s.SerializerErrors, + s.SerializerNewestInTimeStampSeconds, ) return s } @@ -231,10 +231,10 @@ func (s *PrometheusStats) UpdateNetwork(stats NetworkStats) { } func (s *PrometheusStats) UpdateFileQueue(stats SerializerStats) { - s.FilequeueInSeries.Add(float64(stats.SeriesStored)) - s.FilequeueErrors.Add(float64(stats.Errors)) + s.SerializerInSeries.Add(float64(stats.SeriesStored)) + s.SerializerErrors.Add(float64(stats.Errors)) if stats.NewestTimestamp != 0 { - s.FilequeueNewestInTimeStampSeconds.Set(float64(stats.NewestTimestamp)) + s.SerializerNewestInTimeStampSeconds.Set(float64(stats.NewestTimestamp)) s.RemoteStorageInTimestamp.Set(float64(stats.NewestTimestamp)) } } diff --git a/internal/component/prometheus/remote/queue/types/storage_test.go b/internal/component/prometheus/remote/queue/types/storage_test.go index bfd0d8b67a..f994427792 100644 --- a/internal/component/prometheus/remote/queue/types/storage_test.go +++ b/internal/component/prometheus/remote/queue/types/storage_test.go @@ -8,15 +8,16 @@ import ( ) func TestStorage(t *testing.T) { - ts := GetTimeSeriesBinary() + ts := GetTimeSeriesFromPool() ts.Labels = labels.FromStrings("one", "two") ts.LabelsValues = make([]uint32, 1) ts.LabelsNames = make([]uint32, 1) ts.LabelsValues[0] = 1 ts.LabelsNames[0] = 2 - PutTimeSeriesBinary(ts) - ts = GetTimeSeriesBinary() + PutTimeSeriesIntoPool(ts) + ts = GetTimeSeriesFromPool() + defer PutTimeSeriesIntoPool(ts) require.Len(t, ts.Labels, 0) require.True(t, cap(ts.LabelsValues) == 1) require.True(t, cap(ts.LabelsNames) == 1) From e06efff6167d9e466340039dd4b0b85fda33bb4e Mon Sep 17 00:00:00 2001 From: mattdurham Date: Fri, 4 Oct 2024 08:55:02 -0400 Subject: [PATCH 11/44] Update docs/sources/reference/components/prometheus/prometheus.remote.queue.md Co-authored-by: Clayton Cornell <131809008+clayton-cornell@users.noreply.github.com> --- .../reference/components/prometheus/prometheus.remote.queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index cdde61e751..60b352ac0e 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -215,7 +215,7 @@ Network errors will be retried. 429 errors will be retried. 5XX errors will retr `prometheus.remote.queue` is meant to be memory efficient. By adjusting the `max_signals_to_batch`, `queue_count`, and `batch_size` the amount of memory can be controlled. A higher `max_signals_to_batch` allows for more efficient disk compression. A higher `queue_count` allows more concurrent writes and `batch_size` -allows more data sent at one time. This can allow greater throughput, at the cost of more memory on both Alloy and the endpoint. The defaults are good for most +allows more data sent at one time. This can allow greater throughput, at the cost of more memory on both {{< param "PRODUCT_NAME" >}} and the endpoint. The defaults are good for most common usages. ## Compatible components From 992c7031334e969cdf49364dc0c628191909b7f7 Mon Sep 17 00:00:00 2001 From: mattdurham Date: Fri, 4 Oct 2024 08:55:08 -0400 Subject: [PATCH 12/44] Update docs/sources/reference/components/prometheus/prometheus.remote.queue.md Co-authored-by: Clayton Cornell <131809008+clayton-cornell@users.noreply.github.com> --- .../reference/components/prometheus/prometheus.remote.queue.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index 60b352ac0e..73068b8abf 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -1,7 +1,5 @@ --- canonical: https://grafana.com/docs/alloy/latest/reference/components/prometheus/prometheus.remote.queue/ -aliases: - - ../prometheus.remote.queue/ # /docs/alloy/latest/reference/components/prometheus.remote.queue/ description: Learn about prometheus.remote.queue title: prometheus.remote.queue --- From 05467072edece4d60108eb5b71e4bc9a5028b2cd Mon Sep 17 00:00:00 2001 From: mattdurham Date: Fri, 4 Oct 2024 08:55:13 -0400 Subject: [PATCH 13/44] Update docs/sources/reference/components/prometheus/prometheus.remote.queue.md Co-authored-by: Clayton Cornell <131809008+clayton-cornell@users.noreply.github.com> --- .../reference/components/prometheus/prometheus.remote.queue.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index 73068b8abf..1e7cb2e639 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -11,8 +11,7 @@ Write-Ahead Log (WAL) and forwards them over the network to a series of user-supplied endpoints. Metrics are sent over the network using the [Prometheus Remote Write protocol][remote_write-spec]. -Multiple `prometheus.remote.queue` components can be specified by giving them -different labels. +You can specify multiple `prometheus.remote.queue` components by giving them different labels. [remote_write-spec]: https://docs.google.com/document/d/1LPhVRSFkGNSuU1fBd81ulhsCPR4hkSZyyBj1SZ8fWOM/edit From e75e34f1c22f7147d01e038a4a7c7db05af891e8 Mon Sep 17 00:00:00 2001 From: mattdurham Date: Fri, 4 Oct 2024 08:55:19 -0400 Subject: [PATCH 14/44] Update docs/sources/reference/components/prometheus/prometheus.remote.queue.md Co-authored-by: Clayton Cornell <131809008+clayton-cornell@users.noreply.github.com> --- .../reference/components/prometheus/prometheus.remote.queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index 1e7cb2e639..5d04c1457b 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -87,7 +87,7 @@ The following fields are exported and can be referenced by other components: Name | Type | Description ---- | ---- | ----------- -`receiver` | `MetricsReceiver` | A value which other components can use to send metrics to. +`receiver` | `MetricsReceiver` | A value that other components can use to send metrics to. ## Component health From 0ea1f7159a4cafef82b0c0e9df31bf6339950b97 Mon Sep 17 00:00:00 2001 From: mattdurham Date: Fri, 4 Oct 2024 08:55:25 -0400 Subject: [PATCH 15/44] Update docs/sources/reference/components/prometheus/prometheus.remote.queue.md Co-authored-by: Clayton Cornell <131809008+clayton-cornell@users.noreply.github.com> --- .../components/prometheus/prometheus.remote.queue.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index 5d04c1457b..bb5dccbc40 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -102,8 +102,8 @@ information. ## Debug metrics -The below metrics are provided for backwards compatibility, they behave generally the same but there are likely -edge cases where they differ. +The following metrics are provided for backward compatibility. +They generally behave the same, but there are likely edge cases where they differ. * `prometheus_remote_write_wal_storage_created_series_total` (counter): Total number of created series appended to the WAL. From 9a90c2be864e16012f9d9b007293e527b11a5b22 Mon Sep 17 00:00:00 2001 From: mattdurham Date: Fri, 4 Oct 2024 08:55:30 -0400 Subject: [PATCH 16/44] Update docs/sources/reference/components/prometheus/prometheus.remote.queue.md Co-authored-by: Clayton Cornell <131809008+clayton-cornell@users.noreply.github.com> --- .../reference/components/prometheus/prometheus.remote.queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index bb5dccbc40..d42111546d 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -139,7 +139,7 @@ They generally behave the same, but there are likely edge cases where they diffe samples which were dropped after being read from the WAL before being sent to remote_write because of an unknown reference ID. * `prometheus_remote_storage_exemplars_dropped_total` (counter): Total number - of exemplars which were dropped after being read from the WAL before being + of exemplars that were dropped after being read from the WAL before being sent to remote_write because of an unknown reference ID. * `prometheus_remote_storage_enqueue_retries_total` (counter): Total number of times enqueue has failed because a shard's queue was full. From f04edb4587a0cc2ae0c13a425a5b4ee33fade82e Mon Sep 17 00:00:00 2001 From: mattdurham Date: Fri, 4 Oct 2024 08:55:35 -0400 Subject: [PATCH 17/44] Update docs/sources/reference/components/prometheus/prometheus.remote.queue.md Co-authored-by: Clayton Cornell <131809008+clayton-cornell@users.noreply.github.com> --- .../reference/components/prometheus/prometheus.remote.queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index d42111546d..db93f19597 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -202,7 +202,7 @@ Any labels that start with `__` will be removed before sending to the endpoint. ### Data retention Data is written to disk in blocks utilizing [snappy][] compression. These blocks are read on startup and resent if they are still within the TTL. -Any data that has not been written to disk, or that is in the network queues is lost if Alloy is restarted. +Any data that has not been written to disk, or that is in the network queues is lost if {{< param "PRODUCT_NAME" >}} is restarted. ### Retries From 0d99288888c9d5d60059fa2042010cbfa03f05b5 Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 8 Oct 2024 07:52:36 -0400 Subject: [PATCH 18/44] Changes and testing. --- internal/component/all/all.go | 3 +- .../prometheus/remote/queue/component.go | 39 ++--- .../prometheus/remote/queue/e2e_bench_test.go | 11 +- .../prometheus/remote/queue/e2e_stats_test.go | 151 +++++++++++++++--- .../prometheus/remote/queue/e2e_test.go | 11 +- .../prometheus/remote/queue/endpoint.go | 6 +- .../prometheus/remote/queue/network/loop.go | 13 +- .../remote/queue/network/manager.go | 1 + .../remote/queue/serialization/appender.go | 11 +- .../remote/queue/serialization/serializer.go | 17 +- .../prometheus/remote/queue/types.go | 49 +++--- .../prometheus/remote/queue/types/stats.go | 18 ++- .../pyroscope/scrape/manager_test.go | 7 +- 13 files changed, 231 insertions(+), 106 deletions(-) diff --git a/internal/component/all/all.go b/internal/component/all/all.go index bfdde5c5b3..99edcd7c30 100644 --- a/internal/component/all/all.go +++ b/internal/component/all/all.go @@ -81,10 +81,10 @@ import ( _ "github.com/grafana/alloy/internal/component/otelcol/processor/attributes" // Import otelcol.processor.attributes _ "github.com/grafana/alloy/internal/component/otelcol/processor/batch" // Import otelcol.processor.batch _ "github.com/grafana/alloy/internal/component/otelcol/processor/deltatocumulative" // Import otelcol.processor.deltatocumulative - _ "github.com/grafana/alloy/internal/component/otelcol/processor/interval" // Import otelcol.processor.interval _ "github.com/grafana/alloy/internal/component/otelcol/processor/discovery" // Import otelcol.processor.discovery _ "github.com/grafana/alloy/internal/component/otelcol/processor/filter" // Import otelcol.processor.filter _ "github.com/grafana/alloy/internal/component/otelcol/processor/groupbyattrs" // Import otelcol.processor.groupbyattrs + _ "github.com/grafana/alloy/internal/component/otelcol/processor/interval" // Import otelcol.processor.interval _ "github.com/grafana/alloy/internal/component/otelcol/processor/k8sattributes" // Import otelcol.processor.k8sattributes _ "github.com/grafana/alloy/internal/component/otelcol/processor/memorylimiter" // Import otelcol.processor.memory_limiter _ "github.com/grafana/alloy/internal/component/otelcol/processor/probabilistic_sampler" // Import otelcol.processor.probabilistic_sampler @@ -134,6 +134,7 @@ import ( _ "github.com/grafana/alloy/internal/component/prometheus/operator/servicemonitors" // Import prometheus.operator.servicemonitors _ "github.com/grafana/alloy/internal/component/prometheus/receive_http" // Import prometheus.receive_http _ "github.com/grafana/alloy/internal/component/prometheus/relabel" // Import prometheus.relabel + _ "github.com/grafana/alloy/internal/component/prometheus/remote/queue" // Import prometheus.remote.queue _ "github.com/grafana/alloy/internal/component/prometheus/remotewrite" // Import prometheus.remote_write _ "github.com/grafana/alloy/internal/component/prometheus/scrape" // Import prometheus.scrape _ "github.com/grafana/alloy/internal/component/pyroscope/ebpf" // Import pyroscope.ebpf diff --git a/internal/component/prometheus/remote/queue/component.go b/internal/component/prometheus/remote/queue/component.go index e82ca88a23..59ea239a3d 100644 --- a/internal/component/prometheus/remote/queue/component.go +++ b/internal/component/prometheus/remote/queue/component.go @@ -38,6 +38,9 @@ func NewComponent(opts component.Options, args Arguments) (*Queue, error) { } s.opts.OnStateChange(Exports{Receiver: s}) err := s.createEndpoints() + for _, ep := range s.endpoints { + ep.Start() + } if err != nil { return nil, err } @@ -58,9 +61,6 @@ type Queue struct { // suffers a fatal error. Run is guaranteed to be called exactly once per // Component. func (s *Queue) Run(ctx context.Context) error { - for _, ep := range s.endpoints { - ep.Start() - } defer func() { s.mut.Lock() defer s.mut.Unlock() @@ -94,7 +94,8 @@ func (s *Queue) Update(args component.Arguments) error { return nil } s.args = newArgs - // TODO @mattdurham need to cycle through the endpoints figuring out what changed instead of this global stop and start.. + // TODO @mattdurham need to cycle through the endpoints figuring out what changed instead of this global stop and start. + // TODO @mattdurham is there an issue/race condition with stopping these while the appender is still going on. if len(s.endpoints) > 0 { for _, ep := range s.endpoints { ep.Stop() @@ -112,31 +113,19 @@ func (s *Queue) Update(args component.Arguments) error { } func (s *Queue) createEndpoints() error { - for _, ep := range s.args.Connections { + // @mattdurham not in love with this code. + for _, ep := range s.args.Endpoints { reg := prometheus.WrapRegistererWith(prometheus.Labels{"endpoint": ep.Name}, s.opts.Registerer) stats := types.NewStats("alloy", "queue_series", reg) - stats.BackwardsCompatibility(reg) + stats.SeriesBackwardsCompatibility(reg) meta := types.NewStats("alloy", "queue_metadata", reg) - cfg := types.ConnectionConfig{ - URL: ep.URL, - BatchCount: ep.BatchCount, - FlushFrequency: ep.FlushFrequency, - Timeout: ep.Timeout, - UserAgent: "alloy", - ExternalLabels: s.args.ExternalLabels, - Connections: ep.QueueCount, - } - if ep.BasicAuth != nil { - cfg.BasicAuth = &types.BasicAuth{ - Username: ep.BasicAuth.Username, - Password: string(ep.BasicAuth.Password), - } - } + meta.MetaBackwardsCompatibility(reg) + cfg := ep.ToNativeType() client, err := network.New(cfg, s.log, stats.UpdateNetwork, meta.UpdateNetwork) if err != nil { return err } - end := NewEndpoint(client, nil, stats, meta, s.args.TTL, s.opts.Logger) + end := NewEndpoint(client, nil, s.args.TTL, s.opts.Logger) fq, err := filequeue.NewQueue(filepath.Join(s.opts.DataPath, ep.Name, "wal"), func(ctx context.Context, dh types.DataHandle) { _ = end.incoming.Send(ctx, dh) }, s.opts.Logger) @@ -144,9 +133,9 @@ func (s *Queue) createEndpoints() error { return err } serial, err := serialization.NewSerializer(types.SerializerConfig{ - MaxSignalsInBatch: uint32(s.args.MaxSignalsToBatch), - FlushFrequency: s.args.BatchFrequency, - }, fq, stats.UpdateFileQueue, s.opts.Logger) + MaxSignalsInBatch: uint32(s.args.Serialization.MaxSignalsToBatch), + FlushFrequency: s.args.Serialization.BatchFrequency, + }, fq, stats.UpdateSerializer, s.opts.Logger) if err != nil { return err } diff --git a/internal/component/prometheus/remote/queue/e2e_bench_test.go b/internal/component/prometheus/remote/queue/e2e_bench_test.go index 1de030d39c..a730687cd0 100644 --- a/internal/component/prometheus/remote/queue/e2e_bench_test.go +++ b/internal/component/prometheus/remote/queue/e2e_bench_test.go @@ -92,10 +92,12 @@ func newComponentBenchmark(t *testing.B, l log.Logger, url string, exp chan Expo Registerer: fakeRegistry{}, Tracer: nil, }, Arguments{ - TTL: 2 * time.Hour, - MaxSignalsToBatch: 100_000, - BatchFrequency: 1 * time.Second, - Connections: []ConnectionConfig{{ + TTL: 2 * time.Hour, + Serialization: Serialization{ + MaxSignalsToBatch: 100_000, + BatchFrequency: 1 * time.Second, + }, + Endpoints: []EndpointConfig{{ Name: "test", URL: url, Timeout: 10 * time.Second, @@ -105,7 +107,6 @@ func newComponentBenchmark(t *testing.B, l log.Logger, url string, exp chan Expo FlushFrequency: 1 * time.Second, QueueCount: 1, }}, - ExternalLabels: nil, }) } diff --git a/internal/component/prometheus/remote/queue/e2e_stats_test.go b/internal/component/prometheus/remote/queue/e2e_stats_test.go index 1fe7601184..38cb36dd11 100644 --- a/internal/component/prometheus/remote/queue/e2e_stats_test.go +++ b/internal/component/prometheus/remote/queue/e2e_stats_test.go @@ -33,14 +33,113 @@ const retriedMetadata = "prometheus_remote_storage_metadata_retried_total" const prometheusDuration = "prometheus_remote_storage_queue_duration_seconds" -const filequeueIncoming = "alloy_queue_series_filequeue_incoming" +const serializerIncoming = "alloy_queue_series_serializer_incoming_signals" const alloySent = "alloy_queue_series_network_sent" -const alloyFileQueueIncoming = "alloy_queue_series_filequeue_incoming_timestamp_seconds" +const alloySerializerIncoming = "alloy_queue_series_serializer_incoming_timestamp_seconds" const alloyNetworkDuration = "alloy_queue_series_network_duration_seconds" const alloyFailures = "alloy_queue_series_network_failed" const alloyRetries = "alloy_queue_series_network_retried" const alloy429 = "alloy_queue_series_network_retried_429" +const alloyMetadataDuration = "alloy_queue_metadata_network_duration_seconds" +const alloyMetadataSent = "alloy_queue_metadata_network_sent" +const alloyMetadataFailed = "alloy_queue_metadata_network_failed" +const alloyMetadataRetried429 = "alloy_queue_metadata_network_retried_429" +const alloyMetadataRetried = "alloy_queue_metadata_network_retried" + +// TestMetadata is the large end to end testing for the queue based wal, specifically for metadata. +func TestMetadata(t *testing.T) { + // Check assumes you are checking for any value that is not 0. + // The test at the end will see if there are any values that were not 0. + tests := []statsTest{ + // Metadata Tests + { + name: "metadata success", + returnStatusCode: http.StatusOK, + dtype: Metadata, + checks: []check{ + { + name: serializerIncoming, + value: 10, + }, + { + name: remoteMetadata, + value: 10, + }, + { + name: sentMetadataBytes, + valueFunc: greaterThenZero, + }, + { + name: alloyMetadataDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyMetadataSent, + value: 10, + }, + }, + }, + { + name: "metadata failure", + returnStatusCode: http.StatusBadRequest, + dtype: Metadata, + checks: []check{ + { + name: alloyMetadataFailed, + value: 10, + }, + { + name: serializerIncoming, + value: 10, + }, + { + name: failedMetadata, + value: 10, + }, + { + name: alloyMetadataDuration, + valueFunc: greaterThenZero, + }, + }, + }, + { + name: "metadata retry", + returnStatusCode: http.StatusTooManyRequests, + dtype: Metadata, + checks: []check{ + { + name: serializerIncoming, + value: 10, + }, + { + name: retriedMetadata, + // This will be more than 10 since it retries in a loop. + valueFunc: greaterThenZero, + }, + { + name: alloyMetadataDuration, + valueFunc: greaterThenZero, + }, + { + name: alloyMetadataRetried, + valueFunc: greaterThenZero, + }, + { + name: alloyMetadataRetried429, + valueFunc: greaterThenZero, + }, + }, + }, + } + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + runE2eStats(t, test) + }) + } + +} + // TestMetrics is the large end to end testing for the queue based wal. func TestMetrics(t *testing.T) { // Check assumes you are checking for any value that is not 0. @@ -53,7 +152,7 @@ func TestMetrics(t *testing.T) { dtype: Sample, checks: []check{ { - name: filequeueIncoming, + name: serializerIncoming, value: 10, }, { @@ -73,7 +172,7 @@ func TestMetrics(t *testing.T) { valueFunc: greaterThenZero, }, { - name: alloyFileQueueIncoming, + name: alloySerializerIncoming, valueFunc: isReasonableTimeStamp, }, { @@ -100,7 +199,7 @@ func TestMetrics(t *testing.T) { value: 10, }, { - name: filequeueIncoming, + name: serializerIncoming, value: 10, }, { @@ -116,7 +215,7 @@ func TestMetrics(t *testing.T) { valueFunc: greaterThenZero, }, { - name: alloyFileQueueIncoming, + name: alloySerializerIncoming, valueFunc: isReasonableTimeStamp, }, { @@ -131,7 +230,7 @@ func TestMetrics(t *testing.T) { dtype: Sample, checks: []check{ { - name: filequeueIncoming, + name: serializerIncoming, value: 10, }, { @@ -158,7 +257,7 @@ func TestMetrics(t *testing.T) { valueFunc: greaterThenZero, }, { - name: alloyFileQueueIncoming, + name: alloySerializerIncoming, valueFunc: isReasonableTimeStamp, }, { @@ -174,7 +273,7 @@ func TestMetrics(t *testing.T) { dtype: Histogram, checks: []check{ { - name: filequeueIncoming, + name: serializerIncoming, value: 10, }, { @@ -194,7 +293,7 @@ func TestMetrics(t *testing.T) { valueFunc: greaterThenZero, }, { - name: alloyFileQueueIncoming, + name: alloySerializerIncoming, valueFunc: isReasonableTimeStamp, }, { @@ -221,7 +320,7 @@ func TestMetrics(t *testing.T) { value: 10, }, { - name: filequeueIncoming, + name: serializerIncoming, value: 10, }, { @@ -237,7 +336,7 @@ func TestMetrics(t *testing.T) { valueFunc: greaterThenZero, }, { - name: alloyFileQueueIncoming, + name: alloySerializerIncoming, valueFunc: isReasonableTimeStamp, }, { @@ -252,7 +351,7 @@ func TestMetrics(t *testing.T) { dtype: Histogram, checks: []check{ { - name: filequeueIncoming, + name: serializerIncoming, value: 10, }, { @@ -279,7 +378,7 @@ func TestMetrics(t *testing.T) { valueFunc: greaterThenZero, }, { - name: alloyFileQueueIncoming, + name: alloySerializerIncoming, valueFunc: isReasonableTimeStamp, }, { @@ -295,7 +394,7 @@ func TestMetrics(t *testing.T) { dtype: Exemplar, checks: []check{ { - name: filequeueIncoming, + name: serializerIncoming, value: 10, }, { @@ -315,7 +414,7 @@ func TestMetrics(t *testing.T) { valueFunc: greaterThenZero, }, { - name: alloyFileQueueIncoming, + name: alloySerializerIncoming, valueFunc: isReasonableTimeStamp, }, { @@ -342,7 +441,7 @@ func TestMetrics(t *testing.T) { value: 10, }, { - name: filequeueIncoming, + name: serializerIncoming, value: 10, }, { @@ -358,7 +457,7 @@ func TestMetrics(t *testing.T) { valueFunc: greaterThenZero, }, { - name: alloyFileQueueIncoming, + name: alloySerializerIncoming, valueFunc: isReasonableTimeStamp, }, { @@ -373,7 +472,7 @@ func TestMetrics(t *testing.T) { dtype: Exemplar, checks: []check{ { - name: filequeueIncoming, + name: serializerIncoming, value: 10, }, { @@ -400,7 +499,7 @@ func TestMetrics(t *testing.T) { valueFunc: greaterThenZero, }, { - name: alloyFileQueueIncoming, + name: alloySerializerIncoming, valueFunc: isReasonableTimeStamp, }, { @@ -491,6 +590,10 @@ func runE2eStats(t *testing.T, test statsTest) { ex := makeExemplar(index) _, errApp := app.AppendExemplar(0, nil, ex) require.NoError(t, errApp) + case Metadata: + md, lbls := makeMetadata(index) + _, errApp := app.UpdateMetadata(0, lbls, md) + require.NoError(t, errApp) default: require.True(t, false) } @@ -504,6 +607,7 @@ func runE2eStats(t *testing.T, test statsTest) { require.Eventually(t, func() bool { dtos, gatherErr := reg.Gather() require.NoError(t, gatherErr) + // Check if we have some valid metrics. for _, d := range dtos { if getValue(d) > 0 { return true @@ -514,12 +618,15 @@ func runE2eStats(t *testing.T, test statsTest) { metrics := make(map[string]float64) dtos, err := reg.Gather() require.NoError(t, err) + // Get the value of metrics. for _, d := range dtos { metrics[*d.Name] = getValue(d) } // Check for the metrics that matter. for _, valChk := range test.checks { + // These check functions will return the list of metrics with the one checked for deleted. + // Ideally at the end we should only be left with metrics with a value of zero.s if valChk.valueFunc != nil { metrics = checkValueCondition(t, valChk.name, valChk.valueFunc, metrics) } else { @@ -561,8 +668,8 @@ func checkValue(t *testing.T, name string, value float64, metrics map[string]flo func checkValueCondition(t *testing.T, name string, chk func(float64) bool, metrics map[string]float64) map[string]float64 { v, ok := metrics[name] - require.True(t, ok) - require.True(t, chk(v)) + require.Truef(t, ok, "invalid metric name %s", name) + require.Truef(t, chk(v), "false test for metric name %s", name) delete(metrics, name) return metrics } diff --git a/internal/component/prometheus/remote/queue/e2e_test.go b/internal/component/prometheus/remote/queue/e2e_test.go index 58e5597d61..38dc96efb0 100644 --- a/internal/component/prometheus/remote/queue/e2e_test.go +++ b/internal/component/prometheus/remote/queue/e2e_test.go @@ -345,10 +345,12 @@ func newComponent(t *testing.T, l *logging.Logger, url string, exp chan Exports, Registerer: reg, Tracer: nil, }, Arguments{ - TTL: 2 * time.Hour, - MaxSignalsToBatch: 10_000, - BatchFrequency: 1 * time.Second, - Connections: []ConnectionConfig{{ + TTL: 2 * time.Hour, + Serialization: Serialization{ + MaxSignalsToBatch: 10_000, + BatchFrequency: 1 * time.Second, + }, + Endpoints: []EndpointConfig{{ Name: "test", URL: url, Timeout: 20 * time.Second, @@ -358,6 +360,5 @@ func newComponent(t *testing.T, l *logging.Logger, url string, exp chan Exports, FlushFrequency: 1 * time.Second, QueueCount: 1, }}, - ExternalLabels: nil, }) } diff --git a/internal/component/prometheus/remote/queue/endpoint.go b/internal/component/prometheus/remote/queue/endpoint.go index 3f80528aff..643171be25 100644 --- a/internal/component/prometheus/remote/queue/endpoint.go +++ b/internal/component/prometheus/remote/queue/endpoint.go @@ -18,8 +18,6 @@ var _ actor.Worker = (*endpoint)(nil) type endpoint struct { network types.NetworkClient serializer types.Serializer - stat *types.PrometheusStats - metaStats *types.PrometheusStats log log.Logger ttl time.Duration incoming actor.Mailbox[types.DataHandle] @@ -27,12 +25,10 @@ type endpoint struct { self actor.Actor } -func NewEndpoint(client types.NetworkClient, serializer types.Serializer, stats, metatStats *types.PrometheusStats, ttl time.Duration, logger log.Logger) *endpoint { +func NewEndpoint(client types.NetworkClient, serializer types.Serializer, ttl time.Duration, logger log.Logger) *endpoint { return &endpoint{ network: client, serializer: serializer, - stat: stats, - metaStats: metatStats, log: logger, ttl: ttl, incoming: actor.NewMailbox[types.DataHandle](actor.OptCapacity(1)), diff --git a/internal/component/prometheus/remote/queue/network/loop.go b/internal/component/prometheus/remote/queue/network/loop.go index 72fb17ac39..198ff2565b 100644 --- a/internal/component/prometheus/remote/queue/network/loop.go +++ b/internal/component/prometheus/remote/queue/network/loop.go @@ -312,19 +312,16 @@ func createWriteRequest(wr *prompb.WriteRequest, series []*types.TimeSeriesBinar } func createWriteRequestMetadata(l log.Logger, wr *prompb.WriteRequest, series []*types.TimeSeriesBinary, data *proto.Buffer) ([]byte, error) { - if cap(wr.Metadata) < len(series) { - wr.Metadata = make([]prompb.MetricMetadata, len(series)) - } else { - wr.Metadata = wr.Metadata[:len(series)] - } - - for i, ts := range series { + // Metadata is rarely sent so having this being less than optimal is fine. + wr.Metadata = make([]prompb.MetricMetadata, 0) + for _, ts := range series { mt, valid := toMetadata(ts) + // TODO @mattdurham somewhere there is a bug where metadata with no labels are being passed through. if !valid { level.Error(l).Log("msg", "invalid metadata was found", "labels", ts.Labels.String()) continue } - wr.Metadata[i] = mt + wr.Metadata = append(wr.Metadata, mt) } data.Reset() err := data.Marshal(wr) diff --git a/internal/component/prometheus/remote/queue/network/manager.go b/internal/component/prometheus/remote/queue/network/manager.go index 6281536838..543dcf0f1a 100644 --- a/internal/component/prometheus/remote/queue/network/manager.go +++ b/internal/component/prometheus/remote/queue/network/manager.go @@ -38,6 +38,7 @@ func New(cc types.ConnectionConfig, logger log.Logger, seriesStats, metadataStat metaInbox: actor.NewMailbox[*types.TimeSeriesBinary](actor.OptCapacity(1)), configInbox: actor.NewMailbox[types.ConnectionConfig](), stats: seriesStats, + metaStats: metadataStats, cfg: cc, } diff --git a/internal/component/prometheus/remote/queue/serialization/appender.go b/internal/component/prometheus/remote/queue/serialization/appender.go index e2cdd56272..03f8b27a70 100644 --- a/internal/component/prometheus/remote/queue/serialization/appender.go +++ b/internal/component/prometheus/remote/queue/serialization/appender.go @@ -2,6 +2,7 @@ package serialization import ( "context" + "fmt" "time" "github.com/go-kit/log" @@ -99,10 +100,13 @@ func (a *appender) AppendHistogram(ref storage.SeriesRef, l labels.Labels, t int // UpdateMetadata updates metadata. func (a *appender) UpdateMetadata(ref storage.SeriesRef, l labels.Labels, m metadata.Metadata) (_ storage.SeriesRef, _ error) { + if !l.Has("__name__") { + return ref, fmt.Errorf("missing __name__ label for metadata") + } ts := types.GetTimeSeriesFromPool() // We are going to handle converting some strings to hopefully not reused label names. TimeSeriesBinary has a lot of work // to ensure its efficient it makes sense to encode metadata into it. - combinedLabels := l.Copy() + combinedLabels := labels.EmptyLabels() combinedLabels = append(combinedLabels, labels.Label{ Name: types.MetaType, Value: string(m.Type), @@ -115,6 +119,11 @@ func (a *appender) UpdateMetadata(ref storage.SeriesRef, l labels.Labels, m meta Name: types.MetaUnit, Value: m.Unit, }) + // We ONLY want __name__ from labels + combinedLabels = append(combinedLabels, labels.Label{ + Name: "__name__", + Value: l.Get("__name__"), + }) ts.Labels = combinedLabels err := a.s.SendMetadata(a.ctx, ts) return ref, err diff --git a/internal/component/prometheus/remote/queue/serialization/serializer.go b/internal/component/prometheus/remote/queue/serialization/serializer.go index 56307163a0..f6a38c1b8c 100644 --- a/internal/component/prometheus/remote/queue/serialization/serializer.go +++ b/internal/component/prometheus/remote/queue/serialization/serializer.go @@ -2,6 +2,7 @@ package serialization import ( "context" + "fmt" "strconv" "time" @@ -10,6 +11,7 @@ import ( "github.com/grafana/alloy/internal/component/prometheus/remote/queue/types" "github.com/grafana/alloy/internal/runtime/logging/level" "github.com/vladopajic/go-actor/actor" + "go.uber.org/atomic" ) // serializer collects data from multiple appenders in-memory and will periodically flush the data to file.Storage. @@ -30,6 +32,7 @@ type serializer struct { meta []*types.TimeSeriesBinary msgpBuffer []byte stats func(stats types.SerializerStats) + stopped *atomic.Bool } func NewSerializer(cfg types.SerializerConfig, q types.FileStorage, stats func(stats types.SerializerStats), l log.Logger) (types.Serializer, error) { @@ -46,6 +49,7 @@ func NewSerializer(cfg types.SerializerConfig, q types.FileStorage, stats func(s msgpBuffer: make([]byte, 0), lastFlush: time.Now(), stats: stats, + stopped: atomic.NewBool(false), } return s, nil @@ -58,19 +62,29 @@ func (s *serializer) Start() { } func (s *serializer) Stop() { + s.stopped.Store(true) s.queue.Stop() s.self.Stop() } func (s *serializer) SendSeries(ctx context.Context, data *types.TimeSeriesBinary) error { + if s.stopped.Load() { + return fmt.Errorf("serializer is stopped") + } return s.inbox.Send(ctx, data) } func (s *serializer) SendMetadata(ctx context.Context, data *types.TimeSeriesBinary) error { + if s.stopped.Load() { + return fmt.Errorf("serializer is stopped") + } return s.metaInbox.Send(ctx, data) } func (s *serializer) UpdateConfig(ctx context.Context, cfg types.SerializerConfig) error { + if s.stopped.Load() { + return fmt.Errorf("serializer is stopped") + } return s.cfgInbox.Send(ctx, cfg) } @@ -150,7 +164,7 @@ func (s *serializer) flushToDisk(ctx actor.Context) error { types.PutTimeSeriesSliceIntoPool(s.series) types.PutTimeSeriesSliceIntoPool(s.meta) s.series = s.series[:0] - s.meta = s.series[:0] + s.meta = s.meta[:0] }() // This maps strings to index position in a slice. This is doing to reduce the file size of the data. @@ -197,7 +211,6 @@ func (s *serializer) storeStats(err error) { for _, ts := range s.series { if ts.TS > newestTS { newestTS = ts.TS - } } s.stats(types.SerializerStats{ diff --git a/internal/component/prometheus/remote/queue/types.go b/internal/component/prometheus/remote/queue/types.go index 7e424a62d9..36fd3fff2e 100644 --- a/internal/component/prometheus/remote/queue/types.go +++ b/internal/component/prometheus/remote/queue/types.go @@ -6,26 +6,32 @@ import ( "github.com/grafana/alloy/internal/component/prometheus/remote/queue/types" "github.com/grafana/alloy/syntax/alloytypes" + "github.com/prometheus/common/version" "github.com/prometheus/prometheus/storage" ) func defaultArgs() Arguments { return Arguments{ - TTL: 2 * time.Hour, - MaxSignalsToBatch: 10_000, - BatchFrequency: 5 * time.Second, + TTL: 2 * time.Hour, + Serialization: Serialization{ + MaxSignalsToBatch: 10_000, + BatchFrequency: 5 * time.Second, + }, } } type Arguments struct { // TTL is how old a series can be. - TTL time.Duration `alloy:"ttl,attr,optional"` + TTL time.Duration `alloy:"ttl,attr,optional"` + Serialization Serialization `alloy:"serialization,block,optional"` + Endpoints []EndpointConfig `alloy:"endpoint,block"` +} + +type Serialization struct { // The batch size to persist to the file queue. MaxSignalsToBatch int `alloy:"max_signals_to_batch,attr,optional"` // How often to flush to the file queue if BatchSize isn't met. - // TODO @mattdurham this may need to go into a specific block for the serializer. - BatchFrequency time.Duration `alloy:"batch_frequency,attr,optional"` - Connections []ConnectionConfig `alloy:"endpoint,block"` + BatchFrequency time.Duration `alloy:"batch_frequency,attr,optional"` } type Exports struct { @@ -37,8 +43,8 @@ func (rc *Arguments) SetToDefault() { *rc = defaultArgs() } -func defaultCC() ConnectionConfig { - return ConnectionConfig{ +func defaultEndpointConfig() EndpointConfig { + return EndpointConfig{ Timeout: 30 * time.Second, RetryBackoff: 1 * time.Second, MaxRetryBackoffAttempts: 0, @@ -48,12 +54,12 @@ func defaultCC() ConnectionConfig { } } -func (cc *ConnectionConfig) SetToDefault() { - *cc = defaultCC() +func (cc *EndpointConfig) SetToDefault() { + *cc = defaultEndpointConfig() } func (r *Arguments) Validate() error { - for _, conn := range r.Connections { + for _, conn := range r.Endpoints { if conn.BatchCount <= 0 { return fmt.Errorf("batch_count must be greater than 0") } @@ -65,11 +71,8 @@ func (r *Arguments) Validate() error { return nil } -// ConnectionConfig is the alloy specific version of ConnectionConfig. This looks odd, the idea -// -// is that once this code is tested that the bulk of the underlying code will be used elsewhere. -// this means we need a very generic interface for that code, and a specific alloy implementation here. -type ConnectionConfig struct { +// EndpointConfig is the alloy specific version of ConnectionConfig. +type EndpointConfig struct { Name string `alloy:",label"` URL string `alloy:"url,attr"` BasicAuth *BasicAuth `alloy:"basic_auth,block,optional"` @@ -83,16 +86,16 @@ type ConnectionConfig struct { // How long to wait before sending regardless of batch count. FlushFrequency time.Duration `alloy:"flush_frequency,attr,optional"` // How many concurrent queues to have. - QueueCount uint `alloy:"queue_count,attr,optional"` - + QueueCount uint `alloy:"queue_count,attr,optional"` ExternalLabels map[string]string `alloy:"external_labels,attr,optional"` } -func (cc ConnectionConfig) ToNativeType() types.ConnectionConfig { +var UserAgent = fmt.Sprintf("Alloy/%s", version.Version) + +func (cc EndpointConfig) ToNativeType() types.ConnectionConfig { tcc := types.ConnectionConfig{ - URL: cc.URL, - // TODO @mattdurham generate this with build information. - UserAgent: "alloy", + URL: cc.URL, + UserAgent: UserAgent, Timeout: cc.Timeout, RetryBackoff: cc.RetryBackoff, MaxRetryBackoffAttempts: cc.MaxRetryBackoffAttempts, diff --git a/internal/component/prometheus/remote/queue/types/stats.go b/internal/component/prometheus/remote/queue/types/stats.go index f68600076e..7307f2c926 100644 --- a/internal/component/prometheus/remote/queue/types/stats.go +++ b/internal/component/prometheus/remote/queue/types/stats.go @@ -182,21 +182,26 @@ func NewStats(namespace, subsystem string, registry prometheus.Registerer) *Prom return s } -func (s *PrometheusStats) BackwardsCompatibility(registry prometheus.Registerer) { +func (s *PrometheusStats) SeriesBackwardsCompatibility(registry prometheus.Registerer) { registry.MustRegister( s.RemoteStorageDuration, s.RemoteStorageInTimestamp, s.RemoteStorageOutTimestamp, s.SamplesTotal, s.HistogramsTotal, - s.MetadataTotal, s.FailedSamplesTotal, s.FailedHistogramsTotal, - s.FailedMetadataTotal, s.RetriedSamplesTotal, s.RetriedHistogramsTotal, - s.RetriedMetadataTotal, s.SentBytesTotal, + ) +} + +func (s *PrometheusStats) MetaBackwardsCompatibility(registry prometheus.Registerer) { + registry.MustRegister( + s.MetadataTotal, + s.FailedMetadataTotal, + s.RetriedMetadataTotal, s.MetadataBytesTotal, ) } @@ -212,6 +217,7 @@ func (s *PrometheusStats) UpdateNetwork(stats NetworkStats) { // The newest timestamp is no always sent. if stats.NewestTimestamp != 0 { s.RemoteStorageOutTimestamp.Set(float64(stats.NewestTimestamp)) + s.NetworkNewestOutTimeStampSeconds.Set(float64(stats.NewestTimestamp)) } s.SamplesTotal.Add(float64(stats.Series.SeriesSent)) @@ -230,13 +236,15 @@ func (s *PrometheusStats) UpdateNetwork(stats NetworkStats) { s.SentBytesTotal.Add(float64(stats.SeriesBytes)) } -func (s *PrometheusStats) UpdateFileQueue(stats SerializerStats) { +func (s *PrometheusStats) UpdateSerializer(stats SerializerStats) { s.SerializerInSeries.Add(float64(stats.SeriesStored)) + s.SerializerInSeries.Add(float64(stats.MetadataStored)) s.SerializerErrors.Add(float64(stats.Errors)) if stats.NewestTimestamp != 0 { s.SerializerNewestInTimeStampSeconds.Set(float64(stats.NewestTimestamp)) s.RemoteStorageInTimestamp.Set(float64(stats.NewestTimestamp)) } + } type NetworkStats struct { diff --git a/internal/component/pyroscope/scrape/manager_test.go b/internal/component/pyroscope/scrape/manager_test.go index c72cc256e7..e1249641a0 100644 --- a/internal/component/pyroscope/scrape/manager_test.go +++ b/internal/component/pyroscope/scrape/manager_test.go @@ -66,9 +66,8 @@ func TestManager(t *testing.T) { require.Equal(t, 1*time.Second, ts.config.ScrapeInterval) } - targetSetsChan <- map[string][]*targetgroup.Group{"group1": {}, "group2": {}} + targetSetsChan <- map[string][]*targetgroup.Group{} - require.Eventually(t, func() bool { - return len(m.TargetsAll()["group2"]) == 0 && len(m.TargetsAll()["group1"]) == 0 - }, time.Second, 10*time.Millisecond) + time.Sleep(5 * time.Second) + println(m.TargetsAll()) } From ed64bc36c8a31b5f0c45229cbd8316e88d4bd1eb Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 8 Oct 2024 08:48:15 -0400 Subject: [PATCH 19/44] Update docs. --- .../prometheus/prometheus.remote.queue.md | 54 ++++++++++++++++--- .../prometheus/remote/queue/types/stats.go | 2 + 2 files changed, 49 insertions(+), 7 deletions(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index db93f19597..151ad37917 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -4,6 +4,9 @@ description: Learn about prometheus.remote.queue title: prometheus.remote.queue --- + +Experimental + # prometheus.remote.queue `prometheus.remote.queue` collects metrics sent from other components into a @@ -13,13 +16,16 @@ user-supplied endpoints. Metrics are sent over the network using the You can specify multiple `prometheus.remote.queue` components by giving them different labels. -[remote_write-spec]: https://docs.google.com/document/d/1LPhVRSFkGNSuU1fBd81ulhsCPR4hkSZyyBj1SZ8fWOM/edit +Everything here should be considered extremely experimental and highly subject to change. +[emote_write-spec]: https://docs.google.com/document/d/1LPhVRSFkGNSuU1fBd81ulhsCPR4hkSZyyBj1SZ8fWOM/edit + + ## Usage ```alloy prometheus.remote.queue "LABEL" { - endpoint { + endpoint "default "{ url = REMOTE_WRITE_URL ... @@ -36,9 +42,6 @@ The following arguments are supported: Name | Type | Description | Default | Required ---- | ---- | ----------- | ------- | -------- `ttl` | `time` | `duration` | How long the timestamp of a signal is valid for, before the signal is discarded. | `2h` | no -`max_signals_to_batch` | `uint` | The maximum number of signals before they are batched to disk. | `10,000` | no -`batch_frequency` | `duration` | How often to batch signals to disk if `max_signals_to_batch` is not reached. | no - ## Blocks @@ -47,6 +50,7 @@ The following blocks are supported inside the definition of Hierarchy | Block | Description | Required --------- | ----- | ----------- | -------- +serialization | [serialization][] | Configuration for serializing and writing to disk | no endpoint | [endpoint][] | Location to send metrics to. | no endpoint > basic_auth | [basic_auth][] | Configure basic_auth for authenticating to the endpoint. | no @@ -56,11 +60,26 @@ basic_auth` refers to a `basic_auth` block defined inside an [endpoint]: #endpoint-block [basic_auth]: #basic_auth-block +[serialization]: #serialization-block + +### serialization block + +The `serialization` block describes how often and at what limits to write to disk. Serialization settings +are shared for each `endpoint.` + +The following arguments are supported: + +Name | Type | Description | Default | Required +---- | ---- | ----------- | ------- | -------- +`max_signals_to_batch` | `uint` | The maximum number of signals before they are batched to disk. | `10,000` | no +`batch_frequency` | `duration` | How often to batch signals to disk if `max_signals_to_batch` is not reached. | no + ### endpoint block The `endpoint` block describes a single location to send metrics to. Multiple -`endpoint` blocks can be provided to send metrics to multiple locations. +`endpoint` blocks can be provided to send metrics to multiple locations. Each +`endpoint` will have it's own WAL folder The following arguments are supported: @@ -156,7 +175,28 @@ They generally behave the same, but there are likely edge cases where they diffe * `prometheus_remote_storage_exemplars_in_total` (counter): Exemplars read into remote storage. -TODO document new metrics. +Metrics that are new to `prometheus.remote.write`. These are highly subject to change. + +* `alloy_queue_series_serializer_incoming_signals` (counter): Total number of series written to serialization. +* `alloy_queue_metadata_serializer_incoming_signals` (counter): Total number of metadata written to serialization. +* `alloy_queue_series_serializer_incoming_timestamp_seconds` (gauge): Highest timestamp of incoming series. +* `alloy_queue_series_serializer_errors` (gauge): Number of errors for series written to serializer. +* `alloy_queue_metadata_serializer_errors` (gauge): Number of errors for metadata written to serializer. +* `alloy_queue_series_network_timestamp_seconds` (gauge): Highest timestamp written to an endpoint. +* `alloy_queue_series_network_sent` (counter): Number of series sent successfully. +* `alloy_queue_metadata_network_sent` (counter): Number of metadata sent successful. +* `alloy_queue_network_series_failed` (counter): Number of series failed. +* `alloy_queue_network_metadata_failed` (counter): Number of metadata failed. +* `alloy_queue_network_series_retried` (counter): Number of series retried due to network issues. +* `alloy_queue_network_metadata_retried` (counter): Number of metadata retried due to network issues. +* `alloy_queue_network_series_retried_429` (counter): Number of series retried due to status code 429. +* `alloy_queue_network_metadata_retried_429` (counter): Number of metadata retried due to status code 429. +* `alloy_queue_network_series_retried_5xx` (counter): Number of series retried due to status code 5xx. +* `alloy_queue_network_metadata_retried_5xx` (counter): Number of metadata retried due to status code 5xx. +* `alloy_queue_network_series_network_duration_seconds` (histogram): Duration writing series to endpoint. +* `alloy_queue_network_metadata_network_duration_seconds` (histogram): Duration writing metadata to endpoint. +* `alloy_queue_network_series_network_errors` (counter): Number of errors writing series to network. +* `alloy_queue_network_metadata_network_errors` (counter): Number of errors writing metadata to network. ## Examples diff --git a/internal/component/prometheus/remote/queue/types/stats.go b/internal/component/prometheus/remote/queue/types/stats.go index 7307f2c926..732b6255aa 100644 --- a/internal/component/prometheus/remote/queue/types/stats.go +++ b/internal/component/prometheus/remote/queue/types/stats.go @@ -6,6 +6,8 @@ import ( "github.com/prometheus/client_golang/prometheus" ) +// TODO @mattdurham separate this into more manageable chunks, and likely 3 stats series: series, metadata and new ones. + type SerializerStats struct { SeriesStored int MetadataStored int From 98bc8876270ce5dfac590c10070259f9889d1703 Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 8 Oct 2024 09:00:20 -0400 Subject: [PATCH 20/44] Update docs. --- docs/sources/reference/compatibility/_index.md | 1 + .../prometheus/prometheus.remote.queue.md | 14 ++++++++++++++ 2 files changed, 15 insertions(+) diff --git a/docs/sources/reference/compatibility/_index.md b/docs/sources/reference/compatibility/_index.md index 6549abff99..377cd647b8 100644 --- a/docs/sources/reference/compatibility/_index.md +++ b/docs/sources/reference/compatibility/_index.md @@ -174,6 +174,7 @@ The following components, grouped by namespace, _export_ Prometheus `MetricsRece {{< collapse title="prometheus" >}} - [prometheus.relabel](../components/prometheus/prometheus.relabel) +- [prometheus.remote.queue](../components/prometheus/prometheus.remote.queue) - [prometheus.remote_write](../components/prometheus/prometheus.remote_write) {{< /collapse >}} diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index 151ad37917..a32afd1be2 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -272,3 +272,17 @@ Refer to the linked documentation for more details. [WAL block]: #wal-block [Stop]: ../../../../set-up/run/ [run]: ../../../cli/run/ + + +## Compatible components + +`prometheus.remote.queue` has exports that can be consumed by the following components: + +- Components that consume [Prometheus `MetricsReceiver`](../../../compatibility/#prometheus-metricsreceiver-consumers) + +{{< admonition type="note" >}} +Connecting some components may not be sensible or components may require further configuration to make the connection work correctly. +Refer to the linked documentation for more details. +{{< /admonition >}} + + \ No newline at end of file From 2e32ce686fc65ce490827679050747b62fe45f54 Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 8 Oct 2024 15:12:54 -0400 Subject: [PATCH 21/44] Fix race conditions in unit tests. --- .../prometheus/remote/queue/e2e_stats_test.go | 14 +++ .../prometheus/remote/queue/e2e_test.go | 118 +++++++++++++----- 2 files changed, 103 insertions(+), 29 deletions(-) diff --git a/internal/component/prometheus/remote/queue/e2e_stats_test.go b/internal/component/prometheus/remote/queue/e2e_stats_test.go index 38cb36dd11..81cd26c75d 100644 --- a/internal/component/prometheus/remote/queue/e2e_stats_test.go +++ b/internal/component/prometheus/remote/queue/e2e_stats_test.go @@ -47,6 +47,8 @@ const alloyMetadataFailed = "alloy_queue_metadata_network_failed" const alloyMetadataRetried429 = "alloy_queue_metadata_network_retried_429" const alloyMetadataRetried = "alloy_queue_metadata_network_retried" +const alloyNetworkTimestamp = "alloy_queue_series_network_timestamp_seconds" + // TestMetadata is the large end to end testing for the queue based wal, specifically for metadata. func TestMetadata(t *testing.T) { // Check assumes you are checking for any value that is not 0. @@ -187,6 +189,10 @@ func TestMetrics(t *testing.T) { name: inTimestamp, valueFunc: isReasonableTimeStamp, }, + { + name: alloyNetworkTimestamp, + valueFunc: greaterThenZero, + }, }, }, { @@ -308,6 +314,10 @@ func TestMetrics(t *testing.T) { name: inTimestamp, valueFunc: isReasonableTimeStamp, }, + { + name: alloyNetworkTimestamp, + valueFunc: greaterThenZero, + }, }, }, { @@ -429,6 +439,10 @@ func TestMetrics(t *testing.T) { name: inTimestamp, valueFunc: isReasonableTimeStamp, }, + { + name: alloyNetworkTimestamp, + valueFunc: greaterThenZero, + }, }, }, { diff --git a/internal/component/prometheus/remote/queue/e2e_test.go b/internal/component/prometheus/remote/queue/e2e_test.go index 38dc96efb0..9704ff7f6d 100644 --- a/internal/component/prometheus/remote/queue/e2e_test.go +++ b/internal/component/prometheus/remote/queue/e2e_test.go @@ -32,8 +32,8 @@ func TestE2E(t *testing.T) { type e2eTest struct { name string maker func(index int, app storage.Appender) (float64, labels.Labels) - tester func(samples []prompb.TimeSeries) - testMeta func(samples []prompb.MetricMetadata) + tester func(samples *safeSlice[prompb.TimeSeries]) + testMeta func(samples *safeSlice[prompb.MetricMetadata]) } tests := []e2eTest{ { @@ -44,9 +44,10 @@ func TestE2E(t *testing.T) { require.NoError(t, errApp) return v, lbls }, - tester: func(samples []prompb.TimeSeries) { + tester: func(samples *safeSlice[prompb.TimeSeries]) { t.Helper() - for _, s := range samples { + for i := 0; i < samples.Len(); i++ { + s := samples.Get(i) require.True(t, len(s.Samples) == 1) require.True(t, s.Samples[0].Timestamp > 0) require.True(t, s.Samples[0].Value > 0) @@ -64,8 +65,9 @@ func TestE2E(t *testing.T) { require.NoError(t, errApp) return 0, lbls }, - testMeta: func(samples []prompb.MetricMetadata) { - for _, s := range samples { + testMeta: func(samples *safeSlice[prompb.MetricMetadata]) { + for i := 0; i < samples.Len(); i++ { + s := samples.Get(i) require.True(t, s.GetUnit() == "seconds") require.True(t, s.Help == "metadata help") require.True(t, s.Unit == "seconds") @@ -83,9 +85,10 @@ func TestE2E(t *testing.T) { require.NoError(t, errApp) return h.Sum, lbls }, - tester: func(samples []prompb.TimeSeries) { + tester: func(samples *safeSlice[prompb.TimeSeries]) { t.Helper() - for _, s := range samples { + for i := 0; i < samples.Len(); i++ { + s := samples.Get(i) require.True(t, len(s.Samples) == 1) require.True(t, s.Samples[0].Timestamp > 0) require.True(t, s.Samples[0].Value == 0) @@ -102,9 +105,10 @@ func TestE2E(t *testing.T) { require.NoError(t, errApp) return h.Sum, lbls }, - tester: func(samples []prompb.TimeSeries) { + tester: func(samples *safeSlice[prompb.TimeSeries]) { t.Helper() - for _, s := range samples { + for i := 0; i < samples.Len(); i++ { + s := samples.Get(i) require.True(t, len(s.Samples) == 1) require.True(t, s.Samples[0].Timestamp > 0) require.True(t, s.Samples[0].Value == 0) @@ -122,23 +126,23 @@ func TestE2E(t *testing.T) { } const ( - iterations = 100 + iterations = 10 items = 10_000 ) -func runTest(t *testing.T, add func(index int, appendable storage.Appender) (float64, labels.Labels), test func(samples []prompb.TimeSeries), metaTest func(meta []prompb.MetricMetadata)) { +func runTest(t *testing.T, add func(index int, appendable storage.Appender) (float64, labels.Labels), test func(samples *safeSlice[prompb.TimeSeries]), metaTest func(meta *safeSlice[prompb.MetricMetadata])) { l := util.TestAlloyLogger(t) done := make(chan struct{}) var series atomic.Int32 var meta atomic.Int32 - samples := make([]prompb.TimeSeries, 0) - metaSamples := make([]prompb.MetricMetadata, 0) + samples := newSafeSlice[prompb.TimeSeries]() + metaSamples := newSafeSlice[prompb.MetricMetadata]() srv := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { newSamples, newMetadata := handlePost(t, w, r) series.Add(int32(len(newSamples))) meta.Add(int32(len(newMetadata))) - samples = append(samples, newSamples...) - metaSamples = append(metaSamples, newMetadata...) + samples.AddSlice(newSamples) + metaSamples.AddSlice(newMetadata) if series.Load() == iterations*items { done <- struct{}{} } @@ -158,40 +162,43 @@ func runTest(t *testing.T, add func(index int, appendable storage.Appender) (flo // Wait for export to spin up. exp := <-expCh - index := 0 - results := make(map[float64]labels.Labels) - mut := sync.Mutex{} + index := atomic.NewInt64(0) + results := &safeMap{ + results: make(map[float64]labels.Labels), + } for i := 0; i < iterations; i++ { go func() { app := exp.Receiver.Appender(ctx) for j := 0; j < items; j++ { - index++ - v, lbl := add(index, app) - mut.Lock() - results[v] = lbl - mut.Unlock() + val := index.Add(1) + v, lbl := add(int(val), app) + results.Add(v, lbl) } require.NoError(t, app.Commit()) }() } // This is a weird use case to handle eventually. - tm := time.NewTimer(15 * time.Second) + // With race turned on this can take a long time. + tm := time.NewTimer(20 * time.Second) select { case <-done: case <-tm.C: + require.Truef(t, false, "failed to collect signals in the appropriate time") } cancel() - for _, s := range samples { + + for i := 0; i < samples.Len(); i++ { + s := samples.Get(i) if len(s.Histograms) == 1 { - lbls, ok := results[s.Histograms[0].Sum] + lbls, ok := results.Get(s.Histograms[0].Sum) require.True(t, ok) for i, sLbl := range s.Labels { require.True(t, lbls[i].Name == sLbl.Name) require.True(t, lbls[i].Value == sLbl.Value) } } else { - lbls, ok := results[s.Samples[0].Value] + lbls, ok := results.Get(s.Samples[0].Value) require.True(t, ok) for i, sLbl := range s.Labels { require.True(t, lbls[i].Name == sLbl.Name) @@ -204,7 +211,9 @@ func runTest(t *testing.T, add func(index int, appendable storage.Appender) (flo } else { metaTest(metaSamples) } - require.Truef(t, types.OutStandingTimeSeriesBinary.Load() == 0, "there are %d time series not collected", types.OutStandingTimeSeriesBinary.Load()) + require.Eventuallyf(t, func() bool { + return types.OutStandingTimeSeriesBinary.Load() == 0 + }, 2*time.Second, 100*time.Millisecond, "there are %d time series not collected", types.OutStandingTimeSeriesBinary.Load()) } func handlePost(t *testing.T, _ http.ResponseWriter, r *http.Request) ([]prompb.TimeSeries, []prompb.MetricMetadata) { @@ -362,3 +371,54 @@ func newComponent(t *testing.T, l *logging.Logger, url string, exp chan Exports, }}, }) } + +func newSafeSlice[T any]() *safeSlice[T] { + return &safeSlice[T]{slice: make([]T, 0)} +} + +type safeSlice[T any] struct { + slice []T + mut sync.Mutex +} + +func (s *safeSlice[T]) Add(v T) { + s.mut.Lock() + defer s.mut.Unlock() + s.slice = append(s.slice, v) +} + +func (s *safeSlice[T]) AddSlice(v []T) { + s.mut.Lock() + defer s.mut.Unlock() + s.slice = append(s.slice, v...) +} + +func (s *safeSlice[T]) Len() int { + s.mut.Lock() + defer s.mut.Unlock() + return len(s.slice) +} + +func (s *safeSlice[T]) Get(i int) T { + s.mut.Lock() + defer s.mut.Unlock() + return s.slice[i] +} + +type safeMap struct { + mut sync.Mutex + results map[float64]labels.Labels +} + +func (s *safeMap) Add(v float64, ls labels.Labels) { + s.mut.Lock() + defer s.mut.Unlock() + s.results[v] = ls +} + +func (s *safeMap) Get(v float64) (labels.Labels, bool) { + s.mut.Lock() + defer s.mut.Unlock() + res, ok := s.results[v] + return res, ok +} From e1aaa9f4c8e30e3c34a7e24b8048dff4cf62cc25 Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 8 Oct 2024 15:21:25 -0400 Subject: [PATCH 22/44] Tweaking unit tests. --- .../component/prometheus/remote/queue/network/manager_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/component/prometheus/remote/queue/network/manager_test.go b/internal/component/prometheus/remote/queue/network/manager_test.go index 47001fda9f..be45b4aa5a 100644 --- a/internal/component/prometheus/remote/queue/network/manager_test.go +++ b/internal/component/prometheus/remote/queue/network/manager_test.go @@ -96,7 +96,7 @@ func TestUpdatingConfig(t *testing.T) { } require.Eventuallyf(t, func() bool { return recordsFound.Load() == 1_000 - }, 10*time.Second, 1*time.Second, "record count should be 1000 but is %d", recordsFound.Load()) + }, 15*time.Second, 1*time.Second, "record count should be 1000 but is %d", recordsFound.Load()) require.Truef(t, lastBatchSize.Load() == 100, "batch_count should be 100 but is %d", lastBatchSize.Load()) } From 38b15a1cdb21e5418ceb1e7086ebd8f022fe292e Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 8 Oct 2024 15:29:09 -0400 Subject: [PATCH 23/44] lower threshold more. --- .../prometheus/remote/queue/network/manager_test.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/internal/component/prometheus/remote/queue/network/manager_test.go b/internal/component/prometheus/remote/queue/network/manager_test.go index be45b4aa5a..f7eb245ada 100644 --- a/internal/component/prometheus/remote/queue/network/manager_test.go +++ b/internal/component/prometheus/remote/queue/network/manager_test.go @@ -84,21 +84,21 @@ func TestUpdatingConfig(t *testing.T) { cc2 := types.ConnectionConfig{ URL: svr.URL, Timeout: 5 * time.Second, - BatchCount: 100, + BatchCount: 20, FlushFrequency: 1 * time.Second, Connections: 4, } err = wr.UpdateConfig(context.Background(), cc2) require.NoError(t, err) - for i := 0; i < 1_000; i++ { + for i := 0; i < 100; i++ { send(t, wr, ctx) } require.Eventuallyf(t, func() bool { - return recordsFound.Load() == 1_000 - }, 15*time.Second, 1*time.Second, "record count should be 1000 but is %d", recordsFound.Load()) + return recordsFound.Load() == 100 + }, 15*time.Second, 1*time.Second, "record count should be 100 but is %d", recordsFound.Load()) - require.Truef(t, lastBatchSize.Load() == 100, "batch_count should be 100 but is %d", lastBatchSize.Load()) + require.Truef(t, lastBatchSize.Load() == 20, "batch_count should be 20 but is %d", lastBatchSize.Load()) } func TestRetry(t *testing.T) { From 6f9a820cfce09a24de12f1aeca1c9e8b3e846f02 Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 8 Oct 2024 15:43:51 -0400 Subject: [PATCH 24/44] lower threshold more. --- .../prometheus/remote/queue/network/manager_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/internal/component/prometheus/remote/queue/network/manager_test.go b/internal/component/prometheus/remote/queue/network/manager_test.go index f7eb245ada..ac153ff592 100644 --- a/internal/component/prometheus/remote/queue/network/manager_test.go +++ b/internal/component/prometheus/remote/queue/network/manager_test.go @@ -83,10 +83,10 @@ func TestUpdatingConfig(t *testing.T) { cc2 := types.ConnectionConfig{ URL: svr.URL, - Timeout: 5 * time.Second, + Timeout: 1 * time.Second, BatchCount: 20, FlushFrequency: 1 * time.Second, - Connections: 4, + Connections: 1, } err = wr.UpdateConfig(context.Background(), cc2) @@ -96,7 +96,7 @@ func TestUpdatingConfig(t *testing.T) { } require.Eventuallyf(t, func() bool { return recordsFound.Load() == 100 - }, 15*time.Second, 1*time.Second, "record count should be 100 but is %d", recordsFound.Load()) + }, 20*time.Second, 1*time.Second, "record count should be 100 but is %d", recordsFound.Load()) require.Truef(t, lastBatchSize.Load() == 20, "batch_count should be 20 but is %d", lastBatchSize.Load()) } From c78ea1d5b7701a3af56f0803cb5784486e2cd9ec Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 8 Oct 2024 16:43:50 -0400 Subject: [PATCH 25/44] Fix deadlock in manager tests. --- .../remote/queue/network/manager.go | 48 +++++++++++++++---- .../remote/queue/network/manager_test.go | 19 ++++---- .../prometheus/remote/queue/types/network.go | 2 + 3 files changed, 52 insertions(+), 17 deletions(-) diff --git a/internal/component/prometheus/remote/queue/network/manager.go b/internal/component/prometheus/remote/queue/network/manager.go index 543dcf0f1a..fbe6e430e1 100644 --- a/internal/component/prometheus/remote/queue/network/manager.go +++ b/internal/component/prometheus/remote/queue/network/manager.go @@ -2,11 +2,9 @@ package network import ( "context" - - "github.com/grafana/alloy/internal/runtime/logging/level" - "github.com/go-kit/log" "github.com/grafana/alloy/internal/component/prometheus/remote/queue/types" + "github.com/grafana/alloy/internal/runtime/logging/level" "github.com/vladopajic/go-actor/actor" ) @@ -17,13 +15,19 @@ type manager struct { logger log.Logger inbox actor.Mailbox[*types.TimeSeriesBinary] metaInbox actor.Mailbox[*types.TimeSeriesBinary] - configInbox actor.Mailbox[types.ConnectionConfig] + configInbox actor.Mailbox[configCallback] self actor.Actor cfg types.ConnectionConfig stats func(types.NetworkStats) metaStats func(types.NetworkStats) } +// configCallback allows the config to be synchronous. +type configCallback struct { + cc types.ConnectionConfig + ch chan struct{} +} + var _ types.NetworkClient = (*manager)(nil) var _ actor.Worker = (*manager)(nil) @@ -36,7 +40,7 @@ func New(cc types.ConnectionConfig, logger log.Logger, seriesStats, metadataStat // it will stop the filequeue from feeding more. inbox: actor.NewMailbox[*types.TimeSeriesBinary](actor.OptCapacity(1)), metaInbox: actor.NewMailbox[*types.TimeSeriesBinary](actor.OptCapacity(1)), - configInbox: actor.NewMailbox[types.ConnectionConfig](), + configInbox: actor.NewMailbox[configCallback](), stats: seriesStats, metaStats: metadataStats, cfg: cc, @@ -72,7 +76,17 @@ func (s *manager) SendMetadata(ctx context.Context, data *types.TimeSeriesBinary } func (s *manager) UpdateConfig(ctx context.Context, cc types.ConnectionConfig) error { - return s.configInbox.Send(ctx, cc) + ch := make(chan struct{}) + defer close(ch) + err := s.configInbox.Send(ctx, configCallback{ + cc: cc, + ch: ch, + }) + if err != nil { + return err + } + <-ch + return nil } func (s *manager) DoWork(ctx actor.Context) actor.WorkerStatus { @@ -80,24 +94,31 @@ func (s *manager) DoWork(ctx actor.Context) actor.WorkerStatus { select { case cfg, ok := <-s.configInbox.ReceiveC(): if !ok { + level.Debug(s.logger).Log("msg", "config inbox closed") return actor.WorkerEnd } - s.updateConfig(cfg) + s.updateConfig(cfg.cc) + // Notify the caller we have applied the config. + cfg.ch <- struct{}{} return actor.WorkerContinue default: } + + // main work queue. select { case <-ctx.Done(): s.Stop() return actor.WorkerEnd case ts, ok := <-s.inbox.ReceiveC(): if !ok { + level.Debug(s.logger).Log("msg", "series inbox closed") return actor.WorkerEnd } s.queue(ctx, ts) return actor.WorkerContinue case ts, ok := <-s.metaInbox.ReceiveC(): if !ok { + level.Debug(s.logger).Log("msg", "meta inbox closed") return actor.WorkerEnd } err := s.metadata.seriesMbx.Send(ctx, ts) @@ -105,6 +126,16 @@ func (s *manager) DoWork(ctx actor.Context) actor.WorkerStatus { level.Error(s.logger).Log("msg", "failed to send to metadata loop", "err", err) } return actor.WorkerContinue + // We need to also check the config here, else its possible this will deadlock. + case cfg, ok := <-s.configInbox.ReceiveC(): + if !ok { + level.Debug(s.logger).Log("msg", "config inbox closed") + return actor.WorkerEnd + } + s.updateConfig(cfg.cc) + // Notify the caller we have applied the config. + cfg.ch <- struct{}{} + return actor.WorkerContinue } } @@ -124,13 +155,14 @@ func (s *manager) updateConfig(cc types.ConnectionConfig) { for i := uint(0); i < s.cfg.Connections; i++ { l := newLoop(cc, false, s.logger, s.stats) l.self = actor.New(l) - s.loops = append(s.loops, l) } s.metadata = newLoop(cc, true, s.logger, s.metaStats) s.metadata.self = actor.New(s.metadata) + level.Debug(s.logger).Log("msg", "starting loops") s.startLoops() + level.Debug(s.logger).Log("msg", "loops started") } func (s *manager) Stop() { diff --git a/internal/component/prometheus/remote/queue/network/manager_test.go b/internal/component/prometheus/remote/queue/network/manager_test.go index ac153ff592..f8c1d8e950 100644 --- a/internal/component/prometheus/remote/queue/network/manager_test.go +++ b/internal/component/prometheus/remote/queue/network/manager_test.go @@ -2,6 +2,7 @@ package network import ( "context" + "github.com/grafana/alloy/internal/util" "io" "math/rand" "net/http" @@ -64,20 +65,19 @@ func TestUpdatingConfig(t *testing.T) { })) defer svr.Close() - ctx := context.Background() - ctx, cncl := context.WithCancel(ctx) - defer cncl() cc := types.ConnectionConfig{ URL: svr.URL, Timeout: 1 * time.Second, BatchCount: 10, - FlushFrequency: 1 * time.Second, - Connections: 4, + FlushFrequency: 5 * time.Second, + Connections: 1, } - logger := log.NewNopLogger() + logger := util.TestAlloyLogger(t) + wr, err := New(cc, logger, func(s types.NetworkStats) {}, func(s types.NetworkStats) {}) + require.NoError(t, err) wr.Start() defer wr.Stop() @@ -85,12 +85,13 @@ func TestUpdatingConfig(t *testing.T) { URL: svr.URL, Timeout: 1 * time.Second, BatchCount: 20, - FlushFrequency: 1 * time.Second, + FlushFrequency: 5 * time.Second, Connections: 1, } - - err = wr.UpdateConfig(context.Background(), cc2) + ctx := context.Background() + err = wr.UpdateConfig(ctx, cc2) require.NoError(t, err) + time.Sleep(1 * time.Second) for i := 0; i < 100; i++ { send(t, wr, ctx) } diff --git a/internal/component/prometheus/remote/queue/types/network.go b/internal/component/prometheus/remote/queue/types/network.go index e83a816f87..3fc767af0d 100644 --- a/internal/component/prometheus/remote/queue/types/network.go +++ b/internal/component/prometheus/remote/queue/types/network.go @@ -11,6 +11,8 @@ type NetworkClient interface { Stop() SendSeries(ctx context.Context, d *TimeSeriesBinary) error SendMetadata(ctx context.Context, d *TimeSeriesBinary) error + // UpdateConfig is a synchronous call and will only return once the config + // is applied or an error occurs. UpdateConfig(ctx context.Context, cfg ConnectionConfig) error } type ConnectionConfig struct { From c6239d1c51ba9cd1e5b9807e883ff689845219b5 Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 8 Oct 2024 16:54:03 -0400 Subject: [PATCH 26/44] rollback to previous --- internal/component/pyroscope/scrape/manager_test.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/internal/component/pyroscope/scrape/manager_test.go b/internal/component/pyroscope/scrape/manager_test.go index e1249641a0..c72cc256e7 100644 --- a/internal/component/pyroscope/scrape/manager_test.go +++ b/internal/component/pyroscope/scrape/manager_test.go @@ -66,8 +66,9 @@ func TestManager(t *testing.T) { require.Equal(t, 1*time.Second, ts.config.ScrapeInterval) } - targetSetsChan <- map[string][]*targetgroup.Group{} + targetSetsChan <- map[string][]*targetgroup.Group{"group1": {}, "group2": {}} - time.Sleep(5 * time.Second) - println(m.TargetsAll()) + require.Eventually(t, func() bool { + return len(m.TargetsAll()["group2"]) == 0 && len(m.TargetsAll()["group1"]) == 0 + }, time.Second, 10*time.Millisecond) } From 3bb04d4803933fdd1a3d1df5549ae62f750967d8 Mon Sep 17 00:00:00 2001 From: mattdurham Date: Wed, 9 Oct 2024 08:27:05 -0400 Subject: [PATCH 27/44] Update docs/sources/reference/components/prometheus/prometheus.remote.queue.md Co-authored-by: Clayton Cornell <131809008+clayton-cornell@users.noreply.github.com> --- .../reference/components/prometheus/prometheus.remote.queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index a32afd1be2..c3594870aa 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -16,7 +16,7 @@ user-supplied endpoints. Metrics are sent over the network using the You can specify multiple `prometheus.remote.queue` components by giving them different labels. -Everything here should be considered extremely experimental and highly subject to change. +You should consider everything here extremely experimental and highly subject to change. [emote_write-spec]: https://docs.google.com/document/d/1LPhVRSFkGNSuU1fBd81ulhsCPR4hkSZyyBj1SZ8fWOM/edit From 67135541c07a7aa4e1b9722cb67fedd077dedee6 Mon Sep 17 00:00:00 2001 From: mattdurham Date: Wed, 9 Oct 2024 08:27:10 -0400 Subject: [PATCH 28/44] Update docs/sources/reference/components/prometheus/prometheus.remote.queue.md Co-authored-by: Clayton Cornell <131809008+clayton-cornell@users.noreply.github.com> --- .../reference/components/prometheus/prometheus.remote.queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index c3594870aa..004a641aea 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -41,7 +41,7 @@ The following arguments are supported: Name | Type | Description | Default | Required ---- | ---- | ----------- | ------- | -------- -`ttl` | `time` | `duration` | How long the timestamp of a signal is valid for, before the signal is discarded. | `2h` | no +`ttl` | `time` | `duration` | How long the timestamp of a signal is valid before the signal is discarded. | `2h` | no ## Blocks From d5568d902149ce15e2fd03d698ee67d620b85930 Mon Sep 17 00:00:00 2001 From: mattdurham Date: Wed, 9 Oct 2024 08:27:18 -0400 Subject: [PATCH 29/44] Update docs/sources/reference/components/prometheus/prometheus.remote.queue.md Co-authored-by: Clayton Cornell <131809008+clayton-cornell@users.noreply.github.com> --- .../reference/components/prometheus/prometheus.remote.queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index 004a641aea..e8f8e643e6 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -79,7 +79,7 @@ Name | Type | Description | Default | Required The `endpoint` block describes a single location to send metrics to. Multiple `endpoint` blocks can be provided to send metrics to multiple locations. Each -`endpoint` will have it's own WAL folder +`endpoint` will have its own WAL folder The following arguments are supported: From d8cd01232d0fed8e3d0e1bd2207d0b5732d0b8a8 Mon Sep 17 00:00:00 2001 From: mattdurham Date: Wed, 9 Oct 2024 08:27:25 -0400 Subject: [PATCH 30/44] Update docs/sources/reference/components/prometheus/prometheus.remote.queue.md Co-authored-by: Clayton Cornell <131809008+clayton-cornell@users.noreply.github.com> --- .../reference/components/prometheus/prometheus.remote.queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index e8f8e643e6..300e35c83d 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -184,7 +184,7 @@ Metrics that are new to `prometheus.remote.write`. These are highly subject to c * `alloy_queue_metadata_serializer_errors` (gauge): Number of errors for metadata written to serializer. * `alloy_queue_series_network_timestamp_seconds` (gauge): Highest timestamp written to an endpoint. * `alloy_queue_series_network_sent` (counter): Number of series sent successfully. -* `alloy_queue_metadata_network_sent` (counter): Number of metadata sent successful. +* `alloy_queue_metadata_network_sent` (counter): Number of metadata sent successfully. * `alloy_queue_network_series_failed` (counter): Number of series failed. * `alloy_queue_network_metadata_failed` (counter): Number of metadata failed. * `alloy_queue_network_series_retried` (counter): Number of series retried due to network issues. From 42fbdd9775bd4e9df1da0840616f5c8d90bd683f Mon Sep 17 00:00:00 2001 From: mattdurham Date: Wed, 9 Oct 2024 08:27:35 -0400 Subject: [PATCH 31/44] Update docs/sources/reference/components/prometheus/prometheus.remote.queue.md Co-authored-by: Clayton Cornell <131809008+clayton-cornell@users.noreply.github.com> --- .../components/prometheus/prometheus.remote.queue.md | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index 300e35c83d..bd472ae163 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -250,10 +250,12 @@ Network errors will be retried. 429 errors will be retried. 5XX errors will retr ### Memory -`prometheus.remote.queue` is meant to be memory efficient. By adjusting the `max_signals_to_batch`, `queue_count`, and `batch_size` the amount of memory -can be controlled. A higher `max_signals_to_batch` allows for more efficient disk compression. A higher `queue_count` allows more concurrent writes and `batch_size` -allows more data sent at one time. This can allow greater throughput, at the cost of more memory on both {{< param "PRODUCT_NAME" >}} and the endpoint. The defaults are good for most -common usages. +`prometheus.remote.queue` is meant to be memory efficient. +You can adjust the `max_signals_to_batch`, `queue_count`, and `batch_size` to control how much memory is used. +A higher `max_signals_to_batch` allows for more efficient disk compression. +A higher `queue_count` allows more concurrent writes, and `batch_size` allows more data sent at one time. +This can allow greater throughput at the cost of more memory on both {{< param "PRODUCT_NAME" >}} and the endpoint. +The defaults are suitable for most common usages. ## Compatible components From d5eb26ef66beadcca49d0e4b678983048cc2f7d2 Mon Sep 17 00:00:00 2001 From: mattdurham Date: Wed, 9 Oct 2024 08:40:53 -0400 Subject: [PATCH 32/44] Update docs/sources/reference/components/prometheus/prometheus.remote.queue.md Co-authored-by: Clayton Cornell <131809008+clayton-cornell@users.noreply.github.com> --- .../components/prometheus/prometheus.remote.queue.md | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index bd472ae163..4e333869f3 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -246,7 +246,13 @@ Any data that has not been written to disk, or that is in the network queues is ### Retries -Network errors will be retried. 429 errors will be retried. 5XX errors will retry. Any other non-2XX return codes will not be tried. +`prometheus.remote.queue` will retry sending data if the following errors or HTTP status codes are returned: + + * Network errors. + * HTTP 429 error.s + * HTTP 5XX errors. + +`prometheus.remote.queue` will not retry sending data if any other unsuccessful status codes are returned. ### Memory From 0c9e755b0688144b4b88d9bfef2ae4fd1b36364a Mon Sep 17 00:00:00 2001 From: mattdurham Date: Thu, 10 Oct 2024 10:17:56 -0400 Subject: [PATCH 33/44] Update docs/sources/reference/components/prometheus/prometheus.remote.queue.md Co-authored-by: Clayton Cornell <131809008+clayton-cornell@users.noreply.github.com> --- .../reference/components/prometheus/prometheus.remote.queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index 4e333869f3..4decb97bc5 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -249,7 +249,7 @@ Any data that has not been written to disk, or that is in the network queues is `prometheus.remote.queue` will retry sending data if the following errors or HTTP status codes are returned: * Network errors. - * HTTP 429 error.s + * HTTP 429 errors. * HTTP 5XX errors. `prometheus.remote.queue` will not retry sending data if any other unsuccessful status codes are returned. From ce0ecb04699e2b03823971e38522622e34441a6b Mon Sep 17 00:00:00 2001 From: mattdurham Date: Fri, 11 Oct 2024 10:30:01 -0400 Subject: [PATCH 34/44] Update docs/sources/reference/components/prometheus/prometheus.remote.queue.md Co-authored-by: Paulin Todev --- .../reference/components/prometheus/prometheus.remote.queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index 4decb97bc5..a41b250e18 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -17,7 +17,7 @@ user-supplied endpoints. Metrics are sent over the network using the You can specify multiple `prometheus.remote.queue` components by giving them different labels. You should consider everything here extremely experimental and highly subject to change. -[emote_write-spec]: https://docs.google.com/document/d/1LPhVRSFkGNSuU1fBd81ulhsCPR4hkSZyyBj1SZ8fWOM/edit +[emote_write-spec]: https://prometheus.io/docs/specs/remote_write_spec/ From 872de534f3a221b34051976ec5129889e7e9ad06 Mon Sep 17 00:00:00 2001 From: matt durham Date: Fri, 11 Oct 2024 10:46:00 -0400 Subject: [PATCH 35/44] Docs PR feedback --- .../prometheus/prometheus.remote.queue.md | 41 ++++++------------- .../prometheus/remote/queue/component.go | 8 ++-- .../prometheus/remote/queue/endpoint.go | 1 + 3 files changed, 18 insertions(+), 32 deletions(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index 4decb97bc5..5c82b7068c 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -65,32 +65,31 @@ basic_auth` refers to a `basic_auth` block defined inside an ### serialization block The `serialization` block describes how often and at what limits to write to disk. Serialization settings -are shared for each `endpoint.` +are shared for each `endpoint`. The following arguments are supported: -Name | Type | Description | Default | Required ----- | ---- | ----------- | ------- | -------- -`max_signals_to_batch` | `uint` | The maximum number of signals before they are batched to disk. | `10,000` | no -`batch_frequency` | `duration` | How often to batch signals to disk if `max_signals_to_batch` is not reached. | no +Name | Type | Description | Default | Required +---- | ---- |-------------------------------------------------------------------------------|---------| -------- +`max_signals_to_batch` | `uint` | The maximum number of signals before they are batched to disk. | `10000` | no +`batch_frequency` | `duration` | How often to batch signals to disk if `max_signals_to_batch` is not reached. | `5s` | no ### endpoint block The `endpoint` block describes a single location to send metrics to. Multiple `endpoint` blocks can be provided to send metrics to multiple locations. Each -`endpoint` will have its own WAL folder +`endpoint` will have its own WAL folder. The following arguments are supported: Name | Type | Description | Default | Required ----- | ---- | ----------- | ------- | -------- +---- | ---- | ----------- | ------ | -------- `url` | `string` | Full URL to send metrics to. | | yes -`name` | `string` | Optional name to identify the endpoint in metrics. | | no `write_timeout` | `duration` | Timeout for requests made to the URL. | `"30s"` | no `retry_backoff` | `duration` | How often to wait between retries. | `1s` | no `max_retry_backoff_attempts` | Maximum number of retries before dropping the batch. | `1s` | no -`batch_count` | `uint` | How many series to queue in each queue. | `1,000` | no +`batch_count` | `uint` | How many series to queue in each queue. | `1000` | no `flush_frequency` | `duration` | How often to wait until sending if `batch_count` is not trigger. | `1s` | no `queue_count` | `uint` | How many concurrent batches to write. | 10 | no `external_labels` | `map(string)` | Labels to add to metrics sent over the network. | | no @@ -116,7 +115,7 @@ values. ## Debug information -`prometheus.remote_write` does not expose any component-specific debug +`prometheus.remote.queue` does not expose any component-specific debug information. ## Debug metrics @@ -200,7 +199,7 @@ Metrics that are new to `prometheus.remote.write`. These are highly subject to c ## Examples -The following examples show you how to create `prometheus.remote_write` components that send metrics to different destinations. +The following examples show you how to create `prometheus.remote.queue` components that send metrics to different destinations. ### Send metrics to a local Mimir instance @@ -220,7 +219,7 @@ prometheus.remote.queue "staging" { } // Configure a prometheus.scrape component to send metrics to -// prometheus.remote_write component. +// prometheus.remote.queue component. prometheus.scrape "demo" { targets = [ // Collect metrics from the default HTTP listen address. @@ -231,8 +230,6 @@ prometheus.scrape "demo" { ``` -## TODO Metadata settings - ## Technical details `prometheus.remote.queue` uses [snappy][] for compression. @@ -263,6 +260,8 @@ A higher `queue_count` allows more concurrent writes, and `batch_size` allows mo This can allow greater throughput at the cost of more memory on both {{< param "PRODUCT_NAME" >}} and the endpoint. The defaults are suitable for most common usages. + + ## Compatible components `prometheus.remote.queue` has exports that can be consumed by the following components: @@ -280,17 +279,3 @@ Refer to the linked documentation for more details. [WAL block]: #wal-block [Stop]: ../../../../set-up/run/ [run]: ../../../cli/run/ - - -## Compatible components - -`prometheus.remote.queue` has exports that can be consumed by the following components: - -- Components that consume [Prometheus `MetricsReceiver`](../../../compatibility/#prometheus-metricsreceiver-consumers) - -{{< admonition type="note" >}} -Connecting some components may not be sensible or components may require further configuration to make the connection work correctly. -Refer to the linked documentation for more details. -{{< /admonition >}} - - \ No newline at end of file diff --git a/internal/component/prometheus/remote/queue/component.go b/internal/component/prometheus/remote/queue/component.go index 59ea239a3d..083c3a2b51 100644 --- a/internal/component/prometheus/remote/queue/component.go +++ b/internal/component/prometheus/remote/queue/component.go @@ -38,12 +38,13 @@ func NewComponent(opts component.Options, args Arguments) (*Queue, error) { } s.opts.OnStateChange(Exports{Receiver: s}) err := s.createEndpoints() - for _, ep := range s.endpoints { - ep.Start() - } if err != nil { return nil, err } + + for _, ep := range s.endpoints { + ep.Start() + } return s, nil } @@ -95,7 +96,6 @@ func (s *Queue) Update(args component.Arguments) error { } s.args = newArgs // TODO @mattdurham need to cycle through the endpoints figuring out what changed instead of this global stop and start. - // TODO @mattdurham is there an issue/race condition with stopping these while the appender is still going on. if len(s.endpoints) > 0 { for _, ep := range s.endpoints { ep.Stop() diff --git a/internal/component/prometheus/remote/queue/endpoint.go b/internal/component/prometheus/remote/queue/endpoint.go index 643171be25..bc863a211f 100644 --- a/internal/component/prometheus/remote/queue/endpoint.go +++ b/internal/component/prometheus/remote/queue/endpoint.go @@ -44,6 +44,7 @@ func (ep *endpoint) Start() { } func (ep *endpoint) Stop() { + // Stop in order of data flow. This prevents errors around stopped mailboxes that can pop up. ep.serializer.Stop() ep.network.Stop() ep.network.Stop() From db5bd6a4866986128c5b6720b50af4791fd0fd04 Mon Sep 17 00:00:00 2001 From: mattdurham Date: Fri, 11 Oct 2024 10:47:23 -0400 Subject: [PATCH 36/44] Update docs/sources/reference/components/prometheus/prometheus.remote.queue.md Co-authored-by: Piotr <17101802+thampiotr@users.noreply.github.com> --- .../reference/components/prometheus/prometheus.remote.queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index 64d82c42c7..02d03e350d 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -41,7 +41,7 @@ The following arguments are supported: Name | Type | Description | Default | Required ---- | ---- | ----------- | ------- | -------- -`ttl` | `time` | `duration` | How long the timestamp of a signal is valid before the signal is discarded. | `2h` | no +`ttl` | `time` | `duration` | How long the samples can be queued for before they are discarded. | `2h` | no ## Blocks From 3ee51b3491247d7a0e92f3221e1c7c662d9a4593 Mon Sep 17 00:00:00 2001 From: matt durham Date: Fri, 11 Oct 2024 10:57:28 -0400 Subject: [PATCH 37/44] PR feedback --- .../prometheus/prometheus.remote.queue.md | 26 ++++++------ .../prometheus/remote/queue/e2e_bench_test.go | 16 +++---- .../prometheus/remote/queue/e2e_test.go | 16 +++---- .../prometheus/remote/queue/network/loop.go | 4 +- .../remote/queue/network/manager_test.go | 42 +++++++++---------- .../prometheus/remote/queue/types.go | 32 +++++++------- .../prometheus/remote/queue/types/network.go | 20 ++++----- 7 files changed, 78 insertions(+), 78 deletions(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index 64d82c42c7..bf7a0352a5 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -50,7 +50,7 @@ The following blocks are supported inside the definition of Hierarchy | Block | Description | Required --------- | ----- | ----------- | -------- -serialization | [serialization][] | Configuration for serializing and writing to disk | no +persistence | [persistence][] | Configuration for persistence | no endpoint | [endpoint][] | Location to send metrics to. | no endpoint > basic_auth | [basic_auth][] | Configure basic_auth for authenticating to the endpoint. | no @@ -60,11 +60,11 @@ basic_auth` refers to a `basic_auth` block defined inside an [endpoint]: #endpoint-block [basic_auth]: #basic_auth-block -[serialization]: #serialization-block +[persistence]: #persistence-block -### serialization block +### persistence block -The `serialization` block describes how often and at what limits to write to disk. Serialization settings +The `persistence` block describes how often and at what limits to write to disk. Persistence settings are shared for each `endpoint`. The following arguments are supported: @@ -83,16 +83,16 @@ The `endpoint` block describes a single location to send metrics to. Multiple The following arguments are supported: -Name | Type | Description | Default | Required ----- | ---- | ----------- | ------ | -------- -`url` | `string` | Full URL to send metrics to. | | yes -`write_timeout` | `duration` | Timeout for requests made to the URL. | `"30s"` | no -`retry_backoff` | `duration` | How often to wait between retries. | `1s` | no -`max_retry_backoff_attempts` | Maximum number of retries before dropping the batch. | `1s` | no -`batch_count` | `uint` | How many series to queue in each queue. | `1000` | no +Name | Type | Description | Default | Required +---- | ---- |------------------------------------------------------------------| ------ | -------- +`url` | `string` | Full URL to send metrics to. | | yes +`write_timeout` | `duration` | Timeout for requests made to the URL. | `"30s"` | no +`retry_backoff` | `duration` | How often to wait between retries. | `1s` | no +`max_retry_attempts` | Maximum number of retries before dropping the batch. | `0` | no +`batch_count` | `uint` | How many series to queue in each queue. | `1000` | no `flush_frequency` | `duration` | How often to wait until sending if `batch_count` is not trigger. | `1s` | no -`queue_count` | `uint` | How many concurrent batches to write. | 10 | no -`external_labels` | `map(string)` | Labels to add to metrics sent over the network. | | no +`queue_count` | `uint` | How many concurrent batches to write. | 10 | no +`external_labels` | `map(string)` | Labels to add to metrics sent over the network. | | no ### basic_auth block diff --git a/internal/component/prometheus/remote/queue/e2e_bench_test.go b/internal/component/prometheus/remote/queue/e2e_bench_test.go index a730687cd0..6d033d78d7 100644 --- a/internal/component/prometheus/remote/queue/e2e_bench_test.go +++ b/internal/component/prometheus/remote/queue/e2e_bench_test.go @@ -98,14 +98,14 @@ func newComponentBenchmark(t *testing.B, l log.Logger, url string, exp chan Expo BatchFrequency: 1 * time.Second, }, Endpoints: []EndpointConfig{{ - Name: "test", - URL: url, - Timeout: 10 * time.Second, - RetryBackoff: 1 * time.Second, - MaxRetryBackoffAttempts: 0, - BatchCount: 50, - FlushFrequency: 1 * time.Second, - QueueCount: 1, + Name: "test", + URL: url, + Timeout: 10 * time.Second, + RetryBackoff: 1 * time.Second, + MaxRetryAttempts: 0, + BatchCount: 50, + FlushFrequency: 1 * time.Second, + QueueCount: 1, }}, }) } diff --git a/internal/component/prometheus/remote/queue/e2e_test.go b/internal/component/prometheus/remote/queue/e2e_test.go index 9704ff7f6d..7d3186777a 100644 --- a/internal/component/prometheus/remote/queue/e2e_test.go +++ b/internal/component/prometheus/remote/queue/e2e_test.go @@ -360,14 +360,14 @@ func newComponent(t *testing.T, l *logging.Logger, url string, exp chan Exports, BatchFrequency: 1 * time.Second, }, Endpoints: []EndpointConfig{{ - Name: "test", - URL: url, - Timeout: 20 * time.Second, - RetryBackoff: 5 * time.Second, - MaxRetryBackoffAttempts: 1, - BatchCount: 50, - FlushFrequency: 1 * time.Second, - QueueCount: 1, + Name: "test", + URL: url, + Timeout: 20 * time.Second, + RetryBackoff: 5 * time.Second, + MaxRetryAttempts: 1, + BatchCount: 50, + FlushFrequency: 1 * time.Second, + QueueCount: 1, }}, }) } diff --git a/internal/component/prometheus/remote/queue/network/loop.go b/internal/component/prometheus/remote/queue/network/loop.go index 198ff2565b..87e0cf89cb 100644 --- a/internal/component/prometheus/remote/queue/network/loop.go +++ b/internal/component/prometheus/remote/queue/network/loop.go @@ -108,7 +108,7 @@ func (l *loop) DoWork(ctx actor.Context) actor.WorkerStatus { } } -// trySend is the core functionality for sending data to a endpoint. It will attempt retries as defined in MaxRetryBackoffAttempts. +// trySend is the core functionality for sending data to a endpoint. It will attempt retries as defined in MaxRetryAttempts. func (l *loop) trySend(ctx context.Context) { attempts := 0 for { @@ -130,7 +130,7 @@ func (l *loop) trySend(ctx context.Context) { return } attempts++ - if attempts > int(l.cfg.MaxRetryBackoffAttempts) && l.cfg.MaxRetryBackoffAttempts > 0 { + if attempts > int(l.cfg.MaxRetryAttempts) && l.cfg.MaxRetryAttempts > 0 { level.Debug(l.log).Log("msg", "max retry attempts reached", "attempts", attempts) l.sendingCleanup() return diff --git a/internal/component/prometheus/remote/queue/network/manager_test.go b/internal/component/prometheus/remote/queue/network/manager_test.go index f8c1d8e950..65e73ed065 100644 --- a/internal/component/prometheus/remote/queue/network/manager_test.go +++ b/internal/component/prometheus/remote/queue/network/manager_test.go @@ -159,13 +159,13 @@ func TestRetryBounded(t *testing.T) { defer cncl() cc := types.ConnectionConfig{ - URL: svr.URL, - Timeout: 1 * time.Second, - BatchCount: 1, - FlushFrequency: 1 * time.Second, - RetryBackoff: 100 * time.Millisecond, - MaxRetryBackoffAttempts: 1, - Connections: 1, + URL: svr.URL, + Timeout: 1 * time.Second, + BatchCount: 1, + FlushFrequency: 1 * time.Second, + RetryBackoff: 100 * time.Millisecond, + MaxRetryAttempts: 1, + Connections: 1, } logger := log.NewNopLogger() @@ -197,13 +197,13 @@ func TestRecoverable(t *testing.T) { defer cncl() cc := types.ConnectionConfig{ - URL: svr.URL, - Timeout: 1 * time.Second, - BatchCount: 1, - FlushFrequency: 1 * time.Second, - RetryBackoff: 100 * time.Millisecond, - MaxRetryBackoffAttempts: 1, - Connections: 1, + URL: svr.URL, + Timeout: 1 * time.Second, + BatchCount: 1, + FlushFrequency: 1 * time.Second, + RetryBackoff: 100 * time.Millisecond, + MaxRetryAttempts: 1, + Connections: 1, } logger := log.NewNopLogger() @@ -238,13 +238,13 @@ func TestNonRecoverable(t *testing.T) { defer cncl() cc := types.ConnectionConfig{ - URL: svr.URL, - Timeout: 1 * time.Second, - BatchCount: 1, - FlushFrequency: 1 * time.Second, - RetryBackoff: 100 * time.Millisecond, - MaxRetryBackoffAttempts: 1, - Connections: 1, + URL: svr.URL, + Timeout: 1 * time.Second, + BatchCount: 1, + FlushFrequency: 1 * time.Second, + RetryBackoff: 100 * time.Millisecond, + MaxRetryAttempts: 1, + Connections: 1, } logger := log.NewNopLogger() diff --git a/internal/component/prometheus/remote/queue/types.go b/internal/component/prometheus/remote/queue/types.go index 36fd3fff2e..7288251016 100644 --- a/internal/component/prometheus/remote/queue/types.go +++ b/internal/component/prometheus/remote/queue/types.go @@ -45,12 +45,12 @@ func (rc *Arguments) SetToDefault() { func defaultEndpointConfig() EndpointConfig { return EndpointConfig{ - Timeout: 30 * time.Second, - RetryBackoff: 1 * time.Second, - MaxRetryBackoffAttempts: 0, - BatchCount: 1_000, - FlushFrequency: 1 * time.Second, - QueueCount: 4, + Timeout: 30 * time.Second, + RetryBackoff: 1 * time.Second, + MaxRetryAttempts: 0, + BatchCount: 1_000, + FlushFrequency: 1 * time.Second, + QueueCount: 4, } } @@ -80,7 +80,7 @@ type EndpointConfig struct { // How long to wait between retries. RetryBackoff time.Duration `alloy:"retry_backoff,attr,optional"` // Maximum number of retries. - MaxRetryBackoffAttempts uint `alloy:"max_retry_backoff_attempts,attr,optional"` + MaxRetryAttempts uint `alloy:"max_retry_attempts,attr,optional"` // How many series to write at a time. BatchCount int `alloy:"batch_count,attr,optional"` // How long to wait before sending regardless of batch count. @@ -94,15 +94,15 @@ var UserAgent = fmt.Sprintf("Alloy/%s", version.Version) func (cc EndpointConfig) ToNativeType() types.ConnectionConfig { tcc := types.ConnectionConfig{ - URL: cc.URL, - UserAgent: UserAgent, - Timeout: cc.Timeout, - RetryBackoff: cc.RetryBackoff, - MaxRetryBackoffAttempts: cc.MaxRetryBackoffAttempts, - BatchCount: cc.BatchCount, - FlushFrequency: cc.FlushFrequency, - ExternalLabels: cc.ExternalLabels, - Connections: cc.QueueCount, + URL: cc.URL, + UserAgent: UserAgent, + Timeout: cc.Timeout, + RetryBackoff: cc.RetryBackoff, + MaxRetryAttempts: cc.MaxRetryAttempts, + BatchCount: cc.BatchCount, + FlushFrequency: cc.FlushFrequency, + ExternalLabels: cc.ExternalLabels, + Connections: cc.QueueCount, } if cc.BasicAuth != nil { tcc.BasicAuth = &types.BasicAuth{ diff --git a/internal/component/prometheus/remote/queue/types/network.go b/internal/component/prometheus/remote/queue/types/network.go index 3fc767af0d..caf93ce276 100644 --- a/internal/component/prometheus/remote/queue/types/network.go +++ b/internal/component/prometheus/remote/queue/types/network.go @@ -16,16 +16,16 @@ type NetworkClient interface { UpdateConfig(ctx context.Context, cfg ConnectionConfig) error } type ConnectionConfig struct { - URL string - BasicAuth *BasicAuth - UserAgent string - Timeout time.Duration - RetryBackoff time.Duration - MaxRetryBackoffAttempts uint - BatchCount int - FlushFrequency time.Duration - ExternalLabels map[string]string - Connections uint + URL string + BasicAuth *BasicAuth + UserAgent string + Timeout time.Duration + RetryBackoff time.Duration + MaxRetryAttempts uint + BatchCount int + FlushFrequency time.Duration + ExternalLabels map[string]string + Connections uint } type BasicAuth struct { From bd2d0839e8b7278abf2efe3a56165279829fb919 Mon Sep 17 00:00:00 2001 From: matt durham Date: Fri, 11 Oct 2024 11:08:45 -0400 Subject: [PATCH 38/44] PR feedback --- .../prometheus/remote/queue/component.go | 1 + .../prometheus/remote/queue/endpoint.go | 3 +-- .../remote/queue/network/manager.go | 20 +++++++++---------- .../remote/queue/serialization/serializer.go | 2 +- .../remote/queue/types/serializer.go | 2 ++ 5 files changed, 15 insertions(+), 13 deletions(-) diff --git a/internal/component/prometheus/remote/queue/component.go b/internal/component/prometheus/remote/queue/component.go index 083c3a2b51..f837f221db 100644 --- a/internal/component/prometheus/remote/queue/component.go +++ b/internal/component/prometheus/remote/queue/component.go @@ -96,6 +96,7 @@ func (s *Queue) Update(args component.Arguments) error { } s.args = newArgs // TODO @mattdurham need to cycle through the endpoints figuring out what changed instead of this global stop and start. + // This will cause data in the endpoints and their children to be lost. if len(s.endpoints) > 0 { for _, ep := range s.endpoints { ep.Stop() diff --git a/internal/component/prometheus/remote/queue/endpoint.go b/internal/component/prometheus/remote/queue/endpoint.go index bc863a211f..223dbafd3e 100644 --- a/internal/component/prometheus/remote/queue/endpoint.go +++ b/internal/component/prometheus/remote/queue/endpoint.go @@ -47,7 +47,6 @@ func (ep *endpoint) Stop() { // Stop in order of data flow. This prevents errors around stopped mailboxes that can pop up. ep.serializer.Stop() ep.network.Stop() - ep.network.Stop() ep.self.Stop() } @@ -84,7 +83,7 @@ func (ep *endpoint) deserializeAndSend(ctx context.Context, meta map[string]stri level.Error(ep.log).Log("msg", "version not found for deserialization") return } - if version != "alloy.metrics.queue.v1" { + if version != types.AlloyFileVersion { level.Error(ep.log).Log("msg", "invalid version found for deserialization", "version", version) return } diff --git a/internal/component/prometheus/remote/queue/network/manager.go b/internal/component/prometheus/remote/queue/network/manager.go index fbe6e430e1..277db5335a 100644 --- a/internal/component/prometheus/remote/queue/network/manager.go +++ b/internal/component/prometheus/remote/queue/network/manager.go @@ -22,10 +22,10 @@ type manager struct { metaStats func(types.NetworkStats) } -// configCallback allows the config to be synchronous. +// configCallback allows actors to notify via `done` channel when they're done processing the config `cc`. Useful when synchronous processing is required. type configCallback struct { - cc types.ConnectionConfig - ch chan struct{} + cc types.ConnectionConfig + done chan struct{} } var _ types.NetworkClient = (*manager)(nil) @@ -76,16 +76,16 @@ func (s *manager) SendMetadata(ctx context.Context, data *types.TimeSeriesBinary } func (s *manager) UpdateConfig(ctx context.Context, cc types.ConnectionConfig) error { - ch := make(chan struct{}) - defer close(ch) + done := make(chan struct{}) + defer close(done) err := s.configInbox.Send(ctx, configCallback{ - cc: cc, - ch: ch, + cc: cc, + done: done, }) if err != nil { return err } - <-ch + <-done return nil } @@ -99,7 +99,7 @@ func (s *manager) DoWork(ctx actor.Context) actor.WorkerStatus { } s.updateConfig(cfg.cc) // Notify the caller we have applied the config. - cfg.ch <- struct{}{} + cfg.done <- struct{}{} return actor.WorkerContinue default: } @@ -134,7 +134,7 @@ func (s *manager) DoWork(ctx actor.Context) actor.WorkerStatus { } s.updateConfig(cfg.cc) // Notify the caller we have applied the config. - cfg.ch <- struct{}{} + cfg.done <- struct{}{} return actor.WorkerContinue } } diff --git a/internal/component/prometheus/remote/queue/serialization/serializer.go b/internal/component/prometheus/remote/queue/serialization/serializer.go index f6a38c1b8c..95a89cca59 100644 --- a/internal/component/prometheus/remote/queue/serialization/serializer.go +++ b/internal/component/prometheus/remote/queue/serialization/serializer.go @@ -192,7 +192,7 @@ func (s *serializer) flushToDisk(ctx actor.Context) error { out := snappy.Encode(buf) meta := map[string]string{ // product.signal_type.schema.version - "version": "alloy.metrics.queue.v1", + "version": types.AlloyFileVersion, "compression": "snappy", "series_count": strconv.Itoa(len(group.Series)), "meta_count": strconv.Itoa(len(group.Metadata)), diff --git a/internal/component/prometheus/remote/queue/types/serializer.go b/internal/component/prometheus/remote/queue/types/serializer.go index 6919f666f4..d0041242cc 100644 --- a/internal/component/prometheus/remote/queue/types/serializer.go +++ b/internal/component/prometheus/remote/queue/types/serializer.go @@ -5,6 +5,8 @@ import ( "time" ) +const AlloyFileVersion = "alloy.metrics.queue.v1" + type SerializerConfig struct { // MaxSignalsInBatch controls what the max batch size is. MaxSignalsInBatch uint32 From b2d5cab484d224baec764980ee85bfbde58fc9c7 Mon Sep 17 00:00:00 2001 From: matt durham Date: Fri, 11 Oct 2024 11:13:07 -0400 Subject: [PATCH 39/44] PR feedback --- .../components/prometheus/prometheus.remote.queue.md | 2 +- .../component/prometheus/remote/queue/e2e_bench_test.go | 2 +- internal/component/prometheus/remote/queue/e2e_test.go | 2 +- internal/component/prometheus/remote/queue/types.go | 6 +++--- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md index 639e45e240..095507b270 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.remote.queue.md @@ -91,7 +91,7 @@ Name | Type | Description | `max_retry_attempts` | Maximum number of retries before dropping the batch. | `0` | no `batch_count` | `uint` | How many series to queue in each queue. | `1000` | no `flush_frequency` | `duration` | How often to wait until sending if `batch_count` is not trigger. | `1s` | no -`queue_count` | `uint` | How many concurrent batches to write. | 10 | no +`parallelism` | `uint` | How many parallel batches to write. | 10 | no `external_labels` | `map(string)` | Labels to add to metrics sent over the network. | | no ### basic_auth block diff --git a/internal/component/prometheus/remote/queue/e2e_bench_test.go b/internal/component/prometheus/remote/queue/e2e_bench_test.go index 6d033d78d7..9bf3f63f91 100644 --- a/internal/component/prometheus/remote/queue/e2e_bench_test.go +++ b/internal/component/prometheus/remote/queue/e2e_bench_test.go @@ -105,7 +105,7 @@ func newComponentBenchmark(t *testing.B, l log.Logger, url string, exp chan Expo MaxRetryAttempts: 0, BatchCount: 50, FlushFrequency: 1 * time.Second, - QueueCount: 1, + Parallelism: 1, }}, }) } diff --git a/internal/component/prometheus/remote/queue/e2e_test.go b/internal/component/prometheus/remote/queue/e2e_test.go index 7d3186777a..49c80effc7 100644 --- a/internal/component/prometheus/remote/queue/e2e_test.go +++ b/internal/component/prometheus/remote/queue/e2e_test.go @@ -367,7 +367,7 @@ func newComponent(t *testing.T, l *logging.Logger, url string, exp chan Exports, MaxRetryAttempts: 1, BatchCount: 50, FlushFrequency: 1 * time.Second, - QueueCount: 1, + Parallelism: 1, }}, }) } diff --git a/internal/component/prometheus/remote/queue/types.go b/internal/component/prometheus/remote/queue/types.go index 7288251016..32eef40fd5 100644 --- a/internal/component/prometheus/remote/queue/types.go +++ b/internal/component/prometheus/remote/queue/types.go @@ -50,7 +50,7 @@ func defaultEndpointConfig() EndpointConfig { MaxRetryAttempts: 0, BatchCount: 1_000, FlushFrequency: 1 * time.Second, - QueueCount: 4, + Parallelism: 4, } } @@ -86,7 +86,7 @@ type EndpointConfig struct { // How long to wait before sending regardless of batch count. FlushFrequency time.Duration `alloy:"flush_frequency,attr,optional"` // How many concurrent queues to have. - QueueCount uint `alloy:"queue_count,attr,optional"` + Parallelism uint `alloy:"parallelism,attr,optional"` ExternalLabels map[string]string `alloy:"external_labels,attr,optional"` } @@ -102,7 +102,7 @@ func (cc EndpointConfig) ToNativeType() types.ConnectionConfig { BatchCount: cc.BatchCount, FlushFrequency: cc.FlushFrequency, ExternalLabels: cc.ExternalLabels, - Connections: cc.QueueCount, + Connections: cc.Parallelism, } if cc.BasicAuth != nil { tcc.BasicAuth = &types.BasicAuth{ From 57d81b0ef1810956ca5a141b00c25ccc90b8b6f7 Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 15 Oct 2024 08:48:00 -0400 Subject: [PATCH 40/44] PR feedback --- .../sources/reference/compatibility/_index.md | 2 +- ...ote.queue.md => prometheus.write.queue.md} | 46 +++++++++--------- internal/component/all/all.go | 2 +- .../prometheus/remote/queue/README.md | 4 +- .../prometheus/remote/queue/component.go | 13 +++-- .../prometheus/remote/queue/e2e_bench_test.go | 4 +- .../prometheus/remote/queue/e2e_test.go | 4 +- .../prometheus/remote/queue/network/loop.go | 2 +- .../remote/queue/network/manager_test.go | 48 +++++++++---------- .../prometheus/remote/queue/types.go | 14 +++--- .../prometheus/remote/queue/types/network.go | 2 +- 11 files changed, 72 insertions(+), 69 deletions(-) rename docs/sources/reference/components/prometheus/{prometheus.remote.queue.md => prometheus.write.queue.md} (87%) diff --git a/docs/sources/reference/compatibility/_index.md b/docs/sources/reference/compatibility/_index.md index 377cd647b8..ec3a2eb435 100644 --- a/docs/sources/reference/compatibility/_index.md +++ b/docs/sources/reference/compatibility/_index.md @@ -174,7 +174,7 @@ The following components, grouped by namespace, _export_ Prometheus `MetricsRece {{< collapse title="prometheus" >}} - [prometheus.relabel](../components/prometheus/prometheus.relabel) -- [prometheus.remote.queue](../components/prometheus/prometheus.remote.queue) +- [prometheus.write.queue](../components/prometheus/prometheus.write.queue) - [prometheus.remote_write](../components/prometheus/prometheus.remote_write) {{< /collapse >}} diff --git a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md b/docs/sources/reference/components/prometheus/prometheus.write.queue.md similarity index 87% rename from docs/sources/reference/components/prometheus/prometheus.remote.queue.md rename to docs/sources/reference/components/prometheus/prometheus.write.queue.md index 095507b270..5e4f6a72a5 100644 --- a/docs/sources/reference/components/prometheus/prometheus.remote.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.write.queue.md @@ -1,20 +1,20 @@ --- -canonical: https://grafana.com/docs/alloy/latest/reference/components/prometheus/prometheus.remote.queue/ -description: Learn about prometheus.remote.queue -title: prometheus.remote.queue +canonical: https://grafana.com/docs/alloy/latest/reference/components/prometheus/prometheus.write.queue/ +description: Learn about prometheus.write.queue +title: prometheus.write.queue --- Experimental -# prometheus.remote.queue +# prometheus.write.queue -`prometheus.remote.queue` collects metrics sent from other components into a +`prometheus.write.queue` collects metrics sent from other components into a Write-Ahead Log (WAL) and forwards them over the network to a series of user-supplied endpoints. Metrics are sent over the network using the [Prometheus Remote Write protocol][remote_write-spec]. -You can specify multiple `prometheus.remote.queue` components by giving them different labels. +You can specify multiple `prometheus.write.queue` components by giving them different labels. You should consider everything here extremely experimental and highly subject to change. [emote_write-spec]: https://prometheus.io/docs/specs/remote_write_spec/ @@ -24,7 +24,7 @@ You should consider everything here extremely experimental and highly subject to ## Usage ```alloy -prometheus.remote.queue "LABEL" { +prometheus.write.queue "LABEL" { endpoint "default "{ url = REMOTE_WRITE_URL @@ -46,7 +46,7 @@ Name | Type | Description | Default | Required ## Blocks The following blocks are supported inside the definition of -`prometheus.remote.queue`: +`prometheus.write.queue`: Hierarchy | Block | Description | Required --------- | ----- | ----------- | -------- @@ -72,7 +72,7 @@ The following arguments are supported: Name | Type | Description | Default | Required ---- | ---- |-------------------------------------------------------------------------------|---------| -------- `max_signals_to_batch` | `uint` | The maximum number of signals before they are batched to disk. | `10000` | no -`batch_frequency` | `duration` | How often to batch signals to disk if `max_signals_to_batch` is not reached. | `5s` | no +`batch_interval` | `duration` | How often to batch signals to disk if `max_signals_to_batch` is not reached. | `5s` | no ### endpoint block @@ -90,7 +90,7 @@ Name | Type | Description | `retry_backoff` | `duration` | How often to wait between retries. | `1s` | no `max_retry_attempts` | Maximum number of retries before dropping the batch. | `0` | no `batch_count` | `uint` | How many series to queue in each queue. | `1000` | no -`flush_frequency` | `duration` | How often to wait until sending if `batch_count` is not trigger. | `1s` | no +`flush_interval` | `duration` | How often to wait until sending if `batch_count` is not trigger. | `1s` | no `parallelism` | `uint` | How many parallel batches to write. | 10 | no `external_labels` | `map(string)` | Labels to add to metrics sent over the network. | | no @@ -109,13 +109,13 @@ Name | Type | Description ## Component health -`prometheus.remote.queue` is only reported as unhealthy if given an invalid +`prometheus.write.queue` is only reported as unhealthy if given an invalid configuration. In those cases, exported fields are kept at their last healthy values. ## Debug information -`prometheus.remote.queue` does not expose any component-specific debug +`prometheus.write.queue` does not expose any component-specific debug information. ## Debug metrics @@ -199,14 +199,14 @@ Metrics that are new to `prometheus.remote.write`. These are highly subject to c ## Examples -The following examples show you how to create `prometheus.remote.queue` components that send metrics to different destinations. +The following examples show you how to create `prometheus.write.queue` components that send metrics to different destinations. ### Send metrics to a local Mimir instance -You can create a `prometheus.remote.queue` component that sends your metrics to a local Mimir instance: +You can create a `prometheus.write.queue` component that sends your metrics to a local Mimir instance: ```alloy -prometheus.remote.queue "staging" { +prometheus.write.queue "staging" { // Send metrics to a locally running Mimir. endpoint "mimir" { url = "http://mimir:9009/api/v1/push" @@ -219,21 +219,21 @@ prometheus.remote.queue "staging" { } // Configure a prometheus.scrape component to send metrics to -// prometheus.remote.queue component. +// prometheus.write.queue component. prometheus.scrape "demo" { targets = [ // Collect metrics from the default HTTP listen address. {"__address__" = "127.0.0.1:12345"}, ] - forward_to = [prometheus.remote.queue.staging.receiver] + forward_to = [prometheus.write.queue.staging.receiver] } ``` ## Technical details -`prometheus.remote.queue` uses [snappy][] for compression. -`prometheus.remote.queue` sends native histograms by default. +`prometheus.write.queue` uses [snappy][] for compression. +`prometheus.write.queue` sends native histograms by default. Any labels that start with `__` will be removed before sending to the endpoint. ### Data retention @@ -243,17 +243,17 @@ Any data that has not been written to disk, or that is in the network queues is ### Retries -`prometheus.remote.queue` will retry sending data if the following errors or HTTP status codes are returned: +`prometheus.write.queue` will retry sending data if the following errors or HTTP status codes are returned: * Network errors. * HTTP 429 errors. * HTTP 5XX errors. -`prometheus.remote.queue` will not retry sending data if any other unsuccessful status codes are returned. +`prometheus.write.queue` will not retry sending data if any other unsuccessful status codes are returned. ### Memory -`prometheus.remote.queue` is meant to be memory efficient. +`prometheus.write.queue` is meant to be memory efficient. You can adjust the `max_signals_to_batch`, `queue_count`, and `batch_size` to control how much memory is used. A higher `max_signals_to_batch` allows for more efficient disk compression. A higher `queue_count` allows more concurrent writes, and `batch_size` allows more data sent at one time. @@ -264,7 +264,7 @@ The defaults are suitable for most common usages. ## Compatible components -`prometheus.remote.queue` has exports that can be consumed by the following components: +`prometheus.write.queue` has exports that can be consumed by the following components: - Components that consume [Prometheus `MetricsReceiver`](../../../compatibility/#prometheus-metricsreceiver-consumers) diff --git a/internal/component/all/all.go b/internal/component/all/all.go index 99edcd7c30..a51ca2a3e4 100644 --- a/internal/component/all/all.go +++ b/internal/component/all/all.go @@ -134,7 +134,7 @@ import ( _ "github.com/grafana/alloy/internal/component/prometheus/operator/servicemonitors" // Import prometheus.operator.servicemonitors _ "github.com/grafana/alloy/internal/component/prometheus/receive_http" // Import prometheus.receive_http _ "github.com/grafana/alloy/internal/component/prometheus/relabel" // Import prometheus.relabel - _ "github.com/grafana/alloy/internal/component/prometheus/remote/queue" // Import prometheus.remote.queue + _ "github.com/grafana/alloy/internal/component/prometheus/remote/queue" // Import prometheus.write.queue _ "github.com/grafana/alloy/internal/component/prometheus/remotewrite" // Import prometheus.remote_write _ "github.com/grafana/alloy/internal/component/prometheus/scrape" // Import prometheus.scrape _ "github.com/grafana/alloy/internal/component/pyroscope/ebpf" // Import pyroscope.ebpf diff --git a/internal/component/prometheus/remote/queue/README.md b/internal/component/prometheus/remote/queue/README.md index ab5e2f7cbd..1c40f50ca5 100644 --- a/internal/component/prometheus/remote/queue/README.md +++ b/internal/component/prometheus/remote/queue/README.md @@ -4,9 +4,9 @@ ## Overview -The `prometheus.remote.queue` goals are to set reliable and repeatable memory and cpu usage based on the number of incoming and outgoing series. There are four broad parts to the system. +The `prometheus.write.queue` goals are to set reliable and repeatable memory and cpu usage based on the number of incoming and outgoing series. There are four broad parts to the system. -1. The `prometheus.remote.queue` component itself. This handles the lifecycle of the Alloy system. +1. The `prometheus.write.queue` component itself. This handles the lifecycle of the Alloy system. 2. The `serialization` converts an array of series into a serializable format. This is handled via [msgp]() library. 3. The `filequeue` is where the buffers are written to. This has a series of files that are committed to disk and then are read. 4. The `network` handles sending data. The data is sharded by the label hash across any number of loops that send data. diff --git a/internal/component/prometheus/remote/queue/component.go b/internal/component/prometheus/remote/queue/component.go index f837f221db..c3e8c962d2 100644 --- a/internal/component/prometheus/remote/queue/component.go +++ b/internal/component/prometheus/remote/queue/component.go @@ -19,7 +19,7 @@ import ( func init() { component.Register(component.Registration{ - Name: "prometheus.remote.queue", + Name: "prometheus.write.queue", Args: Arguments{}, Exports: Exports{}, Stability: featuregate.StabilityExperimental, @@ -42,9 +42,6 @@ func NewComponent(opts component.Options, args Arguments) (*Queue, error) { return nil, err } - for _, ep := range s.endpoints { - ep.Start() - } return s, nil } @@ -62,6 +59,12 @@ type Queue struct { // suffers a fatal error. Run is guaranteed to be called exactly once per // Component. func (s *Queue) Run(ctx context.Context) error { + s.mut.Lock() + for _, ep := range s.endpoints { + ep.Start() + } + s.mut.Unlock() + defer func() { s.mut.Lock() defer s.mut.Unlock() @@ -135,7 +138,7 @@ func (s *Queue) createEndpoints() error { } serial, err := serialization.NewSerializer(types.SerializerConfig{ MaxSignalsInBatch: uint32(s.args.Serialization.MaxSignalsToBatch), - FlushFrequency: s.args.Serialization.BatchFrequency, + FlushFrequency: s.args.Serialization.BatchInterval, }, fq, stats.UpdateSerializer, s.opts.Logger) if err != nil { return err diff --git a/internal/component/prometheus/remote/queue/e2e_bench_test.go b/internal/component/prometheus/remote/queue/e2e_bench_test.go index 9bf3f63f91..e405a2aa50 100644 --- a/internal/component/prometheus/remote/queue/e2e_bench_test.go +++ b/internal/component/prometheus/remote/queue/e2e_bench_test.go @@ -95,7 +95,7 @@ func newComponentBenchmark(t *testing.B, l log.Logger, url string, exp chan Expo TTL: 2 * time.Hour, Serialization: Serialization{ MaxSignalsToBatch: 100_000, - BatchFrequency: 1 * time.Second, + BatchInterval: 1 * time.Second, }, Endpoints: []EndpointConfig{{ Name: "test", @@ -104,7 +104,7 @@ func newComponentBenchmark(t *testing.B, l log.Logger, url string, exp chan Expo RetryBackoff: 1 * time.Second, MaxRetryAttempts: 0, BatchCount: 50, - FlushFrequency: 1 * time.Second, + FlushInterval: 1 * time.Second, Parallelism: 1, }}, }) diff --git a/internal/component/prometheus/remote/queue/e2e_test.go b/internal/component/prometheus/remote/queue/e2e_test.go index 49c80effc7..aca9ca29ba 100644 --- a/internal/component/prometheus/remote/queue/e2e_test.go +++ b/internal/component/prometheus/remote/queue/e2e_test.go @@ -357,7 +357,7 @@ func newComponent(t *testing.T, l *logging.Logger, url string, exp chan Exports, TTL: 2 * time.Hour, Serialization: Serialization{ MaxSignalsToBatch: 10_000, - BatchFrequency: 1 * time.Second, + BatchInterval: 1 * time.Second, }, Endpoints: []EndpointConfig{{ Name: "test", @@ -366,7 +366,7 @@ func newComponent(t *testing.T, l *logging.Logger, url string, exp chan Exports, RetryBackoff: 5 * time.Second, MaxRetryAttempts: 1, BatchCount: 50, - FlushFrequency: 1 * time.Second, + FlushInterval: 1 * time.Second, Parallelism: 1, }}, }) diff --git a/internal/component/prometheus/remote/queue/network/loop.go b/internal/component/prometheus/remote/queue/network/loop.go index 87e0cf89cb..e09c92a034 100644 --- a/internal/component/prometheus/remote/queue/network/loop.go +++ b/internal/component/prometheus/remote/queue/network/loop.go @@ -92,7 +92,7 @@ func (l *loop) DoWork(ctx actor.Context) actor.WorkerStatus { if len(l.series) == 0 { return actor.WorkerContinue } - if time.Since(l.lastSend) > l.cfg.FlushFrequency { + if time.Since(l.lastSend) > l.cfg.FlushInterval { l.trySend(ctx) } return actor.WorkerContinue diff --git a/internal/component/prometheus/remote/queue/network/manager_test.go b/internal/component/prometheus/remote/queue/network/manager_test.go index 65e73ed065..46eef2ea71 100644 --- a/internal/component/prometheus/remote/queue/network/manager_test.go +++ b/internal/component/prometheus/remote/queue/network/manager_test.go @@ -34,11 +34,11 @@ func TestSending(t *testing.T) { defer cncl() cc := types.ConnectionConfig{ - URL: svr.URL, - Timeout: 1 * time.Second, - BatchCount: 10, - FlushFrequency: 1 * time.Second, - Connections: 4, + URL: svr.URL, + Timeout: 1 * time.Second, + BatchCount: 10, + FlushInterval: 1 * time.Second, + Connections: 4, } logger := log.NewNopLogger() @@ -67,11 +67,11 @@ func TestUpdatingConfig(t *testing.T) { defer svr.Close() cc := types.ConnectionConfig{ - URL: svr.URL, - Timeout: 1 * time.Second, - BatchCount: 10, - FlushFrequency: 5 * time.Second, - Connections: 1, + URL: svr.URL, + Timeout: 1 * time.Second, + BatchCount: 10, + FlushInterval: 5 * time.Second, + Connections: 1, } logger := util.TestAlloyLogger(t) @@ -82,11 +82,11 @@ func TestUpdatingConfig(t *testing.T) { defer wr.Stop() cc2 := types.ConnectionConfig{ - URL: svr.URL, - Timeout: 1 * time.Second, - BatchCount: 20, - FlushFrequency: 5 * time.Second, - Connections: 1, + URL: svr.URL, + Timeout: 1 * time.Second, + BatchCount: 20, + FlushInterval: 5 * time.Second, + Connections: 1, } ctx := context.Background() err = wr.UpdateConfig(ctx, cc2) @@ -122,12 +122,12 @@ func TestRetry(t *testing.T) { defer cncl() cc := types.ConnectionConfig{ - URL: svr.URL, - Timeout: 1 * time.Second, - BatchCount: 1, - FlushFrequency: 1 * time.Second, - RetryBackoff: 100 * time.Millisecond, - Connections: 1, + URL: svr.URL, + Timeout: 1 * time.Second, + BatchCount: 1, + FlushInterval: 1 * time.Second, + RetryBackoff: 100 * time.Millisecond, + Connections: 1, } logger := log.NewNopLogger() @@ -162,7 +162,7 @@ func TestRetryBounded(t *testing.T) { URL: svr.URL, Timeout: 1 * time.Second, BatchCount: 1, - FlushFrequency: 1 * time.Second, + FlushInterval: 1 * time.Second, RetryBackoff: 100 * time.Millisecond, MaxRetryAttempts: 1, Connections: 1, @@ -200,7 +200,7 @@ func TestRecoverable(t *testing.T) { URL: svr.URL, Timeout: 1 * time.Second, BatchCount: 1, - FlushFrequency: 1 * time.Second, + FlushInterval: 1 * time.Second, RetryBackoff: 100 * time.Millisecond, MaxRetryAttempts: 1, Connections: 1, @@ -241,7 +241,7 @@ func TestNonRecoverable(t *testing.T) { URL: svr.URL, Timeout: 1 * time.Second, BatchCount: 1, - FlushFrequency: 1 * time.Second, + FlushInterval: 1 * time.Second, RetryBackoff: 100 * time.Millisecond, MaxRetryAttempts: 1, Connections: 1, diff --git a/internal/component/prometheus/remote/queue/types.go b/internal/component/prometheus/remote/queue/types.go index 32eef40fd5..097e8f0aaf 100644 --- a/internal/component/prometheus/remote/queue/types.go +++ b/internal/component/prometheus/remote/queue/types.go @@ -15,7 +15,7 @@ func defaultArgs() Arguments { TTL: 2 * time.Hour, Serialization: Serialization{ MaxSignalsToBatch: 10_000, - BatchFrequency: 5 * time.Second, + BatchInterval: 5 * time.Second, }, } } @@ -31,7 +31,7 @@ type Serialization struct { // The batch size to persist to the file queue. MaxSignalsToBatch int `alloy:"max_signals_to_batch,attr,optional"` // How often to flush to the file queue if BatchSize isn't met. - BatchFrequency time.Duration `alloy:"batch_frequency,attr,optional"` + BatchInterval time.Duration `alloy:"batch_interval,attr,optional"` } type Exports struct { @@ -49,7 +49,7 @@ func defaultEndpointConfig() EndpointConfig { RetryBackoff: 1 * time.Second, MaxRetryAttempts: 0, BatchCount: 1_000, - FlushFrequency: 1 * time.Second, + FlushInterval: 1 * time.Second, Parallelism: 4, } } @@ -63,8 +63,8 @@ func (r *Arguments) Validate() error { if conn.BatchCount <= 0 { return fmt.Errorf("batch_count must be greater than 0") } - if conn.FlushFrequency < 1*time.Second { - return fmt.Errorf("flush_frequency must be greater or equal to 1s, the internal timers resolution is 1s") + if conn.FlushInterval < 1*time.Second { + return fmt.Errorf("flush_interval must be greater or equal to 1s, the internal timers resolution is 1s") } } @@ -84,7 +84,7 @@ type EndpointConfig struct { // How many series to write at a time. BatchCount int `alloy:"batch_count,attr,optional"` // How long to wait before sending regardless of batch count. - FlushFrequency time.Duration `alloy:"flush_frequency,attr,optional"` + FlushInterval time.Duration `alloy:"flush_interval,attr,optional"` // How many concurrent queues to have. Parallelism uint `alloy:"parallelism,attr,optional"` ExternalLabels map[string]string `alloy:"external_labels,attr,optional"` @@ -100,7 +100,7 @@ func (cc EndpointConfig) ToNativeType() types.ConnectionConfig { RetryBackoff: cc.RetryBackoff, MaxRetryAttempts: cc.MaxRetryAttempts, BatchCount: cc.BatchCount, - FlushFrequency: cc.FlushFrequency, + FlushInterval: cc.FlushInterval, ExternalLabels: cc.ExternalLabels, Connections: cc.Parallelism, } diff --git a/internal/component/prometheus/remote/queue/types/network.go b/internal/component/prometheus/remote/queue/types/network.go index caf93ce276..c36ea930c4 100644 --- a/internal/component/prometheus/remote/queue/types/network.go +++ b/internal/component/prometheus/remote/queue/types/network.go @@ -23,7 +23,7 @@ type ConnectionConfig struct { RetryBackoff time.Duration MaxRetryAttempts uint BatchCount int - FlushFrequency time.Duration + FlushInterval time.Duration ExternalLabels map[string]string Connections uint } From 21952d84f34b4ec7160cb2ff5eb5884142b056e9 Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 15 Oct 2024 08:49:21 -0400 Subject: [PATCH 41/44] Fix typo --- .../reference/components/prometheus/prometheus.write.queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.write.queue.md b/docs/sources/reference/components/prometheus/prometheus.write.queue.md index 5e4f6a72a5..fadcdeef3f 100644 --- a/docs/sources/reference/components/prometheus/prometheus.write.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.write.queue.md @@ -174,7 +174,7 @@ They generally behave the same, but there are likely edge cases where they diffe * `prometheus_remote_storage_exemplars_in_total` (counter): Exemplars read into remote storage. -Metrics that are new to `prometheus.remote.write`. These are highly subject to change. +Metrics that are new to `prometheus.write.queue`. These are highly subject to change. * `alloy_queue_series_serializer_incoming_signals` (counter): Total number of series written to serialization. * `alloy_queue_metadata_serializer_incoming_signals` (counter): Total number of metadata written to serialization. From 5966b6d21f8b0bdcecae4ffd1d0093ab3a490cd4 Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 15 Oct 2024 08:51:10 -0400 Subject: [PATCH 42/44] Fix typo --- .../components/prometheus/prometheus.write.queue.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/sources/reference/components/prometheus/prometheus.write.queue.md b/docs/sources/reference/components/prometheus/prometheus.write.queue.md index fadcdeef3f..43c9cb560d 100644 --- a/docs/sources/reference/components/prometheus/prometheus.write.queue.md +++ b/docs/sources/reference/components/prometheus/prometheus.write.queue.md @@ -254,9 +254,9 @@ Any data that has not been written to disk, or that is in the network queues is ### Memory `prometheus.write.queue` is meant to be memory efficient. -You can adjust the `max_signals_to_batch`, `queue_count`, and `batch_size` to control how much memory is used. +You can adjust the `max_signals_to_batch`, `parallelism`, and `batch_size` to control how much memory is used. A higher `max_signals_to_batch` allows for more efficient disk compression. -A higher `queue_count` allows more concurrent writes, and `batch_size` allows more data sent at one time. +A higher `parallelism` allows more parallel writes, and `batch_size` allows more data sent at one time. This can allow greater throughput at the cost of more memory on both {{< param "PRODUCT_NAME" >}} and the endpoint. The defaults are suitable for most common usages. @@ -276,6 +276,5 @@ Refer to the linked documentation for more details. [snappy]: https://en.wikipedia.org/wiki/Snappy_(compression) -[WAL block]: #wal-block [Stop]: ../../../../set-up/run/ [run]: ../../../cli/run/ From 561bfebe56e4711f1168f201f8b20219842e6e97 Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 15 Oct 2024 09:08:16 -0400 Subject: [PATCH 43/44] Fix bug. --- .../component/prometheus/remote/queue/component.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/internal/component/prometheus/remote/queue/component.go b/internal/component/prometheus/remote/queue/component.go index c3e8c962d2..10a13b163c 100644 --- a/internal/component/prometheus/remote/queue/component.go +++ b/internal/component/prometheus/remote/queue/component.go @@ -36,11 +36,17 @@ func NewComponent(opts component.Options, args Arguments) (*Queue, error) { log: opts.Logger, endpoints: map[string]*endpoint{}, } - s.opts.OnStateChange(Exports{Receiver: s}) + err := s.createEndpoints() if err != nil { return nil, err } + // This needs to be started before we export the onstatechange so that it can accept + // signals. + for _, ep := range s.endpoints { + ep.Start() + } + s.opts.OnStateChange(Exports{Receiver: s}) return s, nil } @@ -59,12 +65,6 @@ type Queue struct { // suffers a fatal error. Run is guaranteed to be called exactly once per // Component. func (s *Queue) Run(ctx context.Context) error { - s.mut.Lock() - for _, ep := range s.endpoints { - ep.Start() - } - s.mut.Unlock() - defer func() { s.mut.Lock() defer s.mut.Unlock() From 9dc25f1cb476174453fd90abbf9e1ee4a3e16258 Mon Sep 17 00:00:00 2001 From: matt durham Date: Tue, 15 Oct 2024 09:24:01 -0400 Subject: [PATCH 44/44] Fix docs --- docs/sources/reference/compatibility/_index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/reference/compatibility/_index.md b/docs/sources/reference/compatibility/_index.md index ec3a2eb435..e96eb008ab 100644 --- a/docs/sources/reference/compatibility/_index.md +++ b/docs/sources/reference/compatibility/_index.md @@ -174,8 +174,8 @@ The following components, grouped by namespace, _export_ Prometheus `MetricsRece {{< collapse title="prometheus" >}} - [prometheus.relabel](../components/prometheus/prometheus.relabel) -- [prometheus.write.queue](../components/prometheus/prometheus.write.queue) - [prometheus.remote_write](../components/prometheus/prometheus.remote_write) +- [prometheus.write.queue](../components/prometheus/prometheus.write.queue) {{< /collapse >}}