Dynamically reshard the QueueManager based on observed load.

This commit is contained in:
Tom Wilkie 2017-03-07 14:36:40 +00:00
parent 5aa90c075b
commit 9d22f030cf
2 changed files with 301 additions and 54 deletions

53
storage/remote/ewma.go Normal file
View file

@ -0,0 +1,53 @@
package remote
import (
"sync"
"sync/atomic"
"time"
)
// ewmaRate tracks an exponentially weighted moving average of a per-second rate.
type ewmaRate struct {
newEvents int64
alpha float64
interval time.Duration
lastRate float64
init bool
mutex sync.Mutex
}
func newEWMARate(alpha float64, interval time.Duration) ewmaRate {
return ewmaRate{
alpha: alpha,
interval: interval,
}
}
// rate returns the per-second rate.
func (r *ewmaRate) rate() float64 {
r.mutex.Lock()
defer r.mutex.Unlock()
return r.lastRate
}
// tick assumes to be called every r.interval.
func (r *ewmaRate) tick() {
newEvents := atomic.LoadInt64(&r.newEvents)
atomic.AddInt64(&r.newEvents, -newEvents)
instantRate := float64(newEvents) / r.interval.Seconds()
r.mutex.Lock()
defer r.mutex.Unlock()
if r.init {
r.lastRate += r.alpha * (instantRate - r.lastRate)
} else {
r.init = true
r.lastRate = instantRate
}
}
// inc counts one event.
func (r *ewmaRate) incr(incr int64) {
atomic.AddInt64(&r.newEvents, incr)
}

View file

@ -14,6 +14,7 @@
package remote package remote
import ( import (
"math"
"sync" "sync"
"time" "time"
@ -32,13 +33,25 @@ const (
subsystem = "remote_storage" subsystem = "remote_storage"
queue = "queue" 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 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 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 ( var (
@ -97,6 +110,15 @@ var (
}, },
[]string{queue}, []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() { func init() {
@ -106,6 +128,7 @@ func init() {
prometheus.MustRegister(sentBatchDuration) prometheus.MustRegister(sentBatchDuration)
prometheus.MustRegister(queueLength) prometheus.MustRegister(queueLength)
prometheus.MustRegister(queueCapacity) prometheus.MustRegister(queueCapacity)
prometheus.MustRegister(numShards)
} }
// StorageClient defines an interface for sending a batch of samples to an // StorageClient defines an interface for sending a batch of samples to an
@ -120,7 +143,7 @@ type StorageClient interface {
// QueueManagerConfig configures a storage queue. // QueueManagerConfig configures a storage queue.
type QueueManagerConfig struct { type QueueManagerConfig 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. MaxShards int // Max 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 ExternalLabels model.LabelSet
@ -132,11 +155,18 @@ type QueueManagerConfig struct {
// indicated by the provided StorageClient. // indicated by the provided StorageClient.
type QueueManager struct { type QueueManager struct {
cfg QueueManagerConfig cfg QueueManagerConfig
shards []chan *model.Sample
wg sync.WaitGroup
done chan struct{}
queueName string queueName string
logLimiter *rate.Limiter 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. // NewQueueManager builds a new QueueManager.
@ -144,8 +174,8 @@ func NewQueueManager(cfg QueueManagerConfig) *QueueManager {
if cfg.QueueCapacity == 0 { if cfg.QueueCapacity == 0 {
cfg.QueueCapacity = defaultQueueCapacity cfg.QueueCapacity = defaultQueueCapacity
} }
if cfg.Shards == 0 { if cfg.MaxShards == 0 {
cfg.Shards = defaultShards cfg.MaxShards = defaultMaxShards
} }
if cfg.MaxSamplesPerSend == 0 { if cfg.MaxSamplesPerSend == 0 {
cfg.MaxSamplesPerSend = defaultMaxSamplesPerSend cfg.MaxSamplesPerSend = defaultMaxSamplesPerSend
@ -154,21 +184,26 @@ func NewQueueManager(cfg QueueManagerConfig) *QueueManager {
cfg.BatchSendDeadline = defaultBatchSendDeadline 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{ t := &QueueManager{
cfg: cfg, cfg: cfg,
shards: shards, queueName: cfg.Client.Name(),
done: make(chan struct{}), logLimiter: rate.NewLimiter(logRateLimit, logBurst),
queueName: cfg.Client.Name(), numShards: 1,
logLimiter: rate.NewLimiter(logRateLimit, logBurst), 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)) 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 return t
} }
@ -193,13 +228,13 @@ func (t *QueueManager) Append(s *model.Sample) error {
return nil return nil
} }
fp := snew.Metric.FastFingerprint() t.shardsMtx.Lock()
shard := uint64(fp) % uint64(t.cfg.Shards) enqueued := t.shards.enqueue(&snew)
t.shardsMtx.Unlock()
select { if enqueued {
case t.shards[shard] <- &snew:
queueLength.WithLabelValues(t.queueName).Inc() queueLength.WithLabelValues(t.queueName).Inc()
default: } else {
droppedSamplesTotal.WithLabelValues(t.queueName).Inc() droppedSamplesTotal.WithLabelValues(t.queueName).Inc()
if t.logLimiter.Allow() { if t.logLimiter.Allow() {
log.Warn("Remote storage queue full, discarding sample. Multiple subsequent messages of this kind may be suppressed.") 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. // Start the queue manager sending samples to the remote storage.
// Does not block. // Does not block.
func (t *QueueManager) Start() { func (t *QueueManager) Start() {
for i := 0; i < t.cfg.Shards; i++ { t.shardsMtx.Lock()
go t.runShard(i) defer t.shardsMtx.Unlock()
} t.shards.start()
} }
// Stop stops sending samples to the remote storage and waits for pending // Stop stops sending samples to the remote storage and waits for pending
// sends to complete. // sends to complete.
func (t *QueueManager) Stop() { func (t *QueueManager) Stop() {
log.Infof("Stopping remote storage...") log.Infof("Stopping remote storage...")
for _, shard := range t.shards { close(t.quit)
close(shard)
}
t.wg.Wait() t.wg.Wait()
t.shardsMtx.Lock()
defer t.shardsMtx.Unlock()
t.shards.stop()
log.Info("Remote storage stopped.") log.Info("Remote storage stopped.")
} }
func (t *QueueManager) runShard(i int) { func (t *QueueManager) updateShardsLoop() {
defer t.wg.Done() 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. // 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
@ -245,45 +436,48 @@ func (t *QueueManager) runShard(i int) {
for { for {
select { select {
case s, ok := <-shard: case sample, ok := <-queue:
if !ok { if !ok {
if len(pendingSamples) > 0 { if len(pendingSamples) > 0 {
log.Infof("Flushing %d samples to remote storage...", len(pendingSamples)) log.Debugf("Flushing %d samples to remote storage...", len(pendingSamples))
t.sendSamples(pendingSamples) s.sendSamples(pendingSamples)
log.Infof("Done flushing.") log.Debugf("Done flushing.")
} }
return return
} }
queueLength.WithLabelValues(t.queueName).Dec() queueLength.WithLabelValues(s.qm.queueName).Dec()
pendingSamples = append(pendingSamples, s) pendingSamples = append(pendingSamples, sample)
for len(pendingSamples) >= t.cfg.MaxSamplesPerSend { for len(pendingSamples) >= s.qm.cfg.MaxSamplesPerSend {
t.sendSamples(pendingSamples[:t.cfg.MaxSamplesPerSend]) s.sendSamples(pendingSamples[:s.qm.cfg.MaxSamplesPerSend])
pendingSamples = pendingSamples[t.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 { if len(pendingSamples) > 0 {
t.sendSamples(pendingSamples) s.sendSamples(pendingSamples)
pendingSamples = pendingSamples[:0] 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 // 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 // sample isn't sent correctly the first time, it's simply dropped on the
// floor. // floor.
begin := time.Now() begin := time.Now()
err := t.cfg.Client.Store(s) err := s.qm.cfg.Client.Store(samples)
duration := time.Since(begin).Seconds() duration := time.Since(begin)
if err != nil { if err != nil {
log.Warnf("error sending %d samples to remote storage: %s", len(s), err) log.Warnf("error sending %d samples to remote storage: %s", len(samples), err)
failedSamplesTotal.WithLabelValues(t.queueName).Add(float64(len(s))) failedSamplesTotal.WithLabelValues(s.qm.queueName).Add(float64(len(samples)))
} else { } 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))
} }