|
|
|
|
@ -14,6 +14,7 @@ |
|
|
|
|
package remote |
|
|
|
|
|
|
|
|
|
import ( |
|
|
|
|
"math" |
|
|
|
|
"sync" |
|
|
|
|
"time" |
|
|
|
|
|
|
|
|
|
@ -32,13 +33,25 @@ const ( |
|
|
|
|
subsystem = "remote_storage" |
|
|
|
|
queue = "queue" |
|
|
|
|
|
|
|
|
|
defaultShards = 10 |
|
|
|
|
// With a maximum of 500 shards, assuming an average of 100ms remote write
|
|
|
|
|
// time and 100 samples per batch, we will be able to push 500k samples/s.
|
|
|
|
|
defaultMaxShards = 500 |
|
|
|
|
defaultMaxSamplesPerSend = 100 |
|
|
|
|
// The queue capacity is per shard.
|
|
|
|
|
defaultQueueCapacity = 100 * 1024 / defaultShards |
|
|
|
|
|
|
|
|
|
// defaultQueueCapacity is per shard - at 500 shards, this will buffer
|
|
|
|
|
// 50m samples. It is configured to buffer 1024 batches, which at 100ms
|
|
|
|
|
// per batch is 1:40mins.
|
|
|
|
|
defaultQueueCapacity = defaultMaxSamplesPerSend * 1024 |
|
|
|
|
defaultBatchSendDeadline = 5 * time.Second |
|
|
|
|
logRateLimit = 0.1 // Limit to 1 log event every 10s
|
|
|
|
|
logBurst = 10 |
|
|
|
|
|
|
|
|
|
// We track samples in/out and how long pushes take using an Exponentially
|
|
|
|
|
// Weighted Moving Average.
|
|
|
|
|
ewmaWeight = 0.2 |
|
|
|
|
shardUpdateDuration = 10 * time.Second |
|
|
|
|
shardToleranceFraction = 0.3 // allow 30% too many shards before scaling down
|
|
|
|
|
|
|
|
|
|
logRateLimit = 0.1 // Limit to 1 log event every 10s
|
|
|
|
|
logBurst = 10 |
|
|
|
|
) |
|
|
|
|
|
|
|
|
|
var ( |
|
|
|
|
@ -97,6 +110,15 @@ var ( |
|
|
|
|
}, |
|
|
|
|
[]string{queue}, |
|
|
|
|
) |
|
|
|
|
numShards = prometheus.NewGaugeVec( |
|
|
|
|
prometheus.GaugeOpts{ |
|
|
|
|
Namespace: namespace, |
|
|
|
|
Subsystem: subsystem, |
|
|
|
|
Name: "shards_total", |
|
|
|
|
Help: "The number of shards used for parallel sending to the remote storage.", |
|
|
|
|
}, |
|
|
|
|
[]string{queue}, |
|
|
|
|
) |
|
|
|
|
) |
|
|
|
|
|
|
|
|
|
func init() { |
|
|
|
|
@ -106,6 +128,7 @@ func init() { |
|
|
|
|
prometheus.MustRegister(sentBatchDuration) |
|
|
|
|
prometheus.MustRegister(queueLength) |
|
|
|
|
prometheus.MustRegister(queueCapacity) |
|
|
|
|
prometheus.MustRegister(numShards) |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
// StorageClient defines an interface for sending a batch of samples to an
|
|
|
|
|
@ -120,7 +143,7 @@ type StorageClient interface { |
|
|
|
|
// QueueManagerConfig configures a storage queue.
|
|
|
|
|
type QueueManagerConfig 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.
|
|
|
|
|
MaxShards int // Max 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.
|
|
|
|
|
ExternalLabels model.LabelSet |
|
|
|
|
@ -132,11 +155,18 @@ type QueueManagerConfig struct { |
|
|
|
|
// indicated by the provided StorageClient.
|
|
|
|
|
type QueueManager struct { |
|
|
|
|
cfg QueueManagerConfig |
|
|
|
|
shards []chan *model.Sample |
|
|
|
|
wg sync.WaitGroup |
|
|
|
|
done chan struct{} |
|
|
|
|
queueName string |
|
|
|
|
logLimiter *rate.Limiter |
|
|
|
|
|
|
|
|
|
shardsMtx sync.Mutex |
|
|
|
|
shards *shards |
|
|
|
|
numShards int |
|
|
|
|
reshardChan chan int |
|
|
|
|
quit chan struct{} |
|
|
|
|
wg sync.WaitGroup |
|
|
|
|
|
|
|
|
|
samplesIn, samplesOut, samplesOutDuration ewmaRate |
|
|
|
|
integralAccumulator float64 |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
// NewQueueManager builds a new QueueManager.
|
|
|
|
|
@ -144,8 +174,8 @@ func NewQueueManager(cfg QueueManagerConfig) *QueueManager { |
|
|
|
|
if cfg.QueueCapacity == 0 { |
|
|
|
|
cfg.QueueCapacity = defaultQueueCapacity |
|
|
|
|
} |
|
|
|
|
if cfg.Shards == 0 { |
|
|
|
|
cfg.Shards = defaultShards |
|
|
|
|
if cfg.MaxShards == 0 { |
|
|
|
|
cfg.MaxShards = defaultMaxShards |
|
|
|
|
} |
|
|
|
|
if cfg.MaxSamplesPerSend == 0 { |
|
|
|
|
cfg.MaxSamplesPerSend = defaultMaxSamplesPerSend |
|
|
|
|
@ -154,21 +184,26 @@ func NewQueueManager(cfg QueueManagerConfig) *QueueManager { |
|
|
|
|
cfg.BatchSendDeadline = defaultBatchSendDeadline |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
shards := make([]chan *model.Sample, cfg.Shards) |
|
|
|
|
for i := 0; i < cfg.Shards; i++ { |
|
|
|
|
shards[i] = make(chan *model.Sample, cfg.QueueCapacity) |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
t := &QueueManager{ |
|
|
|
|
cfg: cfg, |
|
|
|
|
shards: shards, |
|
|
|
|
done: make(chan struct{}), |
|
|
|
|
queueName: cfg.Client.Name(), |
|
|
|
|
logLimiter: rate.NewLimiter(logRateLimit, logBurst), |
|
|
|
|
} |
|
|
|
|
cfg: cfg, |
|
|
|
|
queueName: cfg.Client.Name(), |
|
|
|
|
logLimiter: rate.NewLimiter(logRateLimit, logBurst), |
|
|
|
|
numShards: 1, |
|
|
|
|
reshardChan: make(chan int), |
|
|
|
|
quit: make(chan struct{}), |
|
|
|
|
|
|
|
|
|
samplesIn: newEWMARate(ewmaWeight, shardUpdateDuration), |
|
|
|
|
samplesOut: newEWMARate(ewmaWeight, shardUpdateDuration), |
|
|
|
|
samplesOutDuration: newEWMARate(ewmaWeight, shardUpdateDuration), |
|
|
|
|
} |
|
|
|
|
t.shards = t.newShards(1) |
|
|
|
|
numShards.WithLabelValues(t.queueName).Set(float64(1)) |
|
|
|
|
queueCapacity.WithLabelValues(t.queueName).Set(float64(t.cfg.QueueCapacity)) |
|
|
|
|
t.wg.Add(cfg.Shards) |
|
|
|
|
|
|
|
|
|
t.wg.Add(2) |
|
|
|
|
go t.updateShardsLoop() |
|
|
|
|
go t.reshardLoop() |
|
|
|
|
|
|
|
|
|
return t |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
@ -193,13 +228,13 @@ func (t *QueueManager) Append(s *model.Sample) error { |
|
|
|
|
return nil |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
fp := snew.Metric.FastFingerprint() |
|
|
|
|
shard := uint64(fp) % uint64(t.cfg.Shards) |
|
|
|
|
t.shardsMtx.Lock() |
|
|
|
|
enqueued := t.shards.enqueue(&snew) |
|
|
|
|
t.shardsMtx.Unlock() |
|
|
|
|
|
|
|
|
|
select { |
|
|
|
|
case t.shards[shard] <- &snew: |
|
|
|
|
if enqueued { |
|
|
|
|
queueLength.WithLabelValues(t.queueName).Inc() |
|
|
|
|
default: |
|
|
|
|
} else { |
|
|
|
|
droppedSamplesTotal.WithLabelValues(t.queueName).Inc() |
|
|
|
|
if t.logLimiter.Allow() { |
|
|
|
|
log.Warn("Remote storage queue full, discarding sample. Multiple subsequent messages of this kind may be suppressed.") |
|
|
|
|
@ -218,25 +253,181 @@ func (*QueueManager) NeedsThrottling() bool { |
|
|
|
|
// Start the queue manager sending samples to the remote storage.
|
|
|
|
|
// Does not block.
|
|
|
|
|
func (t *QueueManager) Start() { |
|
|
|
|
for i := 0; i < t.cfg.Shards; i++ { |
|
|
|
|
go t.runShard(i) |
|
|
|
|
} |
|
|
|
|
t.shardsMtx.Lock() |
|
|
|
|
defer t.shardsMtx.Unlock() |
|
|
|
|
t.shards.start() |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
// Stop stops sending samples to the remote storage and waits for pending
|
|
|
|
|
// sends to complete.
|
|
|
|
|
func (t *QueueManager) Stop() { |
|
|
|
|
log.Infof("Stopping remote storage...") |
|
|
|
|
for _, shard := range t.shards { |
|
|
|
|
close(shard) |
|
|
|
|
} |
|
|
|
|
close(t.quit) |
|
|
|
|
t.wg.Wait() |
|
|
|
|
t.shardsMtx.Lock() |
|
|
|
|
defer t.shardsMtx.Unlock() |
|
|
|
|
t.shards.stop() |
|
|
|
|
log.Info("Remote storage stopped.") |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (t *QueueManager) runShard(i int) { |
|
|
|
|
func (t *QueueManager) updateShardsLoop() { |
|
|
|
|
defer t.wg.Done() |
|
|
|
|
shard := t.shards[i] |
|
|
|
|
|
|
|
|
|
ticker := time.Tick(shardUpdateDuration) |
|
|
|
|
for { |
|
|
|
|
select { |
|
|
|
|
case <-ticker: |
|
|
|
|
t.caclulateDesiredShards() |
|
|
|
|
case <-t.quit: |
|
|
|
|
return |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (t *QueueManager) caclulateDesiredShards() { |
|
|
|
|
t.samplesIn.tick() |
|
|
|
|
t.samplesOut.tick() |
|
|
|
|
t.samplesOutDuration.tick() |
|
|
|
|
|
|
|
|
|
// We use the number of incoming samples as a prediction of how much work we
|
|
|
|
|
// will need to do next iteration. We add to this any pending samples
|
|
|
|
|
// (received - send) so we can catch up with any backlog. We use the average
|
|
|
|
|
// outgoing batch latency to work out how how many shards we need.
|
|
|
|
|
var ( |
|
|
|
|
samplesIn = t.samplesIn.rate() |
|
|
|
|
samplesOut = t.samplesOut.rate() |
|
|
|
|
samplesPending = samplesIn - samplesOut |
|
|
|
|
samplesOutDuration = t.samplesOutDuration.rate() |
|
|
|
|
) |
|
|
|
|
|
|
|
|
|
// We use an integral accumulator, like in a PID, to help dampen oscillation.
|
|
|
|
|
t.integralAccumulator = t.integralAccumulator + (samplesPending * 0.1) |
|
|
|
|
|
|
|
|
|
if samplesOut <= 0 { |
|
|
|
|
return |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
var ( |
|
|
|
|
timePerSample = samplesOutDuration / samplesOut |
|
|
|
|
desiredShards = (timePerSample * (samplesIn + samplesPending + t.integralAccumulator)) / float64(time.Second) |
|
|
|
|
) |
|
|
|
|
log.Debugf("QueueManager.caclulateDesiredShards samplesIn=%f, samplesOut=%f, samplesPending=%f, desiredShards=%f", |
|
|
|
|
samplesIn, samplesOut, samplesPending, desiredShards) |
|
|
|
|
|
|
|
|
|
// Changes in the number of shards must be greated than shardToleranceFraction.
|
|
|
|
|
var ( |
|
|
|
|
lowerBound = float64(t.numShards) * (1. - shardToleranceFraction) |
|
|
|
|
upperBound = float64(t.numShards) * (1. + shardToleranceFraction) |
|
|
|
|
) |
|
|
|
|
log.Debugf("QueueManager.updateShardsLoop %f <= %f <= %f", lowerBound, desiredShards, upperBound) |
|
|
|
|
if lowerBound <= desiredShards && desiredShards <= upperBound { |
|
|
|
|
return |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
numShards := int(math.Ceil(desiredShards)) |
|
|
|
|
if numShards > t.cfg.MaxShards { |
|
|
|
|
numShards = t.cfg.MaxShards |
|
|
|
|
} |
|
|
|
|
if numShards == t.numShards { |
|
|
|
|
return |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
// Resharding can take some time, and we want this loop
|
|
|
|
|
// to stay close to shardUpdateDuration.
|
|
|
|
|
select { |
|
|
|
|
case t.reshardChan <- numShards: |
|
|
|
|
log.Infof("Remote storage resharding from %d to %d shards.", t.numShards, numShards) |
|
|
|
|
t.numShards = numShards |
|
|
|
|
default: |
|
|
|
|
log.Infof("Currently resharding, skipping.") |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (t *QueueManager) reshardLoop() { |
|
|
|
|
defer t.wg.Done() |
|
|
|
|
|
|
|
|
|
for { |
|
|
|
|
select { |
|
|
|
|
case numShards := <-t.reshardChan: |
|
|
|
|
t.reshard(numShards) |
|
|
|
|
case <-t.quit: |
|
|
|
|
return |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (t *QueueManager) reshard(n int) { |
|
|
|
|
numShards.WithLabelValues(t.queueName).Set(float64(n)) |
|
|
|
|
|
|
|
|
|
t.shardsMtx.Lock() |
|
|
|
|
newShards := t.newShards(n) |
|
|
|
|
oldShards := t.shards |
|
|
|
|
t.shards = newShards |
|
|
|
|
t.shardsMtx.Unlock() |
|
|
|
|
|
|
|
|
|
oldShards.stop() |
|
|
|
|
|
|
|
|
|
// We start the newShards after we have stopped (the therefore completely
|
|
|
|
|
// flushed) the oldShards, to guarantee we only every deliver samples in
|
|
|
|
|
// order.
|
|
|
|
|
newShards.start() |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
type shards struct { |
|
|
|
|
qm *QueueManager |
|
|
|
|
queues []chan *model.Sample |
|
|
|
|
done chan struct{} |
|
|
|
|
wg sync.WaitGroup |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (t *QueueManager) newShards(numShards int) *shards { |
|
|
|
|
queues := make([]chan *model.Sample, numShards) |
|
|
|
|
for i := 0; i < numShards; i++ { |
|
|
|
|
queues[i] = make(chan *model.Sample, t.cfg.QueueCapacity) |
|
|
|
|
} |
|
|
|
|
s := &shards{ |
|
|
|
|
qm: t, |
|
|
|
|
queues: queues, |
|
|
|
|
done: make(chan struct{}), |
|
|
|
|
} |
|
|
|
|
s.wg.Add(numShards) |
|
|
|
|
return s |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (s *shards) len() int { |
|
|
|
|
return len(s.queues) |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (s *shards) start() { |
|
|
|
|
for i := 0; i < len(s.queues); i++ { |
|
|
|
|
go s.runShard(i) |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (s *shards) stop() { |
|
|
|
|
for _, shard := range s.queues { |
|
|
|
|
close(shard) |
|
|
|
|
} |
|
|
|
|
s.wg.Wait() |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (s *shards) enqueue(sample *model.Sample) bool { |
|
|
|
|
s.qm.samplesIn.incr(1) |
|
|
|
|
|
|
|
|
|
fp := sample.Metric.FastFingerprint() |
|
|
|
|
shard := uint64(fp) % uint64(len(s.queues)) |
|
|
|
|
|
|
|
|
|
select { |
|
|
|
|
case s.queues[shard] <- sample: |
|
|
|
|
return true |
|
|
|
|
default: |
|
|
|
|
return false |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (s *shards) runShard(i int) { |
|
|
|
|
defer s.wg.Done() |
|
|
|
|
queue := s.queues[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
|
|
|
|
|
@ -245,45 +436,48 @@ func (t *QueueManager) runShard(i int) { |
|
|
|
|
|
|
|
|
|
for { |
|
|
|
|
select { |
|
|
|
|
case s, ok := <-shard: |
|
|
|
|
case sample, ok := <-queue: |
|
|
|
|
if !ok { |
|
|
|
|
if len(pendingSamples) > 0 { |
|
|
|
|
log.Infof("Flushing %d samples to remote storage...", len(pendingSamples)) |
|
|
|
|
t.sendSamples(pendingSamples) |
|
|
|
|
log.Infof("Done flushing.") |
|
|
|
|
log.Debugf("Flushing %d samples to remote storage...", len(pendingSamples)) |
|
|
|
|
s.sendSamples(pendingSamples) |
|
|
|
|
log.Debugf("Done flushing.") |
|
|
|
|
} |
|
|
|
|
return |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
queueLength.WithLabelValues(t.queueName).Dec() |
|
|
|
|
pendingSamples = append(pendingSamples, s) |
|
|
|
|
queueLength.WithLabelValues(s.qm.queueName).Dec() |
|
|
|
|
pendingSamples = append(pendingSamples, sample) |
|
|
|
|
|
|
|
|
|
for len(pendingSamples) >= t.cfg.MaxSamplesPerSend { |
|
|
|
|
t.sendSamples(pendingSamples[:t.cfg.MaxSamplesPerSend]) |
|
|
|
|
pendingSamples = pendingSamples[t.cfg.MaxSamplesPerSend:] |
|
|
|
|
for len(pendingSamples) >= s.qm.cfg.MaxSamplesPerSend { |
|
|
|
|
s.sendSamples(pendingSamples[:s.qm.cfg.MaxSamplesPerSend]) |
|
|
|
|
pendingSamples = pendingSamples[s.qm.cfg.MaxSamplesPerSend:] |
|
|
|
|
} |
|
|
|
|
case <-time.After(t.cfg.BatchSendDeadline): |
|
|
|
|
case <-time.After(s.qm.cfg.BatchSendDeadline): |
|
|
|
|
if len(pendingSamples) > 0 { |
|
|
|
|
t.sendSamples(pendingSamples) |
|
|
|
|
s.sendSamples(pendingSamples) |
|
|
|
|
pendingSamples = pendingSamples[:0] |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (t *QueueManager) sendSamples(s model.Samples) { |
|
|
|
|
func (s *shards) sendSamples(samples model.Samples) { |
|
|
|
|
// 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.cfg.Client.Store(s) |
|
|
|
|
duration := time.Since(begin).Seconds() |
|
|
|
|
err := s.qm.cfg.Client.Store(samples) |
|
|
|
|
duration := time.Since(begin) |
|
|
|
|
|
|
|
|
|
if err != nil { |
|
|
|
|
log.Warnf("error sending %d samples to remote storage: %s", len(s), err) |
|
|
|
|
failedSamplesTotal.WithLabelValues(t.queueName).Add(float64(len(s))) |
|
|
|
|
log.Warnf("error sending %d samples to remote storage: %s", len(samples), err) |
|
|
|
|
failedSamplesTotal.WithLabelValues(s.qm.queueName).Add(float64(len(samples))) |
|
|
|
|
} else { |
|
|
|
|
sentSamplesTotal.WithLabelValues(t.queueName).Add(float64(len(s))) |
|
|
|
|
sentSamplesTotal.WithLabelValues(s.qm.queueName).Add(float64(len(samples))) |
|
|
|
|
} |
|
|
|
|
sentBatchDuration.WithLabelValues(t.queueName).Observe(duration) |
|
|
|
|
sentBatchDuration.WithLabelValues(s.qm.queueName).Observe(duration.Seconds()) |
|
|
|
|
|
|
|
|
|
s.qm.samplesOut.incr(int64(len(samples))) |
|
|
|
|
s.qm.samplesOutDuration.incr(int64(duration)) |
|
|
|
|
} |
|
|
|
|
|