prometheus/storage/remote/storage.go

221 lines
6.3 KiB
Go
Raw Normal View History

2017-05-10 02:44:13 -07:00
// Copyright 2017 The Prometheus Authors
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
2017-05-10 02:44:13 -07:00
package remote
import (
"context"
"crypto/md5"
"encoding/hex"
"fmt"
2017-05-10 02:44:13 -07:00
"sync"
"time"
2017-05-10 02:44:13 -07:00
"github.com/go-kit/log"
Tail the TSDB WAL for remote_write This change switches the remote_write API to use the TSDB WAL. This should reduce memory usage and prevent sample loss when the remote end point is down. We use the new LiveReader from TSDB to tail WAL segments. Logic for finding the tracking segment is included in this PR. The WAL is tailed once for each remote_write endpoint specified. Reading from the segment is based on a ticker rather than relying on fsnotify write events, which were found to be complicated and unreliable in early prototypes. Enqueuing a sample for sending via remote_write can now block, to provide back pressure. Queues are still required to acheive parallelism and batching. We have updated the queue config based on new defaults for queue capacity and pending samples values - much smaller values are now possible. The remote_write resharding code has been updated to prevent deadlocks, and extra tests have been added for these cases. As part of this change, we attempt to guarantee that samples are not lost; however this initial version doesn't guarantee this across Prometheus restarts or non-retryable errors from the remote end (eg 400s). This changes also includes the following optimisations: - only marshal the proto request once, not once per retry - maintain a single copy of the labels for given series to reduce GC pressure Other minor tweaks: - only reshard if we've also successfully sent recently - add pending samples, latest sent timestamp, WAL events processed metrics Co-authored-by: Chris Marchbanks <csmarchbanks.com> (initial prototype) Co-authored-by: Tom Wilkie <tom.wilkie@gmail.com> (sharding changes) Signed-off-by: Callum Styan <callumstyan@gmail.com>
2018-09-07 14:26:04 -07:00
"github.com/prometheus/client_golang/prometheus"
2017-05-10 02:44:13 -07:00
"github.com/prometheus/common/model"
"gopkg.in/yaml.v2"
2017-05-10 02:44:13 -07:00
"github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/scrape"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/util/logging"
2017-05-10 02:44:13 -07:00
)
// String constants for instrumentation.
const (
namespace = "prometheus"
subsystem = "remote_storage"
remoteName = "remote_name"
endpoint = "url"
)
type ReadyScrapeManager interface {
Get() (*scrape.Manager, error)
}
// startTimeCallback is a callback func that return the oldest timestamp stored in a storage.
type startTimeCallback func() (int64, error)
2017-08-01 03:19:35 -07:00
// Storage represents all the remote read and write endpoints. It implements
// storage.Storage.
2017-05-10 02:44:13 -07:00
type Storage struct {
logger *logging.Deduper
mtx sync.Mutex
2017-05-10 02:44:13 -07:00
rws *WriteStorage
2017-05-10 02:44:13 -07:00
// For reads.
queryables []storage.SampleAndChunkQueryable
localStartTimeCallback startTimeCallback
2017-05-10 02:44:13 -07:00
}
// NewStorage returns a remote.Storage.
func NewStorage(l log.Logger, reg prometheus.Registerer, stCallback startTimeCallback, walDir string, flushDeadline time.Duration, sm ReadyScrapeManager) *Storage {
2017-08-11 11:45:52 -07:00
if l == nil {
l = log.NewNopLogger()
}
logger := logging.Dedupe(l, 1*time.Minute)
Tail the TSDB WAL for remote_write This change switches the remote_write API to use the TSDB WAL. This should reduce memory usage and prevent sample loss when the remote end point is down. We use the new LiveReader from TSDB to tail WAL segments. Logic for finding the tracking segment is included in this PR. The WAL is tailed once for each remote_write endpoint specified. Reading from the segment is based on a ticker rather than relying on fsnotify write events, which were found to be complicated and unreliable in early prototypes. Enqueuing a sample for sending via remote_write can now block, to provide back pressure. Queues are still required to acheive parallelism and batching. We have updated the queue config based on new defaults for queue capacity and pending samples values - much smaller values are now possible. The remote_write resharding code has been updated to prevent deadlocks, and extra tests have been added for these cases. As part of this change, we attempt to guarantee that samples are not lost; however this initial version doesn't guarantee this across Prometheus restarts or non-retryable errors from the remote end (eg 400s). This changes also includes the following optimisations: - only marshal the proto request once, not once per retry - maintain a single copy of the labels for given series to reduce GC pressure Other minor tweaks: - only reshard if we've also successfully sent recently - add pending samples, latest sent timestamp, WAL events processed metrics Co-authored-by: Chris Marchbanks <csmarchbanks.com> (initial prototype) Co-authored-by: Tom Wilkie <tom.wilkie@gmail.com> (sharding changes) Signed-off-by: Callum Styan <callumstyan@gmail.com>
2018-09-07 14:26:04 -07:00
s := &Storage{
logger: logger,
localStartTimeCallback: stCallback,
}
s.rws = NewWriteStorage(s.logger, reg, walDir, flushDeadline, sm)
Tail the TSDB WAL for remote_write This change switches the remote_write API to use the TSDB WAL. This should reduce memory usage and prevent sample loss when the remote end point is down. We use the new LiveReader from TSDB to tail WAL segments. Logic for finding the tracking segment is included in this PR. The WAL is tailed once for each remote_write endpoint specified. Reading from the segment is based on a ticker rather than relying on fsnotify write events, which were found to be complicated and unreliable in early prototypes. Enqueuing a sample for sending via remote_write can now block, to provide back pressure. Queues are still required to acheive parallelism and batching. We have updated the queue config based on new defaults for queue capacity and pending samples values - much smaller values are now possible. The remote_write resharding code has been updated to prevent deadlocks, and extra tests have been added for these cases. As part of this change, we attempt to guarantee that samples are not lost; however this initial version doesn't guarantee this across Prometheus restarts or non-retryable errors from the remote end (eg 400s). This changes also includes the following optimisations: - only marshal the proto request once, not once per retry - maintain a single copy of the labels for given series to reduce GC pressure Other minor tweaks: - only reshard if we've also successfully sent recently - add pending samples, latest sent timestamp, WAL events processed metrics Co-authored-by: Chris Marchbanks <csmarchbanks.com> (initial prototype) Co-authored-by: Tom Wilkie <tom.wilkie@gmail.com> (sharding changes) Signed-off-by: Callum Styan <callumstyan@gmail.com>
2018-09-07 14:26:04 -07:00
return s
2017-08-11 11:45:52 -07:00
}
2017-05-10 02:44:13 -07:00
// ApplyConfig updates the state as the new config requires.
func (s *Storage) ApplyConfig(conf *config.Config) error {
s.mtx.Lock()
defer s.mtx.Unlock()
if err := s.rws.ApplyConfig(conf); err != nil {
return err
}
// Update read clients
readHashes := make(map[string]struct{})
queryables := make([]storage.SampleAndChunkQueryable, 0, len(conf.RemoteReadConfigs))
for _, rrConf := range conf.RemoteReadConfigs {
hash, err := toHash(rrConf)
if err != nil {
return err
}
// Don't allow duplicate remote read configs.
if _, ok := readHashes[hash]; ok {
return fmt.Errorf("duplicate remote read configs are not allowed, found duplicate for URL: %s", rrConf.URL)
}
readHashes[hash] = struct{}{}
// Set the queue name to the config hash if the user has not set
// a name in their remote write config so we can still differentiate
// between queues that have the same remote write endpoint.
name := hash[:6]
if rrConf.Name != "" {
name = rrConf.Name
}
c, err := NewReadClient(name, &ClientConfig{
URL: rrConf.URL,
Timeout: rrConf.RemoteTimeout,
HTTPClientConfig: rrConf.HTTPClientConfig,
Headers: rrConf.Headers,
})
if err != nil {
return err
}
externalLabels := conf.GlobalConfig.ExternalLabels
if !rrConf.FilterExternalLabels {
externalLabels = make(labels.Labels, 0)
}
queryables = append(queryables, NewSampleAndChunkQueryableClient(
c,
externalLabels,
labelsToEqualityMatchers(rrConf.RequiredMatchers),
rrConf.ReadRecent,
s.localStartTimeCallback,
))
}
s.queryables = queryables
return nil
}
// StartTime implements the Storage interface.
func (s *Storage) StartTime() (int64, error) {
return int64(model.Latest), nil
}
// Querier returns a storage.MergeQuerier combining the remote client queriers
// of each configured remote read endpoint.
*: Consistent Error/Warning handling for SeriesSet iterator: Allowing Async Select (#7251) * Add errors and Warnings to SeriesSet Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Change Querier interface and refactor accordingly Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Refactor promql/engine to propagate warnings at eval stage Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Address review issues Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Make sure all the series from all Selects are pre-advanced Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Address review issues Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Separate merge series sets Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Clean Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Refactor merge querier failure handling Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Refactored and simplified fanout with improvements from incoming chunk iterator PRs. * Secondary logic is hidden, instead of weird failed series set logic we had. * Fanout is well commented * Fanout closing record all errors * MergeQuerier improved API (clearer) * deferredGenericMergeSeriesSet is not needed as we return no samples anyway for failed series sets (next = false). Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fix formatting Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Fix CI issues Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Added final tests for error handling. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Addressed Brian's comments. * Moved hints in populate to be allocated only when needed. * Used sync.Once in secondary Querier to achieve all-or-nothing partial response logic. * Select after first Next is done will panic. NOTE: in lazySeriesSet in theory we could just panic, I think however we can totally just return error, it will panic in expand anyway. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Utilize errWithWarnings Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Fix recently introduced expansion issue Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Add tests for secondary querier error handling Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Implement lazy merge Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Add name to test cases Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Reorganize Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Address review comments Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Address review comments Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Remove redundant warnings Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> * Fix rebase mistake Signed-off-by: Kemal Akkoyun <kakkoyun@gmail.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com>
2020-06-09 09:57:31 -07:00
// Returned querier will never return error as all queryables are assumed best effort.
// Additionally all returned queriers ensure that its Select's SeriesSets have ready data after first `Next` invoke.
// This is because Prometheus (fanout and secondary queries) can't handle the stream failing half way through by design.
func (s *Storage) Querier(ctx context.Context, mint, maxt int64) (storage.Querier, error) {
s.mtx.Lock()
queryables := s.queryables
s.mtx.Unlock()
queriers := make([]storage.Querier, 0, len(queryables))
for _, queryable := range queryables {
q, err := queryable.Querier(ctx, mint, maxt)
if err != nil {
return nil, err
}
queriers = append(queriers, q)
}
tsdb: Added ChunkQueryable implementations to db; unified MergeSeriesSets and vertical to single struct. (#7069) * tsdb: Added ChunkQueryable implementations to db; unified compactor, querier and fanout block iterating. Chained to https://github.com/prometheus/prometheus/pull/7059 * NewMerge(Chunk)Querier now takies multiple primaries allowing tsdb DB code to use it. * Added single SeriesEntry / ChunkEntry for all series implementations. * Unified all vertical, and non vertical for compact and querying to single merge series / chunk sets by reusing VerticalSeriesMergeFunc for overlapping algorithm (same logic as before) * Added block (Base/Chunk/)Querier for block querying. We then use populateAndTomb(Base/Chunk/) to iterate over chunks or samples. * Refactored endpoint tests and querier tests to include subtests. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Addressed comments from Brian and Beorn. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fixed snapshot test and added chunk iterator support for DBReadOnly. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fixed race when iterating over Ats first. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fixed tests. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fixed populate block tests. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fixed endpoints test. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fixed test. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Added test & fixed case of head open chunk. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fixed DBReadOnly tests and bug producing 1 sample chunks. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Added cases for partial block overlap for multiple full chunks. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Added extra tests for chunk meta after compaction. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fixed small vertical merge bug and added more tests for that. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com>
2020-07-31 08:03:02 -07:00
return storage.NewMergeQuerier(nil, queriers, storage.ChainedSeriesMerge), nil
}
// ChunkQuerier returns a storage.MergeQuerier combining the remote client queriers
// of each configured remote read endpoint.
func (s *Storage) ChunkQuerier(ctx context.Context, mint, maxt int64) (storage.ChunkQuerier, error) {
s.mtx.Lock()
queryables := s.queryables
s.mtx.Unlock()
queriers := make([]storage.ChunkQuerier, 0, len(queryables))
for _, queryable := range queryables {
q, err := queryable.ChunkQuerier(ctx, mint, maxt)
if err != nil {
return nil, err
}
queriers = append(queriers, q)
}
tsdb: Added ChunkQueryable implementations to db; unified MergeSeriesSets and vertical to single struct. (#7069) * tsdb: Added ChunkQueryable implementations to db; unified compactor, querier and fanout block iterating. Chained to https://github.com/prometheus/prometheus/pull/7059 * NewMerge(Chunk)Querier now takies multiple primaries allowing tsdb DB code to use it. * Added single SeriesEntry / ChunkEntry for all series implementations. * Unified all vertical, and non vertical for compact and querying to single merge series / chunk sets by reusing VerticalSeriesMergeFunc for overlapping algorithm (same logic as before) * Added block (Base/Chunk/)Querier for block querying. We then use populateAndTomb(Base/Chunk/) to iterate over chunks or samples. * Refactored endpoint tests and querier tests to include subtests. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Addressed comments from Brian and Beorn. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fixed snapshot test and added chunk iterator support for DBReadOnly. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fixed race when iterating over Ats first. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fixed tests. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fixed populate block tests. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fixed endpoints test. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fixed test. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Added test & fixed case of head open chunk. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fixed DBReadOnly tests and bug producing 1 sample chunks. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Added cases for partial block overlap for multiple full chunks. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Added extra tests for chunk meta after compaction. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com> * Fixed small vertical merge bug and added more tests for that. Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com>
2020-07-31 08:03:02 -07:00
return storage.NewMergeChunkQuerier(nil, queriers, storage.NewCompactingChunkSeriesMerger(storage.ChainedSeriesMerge)), nil
}
// Appender implements storage.Storage.
func (s *Storage) Appender(ctx context.Context) storage.Appender {
return s.rws.Appender(ctx)
}
Initial draft of prometheus-agent (#8785) * Initial draft of prometheus-agent This commit introduces a new binary, prometheus-agent, based on the Grafana Agent code. It runs a WAL-only version of prometheus without the TSDB, alerting, or rule evaluations. It is intended to be used to remote_write to Prometheus or another remote_write receiver. By default, prometheus-agent will listen on port 9095 to not collide with the prometheus default of 9090. Truncation of the WAL cooperates on a best-effort case with Remote Write. Every time the WAL is truncated, the minimum timestamp of data to truncate is determined by the lowest sent timestamp of all samples across all remote_write endpoints. This gives loose guarantees that data from the WAL will not try to be removed until the maximum sample lifetime passes or remote_write starts functionining. Signed-off-by: Robert Fratto <robertfratto@gmail.com> * add tests for Prometheus agent (#22) * add tests for Prometheus agent * add tests for Prometheus agent * rearranged tests as per the review comments * update tests for Agent * changes as per code review comments Signed-off-by: SriKrishna Paparaju <paparaju@gmail.com> * incremental changes to prometheus agent Signed-off-by: SriKrishna Paparaju <paparaju@gmail.com> * changes as per code review comments Signed-off-by: SriKrishna Paparaju <paparaju@gmail.com> * Commit feedback from code review Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Co-authored-by: Ganesh Vernekar <ganeshvern@gmail.com> Signed-off-by: Robert Fratto <robertfratto@gmail.com> * Port over some comments from grafana/agent Signed-off-by: Robert Fratto <robertfratto@gmail.com> * Rename agent.Storage to agent.DB for tsdb consistency Signed-off-by: Robert Fratto <robertfratto@gmail.com> * Consolidate agentMode ifs in cmd/prometheus/main.go Signed-off-by: Robert Fratto <robertfratto@gmail.com> * Document PreAction usage requirements better for agent mode flags Signed-off-by: Robert Fratto <robertfratto@gmail.com> * remove unnecessary defaultListenAddr Signed-off-by: Robert Fratto <robertfratto@gmail.com> * `go fmt ./tsdb/agent` and fix lint errors Signed-off-by: Robert Fratto <robertfratto@gmail.com> Co-authored-by: SriKrishna Paparaju <paparaju@gmail.com>
2021-10-29 08:25:05 -07:00
// LowestSentTimestamp returns the lowest sent timestamp across all queues.
func (s *Storage) LowestSentTimestamp() int64 {
return s.rws.LowestSentTimestamp()
}
2017-08-01 03:19:35 -07:00
// Close the background processing of the storage queues.
2017-05-10 02:44:13 -07:00
func (s *Storage) Close() error {
s.logger.Stop()
2017-05-10 02:44:13 -07:00
s.mtx.Lock()
defer s.mtx.Unlock()
return s.rws.Close()
2017-05-10 02:44:13 -07:00
}
func labelsToEqualityMatchers(ls model.LabelSet) []*labels.Matcher {
ms := make([]*labels.Matcher, 0, len(ls))
for k, v := range ls {
ms = append(ms, &labels.Matcher{
Type: labels.MatchEqual,
Name: string(k),
Value: string(v),
})
}
return ms
}
// Used for hashing configs and diff'ing hashes in ApplyConfig.
func toHash(data interface{}) (string, error) {
bytes, err := yaml.Marshal(data)
if err != nil {
return "", err
}
hash := md5.Sum(bytes)
return hex.EncodeToString(hash[:]), nil
}