Re-add multiple remote writers

Each remote write endpoint gets its own set of relabeling rules.

This is based on the (yet-to-be-merged)
https://github.com/prometheus/prometheus/pull/2419, which removes legacy
remote write implementations.
This commit is contained in:
Julius Volz 2017-02-13 21:43:20 +01:00
parent a6d81a9f88
commit e9476b35d5
7 changed files with 122 additions and 89 deletions

View file

@ -204,7 +204,7 @@ type Config struct {
RuleFiles []string `yaml:"rule_files,omitempty"` RuleFiles []string `yaml:"rule_files,omitempty"`
ScrapeConfigs []*ScrapeConfig `yaml:"scrape_configs,omitempty"` ScrapeConfigs []*ScrapeConfig `yaml:"scrape_configs,omitempty"`
RemoteWriteConfig RemoteWriteConfig `yaml:"remote_write,omitempty"` RemoteWriteConfigs []*RemoteWriteConfig `yaml:"remote_write,omitempty"`
// Catches all undefined fields and must be empty after parsing. // Catches all undefined fields and must be empty after parsing.
XXX map[string]interface{} `yaml:",inline"` XXX map[string]interface{} `yaml:",inline"`

View file

@ -26,6 +26,14 @@ import (
"gopkg.in/yaml.v2" "gopkg.in/yaml.v2"
) )
func mustParseURL(u string) *URL {
parsed, err := url.Parse(u)
if err != nil {
panic(err)
}
return &URL{URL: parsed}
}
var expectedConf = &Config{ var expectedConf = &Config{
GlobalConfig: GlobalConfig{ GlobalConfig: GlobalConfig{
ScrapeInterval: model.Duration(15 * time.Second), ScrapeInterval: model.Duration(15 * time.Second),
@ -44,17 +52,24 @@ var expectedConf = &Config{
"testdata/my/*.rules", "testdata/my/*.rules",
}, },
RemoteWriteConfig: RemoteWriteConfig{ RemoteWriteConfigs: []*RemoteWriteConfig{
RemoteTimeout: model.Duration(30 * time.Second), {
WriteRelabelConfigs: []*RelabelConfig{ URL: mustParseURL("http://remote1/push"),
{ RemoteTimeout: model.Duration(30 * time.Second),
SourceLabels: model.LabelNames{"__name__"}, WriteRelabelConfigs: []*RelabelConfig{
Separator: ";", {
Regex: MustNewRegexp("expensive.*"), SourceLabels: model.LabelNames{"__name__"},
Replacement: "$1", Separator: ";",
Action: RelabelDrop, Regex: MustNewRegexp("expensive.*"),
Replacement: "$1",
Action: RelabelDrop,
},
}, },
}, },
{
URL: mustParseURL("http://remote2/push"),
RemoteTimeout: model.Duration(30 * time.Second),
},
}, },
ScrapeConfigs: []*ScrapeConfig{ ScrapeConfigs: []*ScrapeConfig{

View file

@ -14,10 +14,12 @@ rule_files:
- "my/*.rules" - "my/*.rules"
remote_write: remote_write:
write_relabel_configs: - url: http://remote1/push
- source_labels: [__name__] write_relabel_configs:
regex: expensive.* - source_labels: [__name__]
action: drop regex: expensive.*
action: drop
- url: http://remote2/push
scrape_configs: scrape_configs:
- job_name: prometheus - job_name: prometheus

View file

@ -31,13 +31,14 @@ import (
// Client allows sending batches of Prometheus samples to an HTTP endpoint. // Client allows sending batches of Prometheus samples to an HTTP endpoint.
type Client struct { type Client struct {
index int // Used to differentiate metrics.
url config.URL url config.URL
client *http.Client client *http.Client
timeout time.Duration timeout time.Duration
} }
// NewClient creates a new Client. // NewClient creates a new Client.
func NewClient(conf config.RemoteWriteConfig) (*Client, error) { func NewClient(index int, conf *config.RemoteWriteConfig) (*Client, error) {
tlsConfig, err := httputil.NewTLSConfig(conf.TLSConfig) tlsConfig, err := httputil.NewTLSConfig(conf.TLSConfig)
if err != nil { if err != nil {
return nil, err return nil, err
@ -55,6 +56,7 @@ func NewClient(conf config.RemoteWriteConfig) (*Client, error) {
} }
return &Client{ return &Client{
index: index,
url: *conf.URL, url: *conf.URL,
client: httputil.NewClient(rt), client: httputil.NewClient(rt),
timeout: time.Duration(conf.RemoteTimeout), timeout: time.Duration(conf.RemoteTimeout),
@ -114,7 +116,7 @@ func (c *Client) Store(samples model.Samples) error {
return nil return nil
} }
// Name identifies the client as a generic client. // Name identifies the client.
func (c Client) Name() string { func (c Client) Name() string {
return "generic" return fmt.Sprintf("%d:%s", c.index, c.url)
} }

View file

@ -20,6 +20,8 @@ import (
"github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/log" "github.com/prometheus/common/log"
"github.com/prometheus/common/model" "github.com/prometheus/common/model"
"github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/relabel"
) )
// String constants for instrumentation. // String constants for instrumentation.
@ -27,6 +29,12 @@ const (
namespace = "prometheus" namespace = "prometheus"
subsystem = "remote_storage" subsystem = "remote_storage"
queue = "queue" queue = "queue"
defaultShards = 10
defaultMaxSamplesPerSend = 100
// The queue capacity is per shard.
defaultQueueCapacity = 100 * 1024 / defaultShards
defaultBatchSendDeadline = 5 * time.Second
) )
var ( var (
@ -105,25 +113,21 @@ type StorageClient interface {
Name() string Name() string
} }
// StorageQueueManagerConfig configures a storage queue.
type StorageQueueManagerConfig struct { type StorageQueueManagerConfig struct {
QueueCapacity int // Number of samples to buffer per shard before we start dropping them. QueueCapacity int // Number of samples to buffer per shard before we start dropping them.
Shards int // Number of shards, i.e. amount of concurrency. Shards int // Number of shards, i.e. amount of concurrency.
MaxSamplesPerSend int // Maximum number of samples per send. MaxSamplesPerSend int // Maximum number of samples per send.
BatchSendDeadline time.Duration // Maximum time sample will wait in buffer. BatchSendDeadline time.Duration // Maximum time sample will wait in buffer.
} ExternalLabels model.LabelSet
RelabelConfigs []*config.RelabelConfig
var defaultConfig = StorageQueueManagerConfig{ Client StorageClient
QueueCapacity: 100 * 1024 / 10,
Shards: 10,
MaxSamplesPerSend: 100,
BatchSendDeadline: 5 * time.Second,
} }
// StorageQueueManager manages a queue of samples to be sent to the Storage // StorageQueueManager manages a queue of samples to be sent to the Storage
// indicated by the provided StorageClient. // indicated by the provided StorageClient.
type StorageQueueManager struct { type StorageQueueManager struct {
cfg StorageQueueManagerConfig cfg StorageQueueManagerConfig
tsdb StorageClient
shards []chan *model.Sample shards []chan *model.Sample
wg sync.WaitGroup wg sync.WaitGroup
done chan struct{} done chan struct{}
@ -131,9 +135,18 @@ type StorageQueueManager struct {
} }
// NewStorageQueueManager builds a new StorageQueueManager. // NewStorageQueueManager builds a new StorageQueueManager.
func NewStorageQueueManager(tsdb StorageClient, cfg *StorageQueueManagerConfig) *StorageQueueManager { func NewStorageQueueManager(cfg StorageQueueManagerConfig) *StorageQueueManager {
if cfg == nil { if cfg.QueueCapacity == 0 {
cfg = &defaultConfig cfg.QueueCapacity = defaultQueueCapacity
}
if cfg.Shards == 0 {
cfg.Shards = defaultShards
}
if cfg.MaxSamplesPerSend == 0 {
cfg.MaxSamplesPerSend = defaultMaxSamplesPerSend
}
if cfg.BatchSendDeadline == 0 {
cfg.BatchSendDeadline = defaultBatchSendDeadline
} }
shards := make([]chan *model.Sample, cfg.Shards) shards := make([]chan *model.Sample, cfg.Shards)
@ -142,11 +155,10 @@ func NewStorageQueueManager(tsdb StorageClient, cfg *StorageQueueManagerConfig)
} }
t := &StorageQueueManager{ t := &StorageQueueManager{
cfg: *cfg, cfg: cfg,
tsdb: tsdb,
shards: shards, shards: shards,
done: make(chan struct{}), done: make(chan struct{}),
queueName: tsdb.Name(), queueName: cfg.Client.Name(),
} }
queueCapacity.WithLabelValues(t.queueName).Set(float64(t.cfg.QueueCapacity)) queueCapacity.WithLabelValues(t.queueName).Set(float64(t.cfg.QueueCapacity))
@ -158,11 +170,28 @@ func NewStorageQueueManager(tsdb StorageClient, cfg *StorageQueueManagerConfig)
// sample on the floor if the queue is full. // sample on the floor if the queue is full.
// Always returns nil. // Always returns nil.
func (t *StorageQueueManager) Append(s *model.Sample) error { func (t *StorageQueueManager) Append(s *model.Sample) error {
fp := s.Metric.FastFingerprint() var snew model.Sample
snew = *s
snew.Metric = s.Metric.Clone()
for ln, lv := range t.cfg.ExternalLabels {
if _, ok := s.Metric[ln]; !ok {
snew.Metric[ln] = lv
}
}
snew.Metric = model.Metric(
relabel.Process(model.LabelSet(snew.Metric), t.cfg.RelabelConfigs...))
if snew.Metric == nil {
return nil
}
fp := snew.Metric.FastFingerprint()
shard := uint64(fp) % uint64(t.cfg.Shards) shard := uint64(fp) % uint64(t.cfg.Shards)
select { select {
case t.shards[shard] <- s: case t.shards[shard] <- &snew:
queueLength.WithLabelValues(t.queueName).Inc() queueLength.WithLabelValues(t.queueName).Inc()
default: default:
droppedSamplesTotal.WithLabelValues(t.queueName).Inc() droppedSamplesTotal.WithLabelValues(t.queueName).Inc()
@ -239,7 +268,7 @@ func (t *StorageQueueManager) sendSamples(s model.Samples) {
// sample isn't sent correctly the first time, it's simply dropped on the // sample isn't sent correctly the first time, it's simply dropped on the
// floor. // floor.
begin := time.Now() begin := time.Now()
err := t.tsdb.Store(s) err := t.cfg.Client.Store(s)
duration := time.Since(begin).Seconds() duration := time.Since(begin).Seconds()
if err != nil { if err != nil {

View file

@ -81,9 +81,7 @@ func (c *TestStorageClient) Name() string {
func TestSampleDelivery(t *testing.T) { func TestSampleDelivery(t *testing.T) {
// Let's create an even number of send batches so we don't run into the // Let's create an even number of send batches so we don't run into the
// batch timeout case. // batch timeout case.
cfg := defaultConfig n := defaultQueueCapacity * 2
n := cfg.QueueCapacity * 2
cfg.Shards = 1
samples := make(model.Samples, 0, n) samples := make(model.Samples, 0, n)
for i := 0; i < n; i++ { for i := 0; i < n; i++ {
@ -98,7 +96,11 @@ func TestSampleDelivery(t *testing.T) {
c := NewTestStorageClient() c := NewTestStorageClient()
c.expectSamples(samples[:len(samples)/2]) c.expectSamples(samples[:len(samples)/2])
m := NewStorageQueueManager(c, &cfg)
m := NewStorageQueueManager(StorageQueueManagerConfig{
Client: c,
Shards: 1,
})
// These should be received by the client. // These should be received by the client.
for _, s := range samples[:len(samples)/2] { for _, s := range samples[:len(samples)/2] {
@ -115,11 +117,8 @@ func TestSampleDelivery(t *testing.T) {
} }
func TestSampleDeliveryOrder(t *testing.T) { func TestSampleDeliveryOrder(t *testing.T) {
cfg := defaultConfig
ts := 10 ts := 10
n := cfg.MaxSamplesPerSend * ts n := defaultMaxSamplesPerSend * ts
// Ensure we don't drop samples in this test.
cfg.QueueCapacity = n
samples := make(model.Samples, 0, n) samples := make(model.Samples, 0, n)
for i := 0; i < n; i++ { for i := 0; i < n; i++ {
@ -135,7 +134,11 @@ func TestSampleDeliveryOrder(t *testing.T) {
c := NewTestStorageClient() c := NewTestStorageClient()
c.expectSamples(samples) c.expectSamples(samples)
m := NewStorageQueueManager(c, &cfg) m := NewStorageQueueManager(StorageQueueManagerConfig{
Client: c,
// Ensure we don't drop samples in this test.
QueueCapacity: n,
})
// These should be received by the client. // These should be received by the client.
for _, s := range samples { for _, s := range samples {
@ -194,9 +197,7 @@ func TestSpawnNotMoreThanMaxConcurrentSendsGoroutines(t *testing.T) {
// `MaxSamplesPerSend*Shards` samples should be consumed by the // `MaxSamplesPerSend*Shards` samples should be consumed by the
// per-shard goroutines, and then another `MaxSamplesPerSend` // per-shard goroutines, and then another `MaxSamplesPerSend`
// should be left on the queue. // should be left on the queue.
cfg := defaultConfig n := defaultMaxSamplesPerSend*defaultShards + defaultMaxSamplesPerSend
n := cfg.MaxSamplesPerSend*cfg.Shards + cfg.MaxSamplesPerSend
cfg.QueueCapacity = n
samples := make(model.Samples, 0, n) samples := make(model.Samples, 0, n)
for i := 0; i < n; i++ { for i := 0; i < n; i++ {
@ -210,7 +211,10 @@ func TestSpawnNotMoreThanMaxConcurrentSendsGoroutines(t *testing.T) {
} }
c := NewTestBlockedStorageClient() c := NewTestBlockedStorageClient()
m := NewStorageQueueManager(c, &cfg) m := NewStorageQueueManager(StorageQueueManagerConfig{
Client: c,
QueueCapacity: n,
})
m.Start() m.Start()
@ -239,14 +243,14 @@ func TestSpawnNotMoreThanMaxConcurrentSendsGoroutines(t *testing.T) {
time.Sleep(10 * time.Millisecond) time.Sleep(10 * time.Millisecond)
} }
if m.queueLen() != cfg.MaxSamplesPerSend { if m.queueLen() != defaultMaxSamplesPerSend {
t.Fatalf("Failed to drain StorageQueueManager queue, %d elements left", t.Fatalf("Failed to drain StorageQueueManager queue, %d elements left",
m.queueLen(), m.queueLen(),
) )
} }
numCalls := c.NumCalls() numCalls := c.NumCalls()
if numCalls != uint64(cfg.Shards) { if numCalls != uint64(defaultShards) {
t.Errorf("Saw %d concurrent sends, expected %d", numCalls, cfg.Shards) t.Errorf("Saw %d concurrent sends, expected %d", numCalls, defaultShards)
} }
} }

View file

@ -19,16 +19,12 @@ import (
"github.com/prometheus/common/model" "github.com/prometheus/common/model"
"github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/relabel"
) )
// Storage allows queueing samples for remote writes. // Storage allows queueing samples for remote writes.
type Storage struct { type Storage struct {
mtx sync.RWMutex mtx sync.RWMutex
externalLabels model.LabelSet queues []*StorageQueueManager
conf config.RemoteWriteConfig
queue *StorageQueueManager
} }
// ApplyConfig updates the state as the new config requires. // ApplyConfig updates the state as the new config requires.
@ -36,34 +32,36 @@ func (s *Storage) ApplyConfig(conf *config.Config) error {
s.mtx.Lock() s.mtx.Lock()
defer s.mtx.Unlock() defer s.mtx.Unlock()
newQueues := []*StorageQueueManager{}
// TODO: we should only stop & recreate queues which have changes, // TODO: we should only stop & recreate queues which have changes,
// as this can be quite disruptive. // as this can be quite disruptive.
var newQueue *StorageQueueManager for i, rwConf := range conf.RemoteWriteConfigs {
c, err := NewClient(i, rwConf)
if conf.RemoteWriteConfig.URL != nil {
c, err := NewClient(conf.RemoteWriteConfig)
if err != nil { if err != nil {
return err return err
} }
newQueue = NewStorageQueueManager(c, nil) newQueues = append(newQueues, NewStorageQueueManager(StorageQueueManagerConfig{
Client: c,
ExternalLabels: conf.GlobalConfig.ExternalLabels,
RelabelConfigs: rwConf.WriteRelabelConfigs,
}))
} }
if s.queue != nil { for _, q := range s.queues {
s.queue.Stop() q.Stop()
} }
s.queue = newQueue
s.conf = conf.RemoteWriteConfig s.queues = newQueues
s.externalLabels = conf.GlobalConfig.ExternalLabels for _, q := range s.queues {
if s.queue != nil { q.Start()
s.queue.Start()
} }
return nil return nil
} }
// Stop the background processing of the storage queues. // Stop the background processing of the storage queues.
func (s *Storage) Stop() { func (s *Storage) Stop() {
if s.queue != nil { for _, q := range s.queues {
s.queue.Stop() q.Stop()
} }
} }
@ -72,26 +70,9 @@ func (s *Storage) Append(smpl *model.Sample) error {
s.mtx.RLock() s.mtx.RLock()
defer s.mtx.RUnlock() defer s.mtx.RUnlock()
if s.queue == nil { for _, q := range s.queues {
return nil q.Append(smpl)
} }
var snew model.Sample
snew = *smpl
snew.Metric = smpl.Metric.Clone()
for ln, lv := range s.externalLabels {
if _, ok := smpl.Metric[ln]; !ok {
snew.Metric[ln] = lv
}
}
snew.Metric = model.Metric(
relabel.Process(model.LabelSet(snew.Metric), s.conf.WriteRelabelConfigs...))
if snew.Metric == nil {
return nil
}
s.queue.Append(&snew)
return nil return nil
} }