mirror of
https://github.com/prometheus/prometheus.git
synced 2025-03-05 20:59:13 -08:00
Shard/parrallelise samples by fingerprint in StorageQueueManager
By splitting the single queue into multiple queues and flushing each individual queue in serially (and all queues in parallel), we can guarantee to preserve the order of timestampsin samples sent to downstream systems.
This commit is contained in:
parent
fe29e87824
commit
ece12bff93
|
@ -14,6 +14,7 @@
|
||||||
package remote
|
package remote
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/prometheus/client_golang/prometheus"
|
"github.com/prometheus/client_golang/prometheus"
|
||||||
|
@ -21,16 +22,6 @@ import (
|
||||||
"github.com/prometheus/common/model"
|
"github.com/prometheus/common/model"
|
||||||
)
|
)
|
||||||
|
|
||||||
const (
|
|
||||||
// The maximum number of concurrent send requests to the remote storage.
|
|
||||||
maxConcurrentSends = 10
|
|
||||||
// The maximum number of samples to fit into a single request to the remote storage.
|
|
||||||
maxSamplesPerSend = 100
|
|
||||||
// The deadline after which to send queued samples even if the maximum batch
|
|
||||||
// size has not been reached.
|
|
||||||
batchSendDeadline = 5 * time.Second
|
|
||||||
)
|
|
||||||
|
|
||||||
// String constants for instrumentation.
|
// String constants for instrumentation.
|
||||||
const (
|
const (
|
||||||
namespace = "prometheus"
|
namespace = "prometheus"
|
||||||
|
@ -51,14 +42,28 @@ type StorageClient interface {
|
||||||
Name() string
|
Name() string
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type StorageQueueManagerConfig struct {
|
||||||
|
QueueCapacity int // Number of samples to buffer per shard before we start dropping them.
|
||||||
|
Shards int // Number of shards, i.e. amount of concurrency.
|
||||||
|
MaxSamplesPerSend int // Maximum number of samples per send.
|
||||||
|
BatchSendDeadline time.Duration // Maximum time sample will wait in buffer.
|
||||||
|
}
|
||||||
|
|
||||||
|
var defaultConfig = StorageQueueManagerConfig{
|
||||||
|
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 {
|
||||||
tsdb StorageClient
|
cfg StorageQueueManagerConfig
|
||||||
queue chan *model.Sample
|
tsdb StorageClient
|
||||||
pendingSamples model.Samples
|
shards []chan *model.Sample
|
||||||
sendSemaphore chan bool
|
wg sync.WaitGroup
|
||||||
drained chan bool
|
done chan struct{}
|
||||||
|
|
||||||
samplesCount *prometheus.CounterVec
|
samplesCount *prometheus.CounterVec
|
||||||
sendLatency prometheus.Summary
|
sendLatency prometheus.Summary
|
||||||
|
@ -69,16 +74,25 @@ type StorageQueueManager struct {
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewStorageQueueManager builds a new StorageQueueManager.
|
// NewStorageQueueManager builds a new StorageQueueManager.
|
||||||
func NewStorageQueueManager(tsdb StorageClient, queueCapacity int) *StorageQueueManager {
|
func NewStorageQueueManager(tsdb StorageClient, cfg *StorageQueueManagerConfig) *StorageQueueManager {
|
||||||
constLabels := prometheus.Labels{
|
constLabels := prometheus.Labels{
|
||||||
"type": tsdb.Name(),
|
"type": tsdb.Name(),
|
||||||
}
|
}
|
||||||
|
|
||||||
return &StorageQueueManager{
|
if cfg == nil {
|
||||||
tsdb: tsdb,
|
cfg = &defaultConfig
|
||||||
queue: make(chan *model.Sample, queueCapacity),
|
}
|
||||||
sendSemaphore: make(chan bool, maxConcurrentSends),
|
|
||||||
drained: make(chan bool),
|
shards := make([]chan *model.Sample, cfg.Shards)
|
||||||
|
for i := 0; i < cfg.Shards; i++ {
|
||||||
|
shards[i] = make(chan *model.Sample, cfg.QueueCapacity)
|
||||||
|
}
|
||||||
|
|
||||||
|
t := &StorageQueueManager{
|
||||||
|
cfg: *cfg,
|
||||||
|
tsdb: tsdb,
|
||||||
|
shards: shards,
|
||||||
|
done: make(chan struct{}),
|
||||||
|
|
||||||
samplesCount: prometheus.NewCounterVec(
|
samplesCount: prometheus.NewCounterVec(
|
||||||
prometheus.CounterOpts{
|
prometheus.CounterOpts{
|
||||||
|
@ -126,17 +140,23 @@ func NewStorageQueueManager(tsdb StorageClient, queueCapacity int) *StorageQueue
|
||||||
constLabels,
|
constLabels,
|
||||||
),
|
),
|
||||||
prometheus.GaugeValue,
|
prometheus.GaugeValue,
|
||||||
float64(queueCapacity),
|
float64(cfg.QueueCapacity),
|
||||||
),
|
),
|
||||||
}
|
}
|
||||||
|
|
||||||
|
t.wg.Add(cfg.Shards)
|
||||||
|
return t
|
||||||
}
|
}
|
||||||
|
|
||||||
// Append queues a sample to be sent to the remote storage. It drops the
|
// Append queues a sample to be sent to the remote storage. It drops the
|
||||||
// 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()
|
||||||
|
shard := uint64(fp) % uint64(t.cfg.Shards)
|
||||||
|
|
||||||
select {
|
select {
|
||||||
case t.queue <- s:
|
case t.shards[shard] <- s:
|
||||||
default:
|
default:
|
||||||
t.samplesCount.WithLabelValues(dropped).Inc()
|
t.samplesCount.WithLabelValues(dropped).Inc()
|
||||||
log.Warn("Remote storage queue full, discarding sample.")
|
log.Warn("Remote storage queue full, discarding sample.")
|
||||||
|
@ -144,16 +164,11 @@ func (t *StorageQueueManager) Append(s *model.Sample) error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Stop stops sending samples to the remote storage and waits for pending
|
// NeedsThrottling implements storage.SampleAppender. It will always return
|
||||||
// sends to complete.
|
// false as a remote storage drops samples on the floor if backlogging instead
|
||||||
func (t *StorageQueueManager) Stop() {
|
// of asking for throttling.
|
||||||
log.Infof("Stopping remote storage...")
|
func (*StorageQueueManager) NeedsThrottling() bool {
|
||||||
close(t.queue)
|
return false
|
||||||
<-t.drained
|
|
||||||
for i := 0; i < maxConcurrentSends; i++ {
|
|
||||||
t.sendSemaphore <- true
|
|
||||||
}
|
|
||||||
log.Info("Remote storage stopped.")
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Describe implements prometheus.Collector.
|
// Describe implements prometheus.Collector.
|
||||||
|
@ -166,79 +181,96 @@ func (t *StorageQueueManager) Describe(ch chan<- *prometheus.Desc) {
|
||||||
ch <- t.queueCapacity.Desc()
|
ch <- t.queueCapacity.Desc()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// QueueLength returns the number of outstanding samples in the queue.
|
||||||
|
func (t *StorageQueueManager) queueLen() int {
|
||||||
|
queueLength := 0
|
||||||
|
for _, shard := range t.shards {
|
||||||
|
queueLength += len(shard)
|
||||||
|
}
|
||||||
|
return queueLength
|
||||||
|
}
|
||||||
|
|
||||||
// Collect implements prometheus.Collector.
|
// Collect implements prometheus.Collector.
|
||||||
func (t *StorageQueueManager) Collect(ch chan<- prometheus.Metric) {
|
func (t *StorageQueueManager) Collect(ch chan<- prometheus.Metric) {
|
||||||
t.samplesCount.Collect(ch)
|
t.samplesCount.Collect(ch)
|
||||||
t.sendLatency.Collect(ch)
|
t.sendLatency.Collect(ch)
|
||||||
t.queueLength.Set(float64(len(t.queue)))
|
t.queueLength.Set(float64(t.queueLen()))
|
||||||
ch <- t.failedBatches
|
ch <- t.failedBatches
|
||||||
ch <- t.failedSamples
|
ch <- t.failedSamples
|
||||||
ch <- t.queueLength
|
ch <- t.queueLength
|
||||||
ch <- t.queueCapacity
|
ch <- t.queueCapacity
|
||||||
}
|
}
|
||||||
|
|
||||||
func (t *StorageQueueManager) sendSamples(s model.Samples) {
|
|
||||||
t.sendSemaphore <- true
|
|
||||||
|
|
||||||
go func() {
|
|
||||||
defer func() {
|
|
||||||
<-t.sendSemaphore
|
|
||||||
}()
|
|
||||||
|
|
||||||
// Samples are sent to the remote storage on a best-effort basis. If a
|
|
||||||
// sample isn't sent correctly the first time, it's simply dropped on the
|
|
||||||
// floor.
|
|
||||||
begin := time.Now()
|
|
||||||
err := t.tsdb.Store(s)
|
|
||||||
duration := time.Since(begin).Seconds()
|
|
||||||
|
|
||||||
labelValue := success
|
|
||||||
if err != nil {
|
|
||||||
log.Warnf("error sending %d samples to remote storage: %s", len(s), err)
|
|
||||||
labelValue = failure
|
|
||||||
t.failedBatches.Inc()
|
|
||||||
t.failedSamples.Add(float64(len(s)))
|
|
||||||
}
|
|
||||||
t.samplesCount.WithLabelValues(labelValue).Add(float64(len(s)))
|
|
||||||
t.sendLatency.Observe(duration)
|
|
||||||
}()
|
|
||||||
}
|
|
||||||
|
|
||||||
// Run continuously sends samples to the remote storage.
|
// Run continuously sends samples to the remote storage.
|
||||||
func (t *StorageQueueManager) Run() {
|
func (t *StorageQueueManager) Run() {
|
||||||
defer func() {
|
for i := 0; i < t.cfg.Shards; i++ {
|
||||||
close(t.drained)
|
go t.runShard(i)
|
||||||
}()
|
}
|
||||||
|
t.wg.Wait()
|
||||||
|
}
|
||||||
|
|
||||||
// Send batches of at most maxSamplesPerSend samples to the remote storage.
|
// Stop stops sending samples to the remote storage and waits for pending
|
||||||
|
// sends to complete.
|
||||||
|
func (t *StorageQueueManager) Stop() {
|
||||||
|
log.Infof("Stopping remote storage...")
|
||||||
|
for _, shard := range t.shards {
|
||||||
|
close(shard)
|
||||||
|
}
|
||||||
|
t.wg.Wait()
|
||||||
|
log.Info("Remote storage stopped.")
|
||||||
|
}
|
||||||
|
|
||||||
|
func (t *StorageQueueManager) runShard(i int) {
|
||||||
|
defer t.wg.Done()
|
||||||
|
shard := t.shards[i]
|
||||||
|
|
||||||
|
// Send batches of at most MaxSamplesPerSend samples to the remote storage.
|
||||||
// If we have fewer samples than that, flush them out after a deadline
|
// If we have fewer samples than that, flush them out after a deadline
|
||||||
// anyways.
|
// anyways.
|
||||||
|
pendingSamples := model.Samples{}
|
||||||
|
|
||||||
for {
|
for {
|
||||||
select {
|
select {
|
||||||
case s, ok := <-t.queue:
|
case s, ok := <-shard:
|
||||||
if !ok {
|
if !ok {
|
||||||
log.Infof("Flushing %d samples to remote storage...", len(t.pendingSamples))
|
if len(pendingSamples) > 0 {
|
||||||
t.flush()
|
log.Infof("Flushing %d samples to remote storage...", len(pendingSamples))
|
||||||
log.Infof("Done flushing.")
|
t.sendSamples(pendingSamples)
|
||||||
|
log.Infof("Done flushing.")
|
||||||
|
}
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
t.pendingSamples = append(t.pendingSamples, s)
|
pendingSamples = append(pendingSamples, s)
|
||||||
|
|
||||||
for len(t.pendingSamples) >= maxSamplesPerSend {
|
for len(pendingSamples) >= t.cfg.MaxSamplesPerSend {
|
||||||
t.sendSamples(t.pendingSamples[:maxSamplesPerSend])
|
t.sendSamples(pendingSamples[:t.cfg.MaxSamplesPerSend])
|
||||||
t.pendingSamples = t.pendingSamples[maxSamplesPerSend:]
|
pendingSamples = pendingSamples[t.cfg.MaxSamplesPerSend:]
|
||||||
|
}
|
||||||
|
case <-time.After(t.cfg.BatchSendDeadline):
|
||||||
|
if len(pendingSamples) > 0 {
|
||||||
|
t.sendSamples(pendingSamples)
|
||||||
|
pendingSamples = pendingSamples[:0]
|
||||||
}
|
}
|
||||||
case <-time.After(batchSendDeadline):
|
|
||||||
t.flush()
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Flush flushes remaining queued samples.
|
func (t *StorageQueueManager) sendSamples(s model.Samples) {
|
||||||
func (t *StorageQueueManager) flush() {
|
// Samples are sent to the remote storage on a best-effort basis. If a
|
||||||
if len(t.pendingSamples) > 0 {
|
// sample isn't sent correctly the first time, it's simply dropped on the
|
||||||
t.sendSamples(t.pendingSamples)
|
// floor.
|
||||||
|
begin := time.Now()
|
||||||
|
err := t.tsdb.Store(s)
|
||||||
|
duration := time.Since(begin).Seconds()
|
||||||
|
|
||||||
|
labelValue := success
|
||||||
|
if err != nil {
|
||||||
|
log.Warnf("error sending %d samples to remote storage: %s", len(s), err)
|
||||||
|
labelValue = failure
|
||||||
|
t.failedBatches.Inc()
|
||||||
|
t.failedSamples.Add(float64(len(s)))
|
||||||
}
|
}
|
||||||
t.pendingSamples = t.pendingSamples[:0]
|
t.samplesCount.WithLabelValues(labelValue).Add(float64(len(s)))
|
||||||
|
t.sendLatency.Observe(duration)
|
||||||
}
|
}
|
||||||
|
|
|
@ -14,6 +14,7 @@
|
||||||
package remote
|
package remote
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"fmt"
|
||||||
"sync"
|
"sync"
|
||||||
"sync/atomic"
|
"sync/atomic"
|
||||||
"testing"
|
"testing"
|
||||||
|
@ -23,28 +24,53 @@ import (
|
||||||
)
|
)
|
||||||
|
|
||||||
type TestStorageClient struct {
|
type TestStorageClient struct {
|
||||||
receivedSamples model.Samples
|
receivedSamples map[string]model.Samples
|
||||||
expectedSamples model.Samples
|
expectedSamples map[string]model.Samples
|
||||||
wg sync.WaitGroup
|
wg sync.WaitGroup
|
||||||
|
mtx sync.Mutex
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *TestStorageClient) expectSamples(s model.Samples) {
|
func NewTestStorageClient() *TestStorageClient {
|
||||||
c.expectedSamples = append(c.expectedSamples, s...)
|
return &TestStorageClient{
|
||||||
c.wg.Add(len(s))
|
receivedSamples: map[string]model.Samples{},
|
||||||
|
expectedSamples: map[string]model.Samples{},
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func (c *TestStorageClient) expectSamples(ss model.Samples) {
|
||||||
|
c.mtx.Lock()
|
||||||
|
defer c.mtx.Unlock()
|
||||||
|
|
||||||
|
for _, s := range ss {
|
||||||
|
ts := s.Metric.String()
|
||||||
|
c.expectedSamples[ts] = append(c.expectedSamples[ts], s)
|
||||||
|
}
|
||||||
|
c.wg.Add(len(ss))
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *TestStorageClient) waitForExpectedSamples(t *testing.T) {
|
func (c *TestStorageClient) waitForExpectedSamples(t *testing.T) {
|
||||||
c.wg.Wait()
|
c.wg.Wait()
|
||||||
for i, expected := range c.expectedSamples {
|
|
||||||
if !expected.Equal(c.receivedSamples[i]) {
|
c.mtx.Lock()
|
||||||
t.Fatalf("%d. Expected %v, got %v", i, expected, c.receivedSamples[i])
|
defer c.mtx.Unlock()
|
||||||
|
for ts, expectedSamples := range c.expectedSamples {
|
||||||
|
for i, expected := range expectedSamples {
|
||||||
|
if !expected.Equal(c.receivedSamples[ts][i]) {
|
||||||
|
t.Fatalf("%d. Expected %v, got %v", i, expected, c.receivedSamples[ts][i])
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (c *TestStorageClient) Store(s model.Samples) error {
|
func (c *TestStorageClient) Store(ss model.Samples) error {
|
||||||
c.receivedSamples = append(c.receivedSamples, s...)
|
c.mtx.Lock()
|
||||||
c.wg.Add(-len(s))
|
defer c.mtx.Unlock()
|
||||||
|
|
||||||
|
for _, s := range ss {
|
||||||
|
ts := s.Metric.String()
|
||||||
|
c.receivedSamples[ts] = append(c.receivedSamples[ts], s)
|
||||||
|
}
|
||||||
|
c.wg.Add(-len(ss))
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -55,21 +81,24 @@ 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.
|
||||||
n := maxSamplesPerSend * 2
|
cfg := defaultConfig
|
||||||
|
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++ {
|
||||||
|
name := model.LabelValue(fmt.Sprintf("test_metric_%d", i))
|
||||||
samples = append(samples, &model.Sample{
|
samples = append(samples, &model.Sample{
|
||||||
Metric: model.Metric{
|
Metric: model.Metric{
|
||||||
model.MetricNameLabel: "test_metric",
|
model.MetricNameLabel: name,
|
||||||
},
|
},
|
||||||
Value: model.SampleValue(i),
|
Value: model.SampleValue(i),
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
c := &TestStorageClient{}
|
c := NewTestStorageClient()
|
||||||
c.expectSamples(samples[:len(samples)/2])
|
c.expectSamples(samples[:len(samples)/2])
|
||||||
m := NewStorageQueueManager(c, len(samples)/2)
|
m := NewStorageQueueManager(c, &cfg)
|
||||||
|
|
||||||
// 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] {
|
||||||
|
@ -85,6 +114,39 @@ func TestSampleDelivery(t *testing.T) {
|
||||||
c.waitForExpectedSamples(t)
|
c.waitForExpectedSamples(t)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestSampleDeliveryOrder(t *testing.T) {
|
||||||
|
cfg := defaultConfig
|
||||||
|
ts := 10
|
||||||
|
n := cfg.MaxSamplesPerSend * ts
|
||||||
|
// Ensure we don't drop samples in this test.
|
||||||
|
cfg.QueueCapacity = n
|
||||||
|
|
||||||
|
samples := make(model.Samples, 0, n)
|
||||||
|
for i := 0; i < n; i++ {
|
||||||
|
name := model.LabelValue(fmt.Sprintf("test_metric_%d", i%ts))
|
||||||
|
samples = append(samples, &model.Sample{
|
||||||
|
Metric: model.Metric{
|
||||||
|
model.MetricNameLabel: name,
|
||||||
|
},
|
||||||
|
Value: model.SampleValue(i),
|
||||||
|
Timestamp: model.Time(i),
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
c := NewTestStorageClient()
|
||||||
|
c.expectSamples(samples)
|
||||||
|
m := NewStorageQueueManager(c, &cfg)
|
||||||
|
|
||||||
|
// These should be received by the client.
|
||||||
|
for _, s := range samples {
|
||||||
|
m.Append(s)
|
||||||
|
}
|
||||||
|
go m.Run()
|
||||||
|
defer m.Stop()
|
||||||
|
|
||||||
|
c.waitForExpectedSamples(t)
|
||||||
|
}
|
||||||
|
|
||||||
// TestBlockingStorageClient is a queue_manager StorageClient which will block
|
// TestBlockingStorageClient is a queue_manager StorageClient which will block
|
||||||
// on any calls to Store(), until the `block` channel is closed, at which point
|
// on any calls to Store(), until the `block` channel is closed, at which point
|
||||||
// the `numCalls` property will contain a count of how many times Store() was
|
// the `numCalls` property will contain a count of how many times Store() was
|
||||||
|
@ -121,24 +183,26 @@ func (c *TestBlockingStorageClient) Name() string {
|
||||||
|
|
||||||
func TestSpawnNotMoreThanMaxConcurrentSendsGoroutines(t *testing.T) {
|
func TestSpawnNotMoreThanMaxConcurrentSendsGoroutines(t *testing.T) {
|
||||||
// Our goal is to fully empty the queue:
|
// Our goal is to fully empty the queue:
|
||||||
// `maxSamplesPerSend*maxConcurrentSends` samples should be consumed by the
|
// `MaxSamplesPerSend*Shards` samples should be consumed by the
|
||||||
// semaphore-controlled goroutines, and then another `maxSamplesPerSend`
|
// per-shard goroutines, and then another `MaxSamplesPerSend`
|
||||||
// should be consumed by the Run() loop calling sendSample and immediately
|
// should be left on the queue.
|
||||||
// blocking.
|
cfg := defaultConfig
|
||||||
n := maxSamplesPerSend*maxConcurrentSends + maxSamplesPerSend
|
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++ {
|
||||||
|
name := model.LabelValue(fmt.Sprintf("test_metric_%d", i))
|
||||||
samples = append(samples, &model.Sample{
|
samples = append(samples, &model.Sample{
|
||||||
Metric: model.Metric{
|
Metric: model.Metric{
|
||||||
model.MetricNameLabel: "test_metric",
|
model.MetricNameLabel: name,
|
||||||
},
|
},
|
||||||
Value: model.SampleValue(i),
|
Value: model.SampleValue(i),
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
c := NewTestBlockedStorageClient()
|
c := NewTestBlockedStorageClient()
|
||||||
m := NewStorageQueueManager(c, n)
|
m := NewStorageQueueManager(c, &cfg)
|
||||||
|
|
||||||
go m.Run()
|
go m.Run()
|
||||||
|
|
||||||
|
@ -151,7 +215,7 @@ func TestSpawnNotMoreThanMaxConcurrentSendsGoroutines(t *testing.T) {
|
||||||
m.Append(s)
|
m.Append(s)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Wait until the Run() loop drains the queue. If things went right, it
|
// Wait until the runShard() loops drain the queue. If things went right, it
|
||||||
// should then immediately block in sendSamples(), but, in case of error,
|
// should then immediately block in sendSamples(), but, in case of error,
|
||||||
// it would spawn too many goroutines, and thus we'd see more calls to
|
// it would spawn too many goroutines, and thus we'd see more calls to
|
||||||
// client.Store()
|
// client.Store()
|
||||||
|
@ -163,19 +227,18 @@ func TestSpawnNotMoreThanMaxConcurrentSendsGoroutines(t *testing.T) {
|
||||||
// draining the queue. We cap the waiting at 1 second -- that should give
|
// draining the queue. We cap the waiting at 1 second -- that should give
|
||||||
// plenty of time, and keeps the failure fairly quick if we're not draining
|
// plenty of time, and keeps the failure fairly quick if we're not draining
|
||||||
// the queue properly.
|
// the queue properly.
|
||||||
for i := 0; i < 100 && len(m.queue) > 0; i++ {
|
for i := 0; i < 100 && m.queueLen() > 0; i++ {
|
||||||
time.Sleep(10 * time.Millisecond)
|
time.Sleep(10 * time.Millisecond)
|
||||||
}
|
}
|
||||||
|
|
||||||
if len(m.queue) > 0 {
|
if m.queueLen() != cfg.MaxSamplesPerSend {
|
||||||
t.Fatalf("Failed to drain StorageQueueManager queue, %d elements left",
|
t.Fatalf("Failed to drain StorageQueueManager queue, %d elements left",
|
||||||
len(m.queue),
|
m.queueLen(),
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
numCalls := c.NumCalls()
|
numCalls := c.NumCalls()
|
||||||
if numCalls != maxConcurrentSends {
|
if numCalls != uint64(cfg.Shards) {
|
||||||
t.Errorf("Saw %d concurrent sends, expected %d", numCalls, maxConcurrentSends)
|
t.Errorf("Saw %d concurrent sends, expected %d", numCalls, cfg.Shards)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -52,11 +52,11 @@ func New(o *Options) (*Storage, error) {
|
||||||
c := graphite.NewClient(
|
c := graphite.NewClient(
|
||||||
o.GraphiteAddress, o.GraphiteTransport,
|
o.GraphiteAddress, o.GraphiteTransport,
|
||||||
o.StorageTimeout, o.GraphitePrefix)
|
o.StorageTimeout, o.GraphitePrefix)
|
||||||
s.queues = append(s.queues, NewStorageQueueManager(c, 100*1024))
|
s.queues = append(s.queues, NewStorageQueueManager(c, nil))
|
||||||
}
|
}
|
||||||
if o.OpentsdbURL != "" {
|
if o.OpentsdbURL != "" {
|
||||||
c := opentsdb.NewClient(o.OpentsdbURL, o.StorageTimeout)
|
c := opentsdb.NewClient(o.OpentsdbURL, o.StorageTimeout)
|
||||||
s.queues = append(s.queues, NewStorageQueueManager(c, 100*1024))
|
s.queues = append(s.queues, NewStorageQueueManager(c, nil))
|
||||||
}
|
}
|
||||||
if o.InfluxdbURL != nil {
|
if o.InfluxdbURL != nil {
|
||||||
conf := influx.Config{
|
conf := influx.Config{
|
||||||
|
@ -67,7 +67,7 @@ func New(o *Options) (*Storage, error) {
|
||||||
}
|
}
|
||||||
c := influxdb.NewClient(conf, o.InfluxdbDatabase, o.InfluxdbRetentionPolicy)
|
c := influxdb.NewClient(conf, o.InfluxdbDatabase, o.InfluxdbRetentionPolicy)
|
||||||
prometheus.MustRegister(c)
|
prometheus.MustRegister(c)
|
||||||
s.queues = append(s.queues, NewStorageQueueManager(c, 100*1024))
|
s.queues = append(s.queues, NewStorageQueueManager(c, nil))
|
||||||
}
|
}
|
||||||
if o.Address != "" {
|
if o.Address != "" {
|
||||||
c, err := NewClient(o.Address, o.StorageTimeout)
|
c, err := NewClient(o.Address, o.StorageTimeout)
|
||||||
|
|
Loading…
Reference in a new issue