@ -32,6 +32,7 @@ import (
"go.uber.org/atomic"
"github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/model/histogram"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/model/relabel"
"github.com/prometheus/prometheus/prompb"
@ -54,30 +55,35 @@ const (
type queueManagerMetrics struct {
reg prometheus . Registerer
samplesTotal prometheus . Counter
exemplarsTotal prometheus . Counter
metadataTotal prometheus . Counter
failedSamplesTotal prometheus . Counter
failedExemplarsTotal prometheus . Counter
failedMetadataTotal prometheus . Counter
retriedSamplesTotal prometheus . Counter
retriedExemplarsTotal prometheus . Counter
retriedMetadataTotal prometheus . Counter
droppedSamplesTotal prometheus . Counter
droppedExemplarsTotal prometheus . Counter
enqueueRetriesTotal prometheus . Counter
sentBatchDuration prometheus . Histogram
highestSentTimestamp * maxTimestamp
pendingSamples prometheus . Gauge
pendingExemplars prometheus . Gauge
shardCapacity prometheus . Gauge
numShards prometheus . Gauge
maxNumShards prometheus . Gauge
minNumShards prometheus . Gauge
desiredNumShards prometheus . Gauge
sentBytesTotal prometheus . Counter
metadataBytesTotal prometheus . Counter
maxSamplesPerSend prometheus . Gauge
samplesTotal prometheus . Counter
exemplarsTotal prometheus . Counter
histogramsTotal prometheus . Counter
metadataTotal prometheus . Counter
failedSamplesTotal prometheus . Counter
failedExemplarsTotal prometheus . Counter
failedHistogramsTotal prometheus . Counter
failedMetadataTotal prometheus . Counter
retriedSamplesTotal prometheus . Counter
retriedExemplarsTotal prometheus . Counter
retriedHistogramsTotal prometheus . Counter
retriedMetadataTotal prometheus . Counter
droppedSamplesTotal prometheus . Counter
droppedExemplarsTotal prometheus . Counter
droppedHistogramsTotal prometheus . Counter
enqueueRetriesTotal prometheus . Counter
sentBatchDuration prometheus . Histogram
highestSentTimestamp * maxTimestamp
pendingSamples prometheus . Gauge
pendingExemplars prometheus . Gauge
pendingHistograms prometheus . Gauge
shardCapacity prometheus . Gauge
numShards prometheus . Gauge
maxNumShards prometheus . Gauge
minNumShards prometheus . Gauge
desiredNumShards prometheus . Gauge
sentBytesTotal prometheus . Counter
metadataBytesTotal prometheus . Counter
maxSamplesPerSend prometheus . Gauge
}
func newQueueManagerMetrics ( r prometheus . Registerer , rn , e string ) * queueManagerMetrics {
@ -103,6 +109,13 @@ func newQueueManagerMetrics(r prometheus.Registerer, rn, e string) *queueManager
Help : "Total number of exemplars sent to remote storage." ,
ConstLabels : constLabels ,
} )
m . histogramsTotal = prometheus . NewCounter ( prometheus . CounterOpts {
Namespace : namespace ,
Subsystem : subsystem ,
Name : "histograms_total" ,
Help : "Total number of histograms sent to remote storage." ,
ConstLabels : constLabels ,
} )
m . metadataTotal = prometheus . NewCounter ( prometheus . CounterOpts {
Namespace : namespace ,
Subsystem : subsystem ,
@ -124,6 +137,13 @@ func newQueueManagerMetrics(r prometheus.Registerer, rn, e string) *queueManager
Help : "Total number of exemplars which failed on send to remote storage, non-recoverable errors." ,
ConstLabels : constLabels ,
} )
m . failedHistogramsTotal = prometheus . NewCounter ( prometheus . CounterOpts {
Namespace : namespace ,
Subsystem : subsystem ,
Name : "histograms_failed_total" ,
Help : "Total number of histograms which failed on send to remote storage, non-recoverable errors." ,
ConstLabels : constLabels ,
} )
m . failedMetadataTotal = prometheus . NewCounter ( prometheus . CounterOpts {
Namespace : namespace ,
Subsystem : subsystem ,
@ -145,6 +165,13 @@ func newQueueManagerMetrics(r prometheus.Registerer, rn, e string) *queueManager
Help : "Total number of exemplars which failed on send to remote storage but were retried because the send error was recoverable." ,
ConstLabels : constLabels ,
} )
m . retriedHistogramsTotal = prometheus . NewCounter ( prometheus . CounterOpts {
Namespace : namespace ,
Subsystem : subsystem ,
Name : "histograms_retried_total" ,
Help : "Total number of histograms which failed on send to remote storage but were retried because the send error was recoverable." ,
ConstLabels : constLabels ,
} )
m . retriedMetadataTotal = prometheus . NewCounter ( prometheus . CounterOpts {
Namespace : namespace ,
Subsystem : subsystem ,
@ -166,6 +193,13 @@ func newQueueManagerMetrics(r prometheus.Registerer, rn, e string) *queueManager
Help : "Total number of exemplars which were dropped after being read from the WAL before being sent via remote write, either via relabelling or unintentionally because of an unknown reference ID." ,
ConstLabels : constLabels ,
} )
m . droppedHistogramsTotal = prometheus . NewCounter ( prometheus . CounterOpts {
Namespace : namespace ,
Subsystem : subsystem ,
Name : "histograms_dropped_total" ,
Help : "Total number of histograms which were dropped after being read from the WAL before being sent via remote write, either via relabelling or unintentionally because of an unknown reference ID." ,
ConstLabels : constLabels ,
} )
m . enqueueRetriesTotal = prometheus . NewCounter ( prometheus . CounterOpts {
Namespace : namespace ,
Subsystem : subsystem ,
@ -204,6 +238,13 @@ func newQueueManagerMetrics(r prometheus.Registerer, rn, e string) *queueManager
Help : "The number of exemplars pending in the queues shards to be sent to the remote storage." ,
ConstLabels : constLabels ,
} )
m . pendingHistograms = prometheus . NewGauge ( prometheus . GaugeOpts {
Namespace : namespace ,
Subsystem : subsystem ,
Name : "histograms_pending" ,
Help : "The number of histograms pending in the queues shards to be sent to the remote storage." ,
ConstLabels : constLabels ,
} )
m . shardCapacity = prometheus . NewGauge ( prometheus . GaugeOpts {
Namespace : namespace ,
Subsystem : subsystem ,
@ -269,20 +310,25 @@ func (m *queueManagerMetrics) register() {
m . reg . MustRegister (
m . samplesTotal ,
m . exemplarsTotal ,
m . histogramsTotal ,
m . metadataTotal ,
m . failedSamplesTotal ,
m . failedExemplarsTotal ,
m . failedHistogramsTotal ,
m . failedMetadataTotal ,
m . retriedSamplesTotal ,
m . retriedExemplarsTotal ,
m . retriedHistogramsTotal ,
m . retriedMetadataTotal ,
m . droppedSamplesTotal ,
m . droppedExemplarsTotal ,
m . droppedHistogramsTotal ,
m . enqueueRetriesTotal ,
m . sentBatchDuration ,
m . highestSentTimestamp ,
m . pendingSamples ,
m . pendingExemplars ,
m . pendingHistograms ,
m . shardCapacity ,
m . numShards ,
m . maxNumShards ,
@ -299,20 +345,25 @@ func (m *queueManagerMetrics) unregister() {
if m . reg != nil {
m . reg . Unregister ( m . samplesTotal )
m . reg . Unregister ( m . exemplarsTotal )
m . reg . Unregister ( m . histogramsTotal )
m . reg . Unregister ( m . metadataTotal )
m . reg . Unregister ( m . failedSamplesTotal )
m . reg . Unregister ( m . failedExemplarsTotal )
m . reg . Unregister ( m . failedHistogramsTotal )
m . reg . Unregister ( m . failedMetadataTotal )
m . reg . Unregister ( m . retriedSamplesTotal )
m . reg . Unregister ( m . retriedExemplarsTotal )
m . reg . Unregister ( m . retriedHistogramsTotal )
m . reg . Unregister ( m . retriedMetadataTotal )
m . reg . Unregister ( m . droppedSamplesTotal )
m . reg . Unregister ( m . droppedExemplarsTotal )
m . reg . Unregister ( m . droppedHistogramsTotal )
m . reg . Unregister ( m . enqueueRetriesTotal )
m . reg . Unregister ( m . sentBatchDuration )
m . reg . Unregister ( m . highestSentTimestamp )
m . reg . Unregister ( m . pendingSamples )
m . reg . Unregister ( m . pendingExemplars )
m . reg . Unregister ( m . pendingHistograms )
m . reg . Unregister ( m . shardCapacity )
m . reg . Unregister ( m . numShards )
m . reg . Unregister ( m . maxNumShards )
@ -341,15 +392,16 @@ type WriteClient interface {
type QueueManager struct {
lastSendTimestamp atomic . Int64
logger log . Logger
flushDeadline time . Duration
cfg config . QueueConfig
mcfg config . MetadataConfig
externalLabels labels . Labels
relabelConfigs [ ] * relabel . Config
sendExemplars bool
watcher * wal . Watcher
metadataWatcher * MetadataWatcher
logger log . Logger
flushDeadline time . Duration
cfg config . QueueConfig
mcfg config . MetadataConfig
externalLabels labels . Labels
relabelConfigs [ ] * relabel . Config
sendExemplars bool
sendNativeHistograms bool
watcher * wal . Watcher
metadataWatcher * MetadataWatcher
clientMtx sync . RWMutex
storeClient WriteClient
@ -396,6 +448,7 @@ func NewQueueManager(
highestRecvTimestamp * maxTimestamp ,
sm ReadyScrapeManager ,
enableExemplarRemoteWrite bool ,
enableNativeHistogramRemoteWrite bool ,
) * QueueManager {
if logger == nil {
logger = log . NewNopLogger ( )
@ -403,14 +456,15 @@ func NewQueueManager(
logger = log . With ( logger , remoteName , client . Name ( ) , endpoint , client . Endpoint ( ) )
t := & QueueManager {
logger : logger ,
flushDeadline : flushDeadline ,
cfg : cfg ,
mcfg : mCfg ,
externalLabels : externalLabels ,
relabelConfigs : relabelConfigs ,
storeClient : client ,
sendExemplars : enableExemplarRemoteWrite ,
logger : logger ,
flushDeadline : flushDeadline ,
cfg : cfg ,
mcfg : mCfg ,
externalLabels : externalLabels ,
relabelConfigs : relabelConfigs ,
storeClient : client ,
sendExemplars : enableExemplarRemoteWrite ,
sendNativeHistograms : enableNativeHistogramRemoteWrite ,
seriesLabels : make ( map [ chunks . HeadSeriesRef ] labels . Labels ) ,
seriesSegmentIndexes : make ( map [ chunks . HeadSeriesRef ] int ) ,
@ -430,7 +484,7 @@ func NewQueueManager(
highestRecvTimestamp : highestRecvTimestamp ,
}
t . watcher = wal . NewWatcher ( watcherMetrics , readerMetrics , logger , client . Name ( ) , t , dir , enableExemplarRemoteWrite )
t . watcher = wal . NewWatcher ( watcherMetrics , readerMetrics , logger , client . Name ( ) , t , dir , enableExemplarRemoteWrite , enableNativeHistogramRemoteWrite )
if t . mcfg . Send {
t . metadataWatcher = NewMetadataWatcher ( logger , sm , client . Name ( ) , t , t . mcfg . SendInterval , flushDeadline )
}
@ -538,11 +592,11 @@ outer:
return false
default :
}
if t . shards . enqueue ( s . Ref , sampleOrExemplar {
if t . shards . enqueue ( s . Ref , timeSeries {
seriesLabels : lbls ,
timestamp : s . T ,
value : s . V ,
isSample : tru e,
sType : tSampl e,
} ) {
continue outer
}
@ -588,11 +642,59 @@ outer:
return false
default :
}
if t . shards . enqueue ( e . Ref , sampleOrExemplar {
if t . shards . enqueue ( e . Ref , timeSeries {
seriesLabels : lbls ,
timestamp : e . T ,
value : e . V ,
exemplarLabels : e . Labels ,
sType : tExemplar ,
} ) {
continue outer
}
t . metrics . enqueueRetriesTotal . Inc ( )
time . Sleep ( time . Duration ( backoff ) )
backoff = backoff * 2
if backoff > t . cfg . MaxBackoff {
backoff = t . cfg . MaxBackoff
}
}
}
return true
}
func ( t * QueueManager ) AppendHistograms ( histograms [ ] record . RefHistogram ) bool {
if ! t . sendNativeHistograms {
return true
}
outer :
for _ , h := range histograms {
t . seriesMtx . Lock ( )
lbls , ok := t . seriesLabels [ h . Ref ]
if ! ok {
t . metrics . droppedHistogramsTotal . Inc ( )
t . dataDropped . incr ( 1 )
if _ , ok := t . droppedSeries [ h . Ref ] ; ! ok {
level . Info ( t . logger ) . Log ( "msg" , "Dropped histogram for series that was not explicitly dropped via relabelling" , "ref" , h . Ref )
}
t . seriesMtx . Unlock ( )
continue
}
t . seriesMtx . Unlock ( )
backoff := model . Duration ( 5 * time . Millisecond )
for {
select {
case <- t . quit :
return false
default :
}
if t . shards . enqueue ( h . Ref , timeSeries {
seriesLabels : lbls ,
timestamp : h . T ,
histogram : h . H ,
sType : tHistogram ,
} ) {
continue outer
}
@ -921,8 +1023,9 @@ type shards struct {
qm * QueueManager
queues [ ] * queue
// So we can accurately track how many of each are lost during shard shutdowns.
enqueuedSamples atomic . Int64
enqueuedExemplars atomic . Int64
enqueuedSamples atomic . Int64
enqueuedExemplars atomic . Int64
enqueuedHistograms atomic . Int64
// Emulate a wait group with a channel and an atomic int, as you
// cannot select on a wait group.
@ -934,9 +1037,10 @@ type shards struct {
// Hard shutdown context is used to terminate outgoing HTTP connections
// after giving them a chance to terminate.
hardShutdown context . CancelFunc
samplesDroppedOnHardShutdown atomic . Uint32
exemplarsDroppedOnHardShutdown atomic . Uint32
hardShutdown context . CancelFunc
samplesDroppedOnHardShutdown atomic . Uint32
exemplarsDroppedOnHardShutdown atomic . Uint32
histogramsDroppedOnHardShutdown atomic . Uint32
}
// start the shards; must be called before any call to enqueue.
@ -961,8 +1065,10 @@ func (s *shards) start(n int) {
s . done = make ( chan struct { } )
s . enqueuedSamples . Store ( 0 )
s . enqueuedExemplars . Store ( 0 )
s . enqueuedHistograms . Store ( 0 )
s . samplesDroppedOnHardShutdown . Store ( 0 )
s . exemplarsDroppedOnHardShutdown . Store ( 0 )
s . histogramsDroppedOnHardShutdown . Store ( 0 )
for i := 0 ; i < n ; i ++ {
go s . runShard ( hardShutdownCtx , i , newQueues [ i ] )
}
@ -1008,7 +1114,7 @@ func (s *shards) stop() {
// retry. A shard is full when its configured capacity has been reached,
// specifically, when s.queues[shard] has filled its batchQueue channel and the
// partial batch has also been filled.
func ( s * shards ) enqueue ( ref chunks . HeadSeriesRef , data sampleOrExemplar ) bool {
func ( s * shards ) enqueue ( ref chunks . HeadSeriesRef , data timeSeries ) bool {
s . mtx . RLock ( )
defer s . mtx . RUnlock ( )
@ -1021,12 +1127,16 @@ func (s *shards) enqueue(ref chunks.HeadSeriesRef, data sampleOrExemplar) bool {
if ! appended {
return false
}
if data . isSample {
switch data . sType {
case tSample :
s . qm . metrics . pendingSamples . Inc ( )
s . enqueuedSamples . Inc ( )
} else {
case tExemplar :
s . qm . metrics . pendingExemplars . Inc ( )
s . enqueuedExemplars . Inc ( )
case tHistogram :
s . qm . metrics . pendingHistograms . Inc ( )
s . enqueuedHistograms . Inc ( )
}
return true
}
@ -1035,24 +1145,34 @@ func (s *shards) enqueue(ref chunks.HeadSeriesRef, data sampleOrExemplar) bool {
type queue struct {
// batchMtx covers operations appending to or publishing the partial batch.
batchMtx sync . Mutex
batch [ ] sampleOrExemplar
batchQueue chan [ ] sampleOrExemplar
batch [ ] timeSeries
batchQueue chan [ ] timeSeries
// Since we know there are a limited number of batches out, using a stack
// is easy and safe so a sync.Pool is not necessary.
// poolMtx covers adding and removing batches from the batchPool.
poolMtx sync . Mutex
batchPool [ ] [ ] sampleOrExemplar
batchPool [ ] [ ] timeSeries
}
type sampleOrExemplar struct {
type timeSeries struct {
seriesLabels labels . Labels
value float64
histogram * histogram . Histogram
timestamp int64
exemplarLabels labels . Labels
isSample bool
// The type of series: sample, exemplar, or histogram.
sType seriesType
}
type seriesType int
const (
tSample seriesType = iota
tExemplar
tHistogram
)
func newQueue ( batchSize , capacity int ) * queue {
batches := capacity / batchSize
// Always create an unbuffered channel even if capacity is configured to be
@ -1061,17 +1181,17 @@ func newQueue(batchSize, capacity int) *queue {
batches = 1
}
return & queue {
batch : make ( [ ] sampleOrExemplar , 0 , batchSize ) ,
batchQueue : make ( chan [ ] sampleOrExemplar , batches ) ,
batch : make ( [ ] timeSeries , 0 , batchSize ) ,
batchQueue : make ( chan [ ] timeSeries , batches ) ,
// batchPool should have capacity for everything in the channel + 1 for
// the batch being processed.
batchPool : make ( [ ] [ ] sampleOrExemplar , 0 , batches + 1 ) ,
batchPool : make ( [ ] [ ] timeSeries , 0 , batches + 1 ) ,
}
}
// Append the sampleOrExemplar to the buffered batch. Returns false if it
// Append the timeSeries to the buffered batch. Returns false if it
// cannot be added and must be retried.
func ( q * queue ) Append ( datum sampleOrExemplar ) bool {
func ( q * queue ) Append ( datum timeSeries ) bool {
q . batchMtx . Lock ( )
defer q . batchMtx . Unlock ( )
q . batch = append ( q . batch , datum )
@ -1089,12 +1209,12 @@ func (q *queue) Append(datum sampleOrExemplar) bool {
return true
}
func ( q * queue ) Chan ( ) <- chan [ ] sampleOrExemplar {
func ( q * queue ) Chan ( ) <- chan [ ] timeSeries {
return q . batchQueue
}
// Batch returns the current batch and allocates a new batch.
func ( q * queue ) Batch ( ) [ ] sampleOrExemplar {
func ( q * queue ) Batch ( ) [ ] timeSeries {
q . batchMtx . Lock ( )
defer q . batchMtx . Unlock ( )
@ -1109,7 +1229,7 @@ func (q *queue) Batch() []sampleOrExemplar {
}
// ReturnForReuse adds the batch buffer back to the internal pool.
func ( q * queue ) ReturnForReuse ( batch [ ] sampleOrExemplar ) {
func ( q * queue ) ReturnForReuse ( batch [ ] timeSeries ) {
q . poolMtx . Lock ( )
defer q . poolMtx . Unlock ( )
if len ( q . batchPool ) < cap ( q . batchPool ) {
@ -1149,7 +1269,7 @@ func (q *queue) tryEnqueueingBatch(done <-chan struct{}) bool {
}
}
func ( q * queue ) newBatch ( capacity int ) [ ] sampleOrExemplar {
func ( q * queue ) newBatch ( capacity int ) [ ] timeSeries {
q . poolMtx . Lock ( )
defer q . poolMtx . Unlock ( )
batches := len ( q . batchPool )
@ -1158,7 +1278,7 @@ func (q *queue) newBatch(capacity int) []sampleOrExemplar {
q . batchPool = q . batchPool [ : batches - 1 ]
return batch
}
return make ( [ ] sampleOrExemplar , 0 , capacity )
return make ( [ ] timeSeries , 0 , capacity )
}
func ( s * shards ) runShard ( ctx context . Context , shardID int , queue * queue ) {
@ -1209,22 +1329,26 @@ func (s *shards) runShard(ctx context.Context, shardID int, queue *queue) {
// Remove them from pending and mark them as failed.
droppedSamples := int ( s . enqueuedSamples . Load ( ) )
droppedExemplars := int ( s . enqueuedExemplars . Load ( ) )
droppedHistograms := int ( s . enqueuedHistograms . Load ( ) )
s . qm . metrics . pendingSamples . Sub ( float64 ( droppedSamples ) )
s . qm . metrics . pendingExemplars . Sub ( float64 ( droppedExemplars ) )
s . qm . metrics . pendingHistograms . Sub ( float64 ( droppedHistograms ) )
s . qm . metrics . failedSamplesTotal . Add ( float64 ( droppedSamples ) )
s . qm . metrics . failedExemplarsTotal . Add ( float64 ( droppedExemplars ) )
s . qm . metrics . failedHistogramsTotal . Add ( float64 ( droppedHistograms ) )
s . samplesDroppedOnHardShutdown . Add ( uint32 ( droppedSamples ) )
s . exemplarsDroppedOnHardShutdown . Add ( uint32 ( droppedExemplars ) )
s . histogramsDroppedOnHardShutdown . Add ( uint32 ( droppedHistograms ) )
return
case batch , ok := <- batchQueue :
if ! ok {
return
}
nPendingSamples , nPendingExemplars := s . populateTimeSeries ( batch , pendingData )
nPendingSamples , nPendingExemplars , nPendingHistograms := s . populateTimeSeries ( batch , pendingData )
queue . ReturnForReuse ( batch )
n := nPendingSamples + nPendingExemplars
s . sendSamples ( ctx , pendingData [ : n ] , nPendingSamples , nPendingExemplars , pBuf , & buf )
n := nPendingSamples + nPendingExemplars + nPendingHistograms
s . sendSamples ( ctx , pendingData [ : n ] , nPendingSamples , nPendingExemplars , nPendingHistograms , pBuf , & buf )
stop ( )
timer . Reset ( time . Duration ( s . qm . cfg . BatchSendDeadline ) )
@ -1232,10 +1356,10 @@ func (s *shards) runShard(ctx context.Context, shardID int, queue *queue) {
case <- timer . C :
batch := queue . Batch ( )
if len ( batch ) > 0 {
nPendingSamples , nPendingExemplars := s . populateTimeSeries ( batch , pendingData )
n := nPendingSamples + nPendingExemplars
nPendingSamples , nPendingExemplars , nPendingHistograms := s . populateTimeSeries ( batch , pendingData )
n := nPendingSamples + nPendingExemplars + nPendingHistograms
level . Debug ( s . qm . logger ) . Log ( "msg" , "runShard timer ticked, sending buffered data" , "samples" , nPendingSamples , "exemplars" , nPendingExemplars , "shard" , shardNum )
s . sendSamples ( ctx , pendingData [ : n ] , nPendingSamples , nPendingExemplars , pBuf , & buf )
s . sendSamples ( ctx , pendingData [ : n ] , nPendingSamples , nPendingExemplars , nPendingHistograms , pBuf , & buf )
}
queue . ReturnForReuse ( batch )
timer . Reset ( time . Duration ( s . qm . cfg . BatchSendDeadline ) )
@ -1243,43 +1367,51 @@ func (s *shards) runShard(ctx context.Context, shardID int, queue *queue) {
}
}
func ( s * shards ) populateTimeSeries ( batch [ ] sampleOrExemplar , pendingData [ ] prompb . TimeSeries ) ( int , int ) {
var nPendingSamples , nPendingExemplars int
func ( s * shards ) populateTimeSeries ( batch [ ] timeSeries , pendingData [ ] prompb . TimeSeries ) ( int , int , int ) {
var nPendingSamples , nPendingExemplars , nPendingHistograms int
for nPending , d := range batch {
pendingData [ nPending ] . Samples = pendingData [ nPending ] . Samples [ : 0 ]
if s . qm . sendExemplars {
pendingData [ nPending ] . Exemplars = pendingData [ nPending ] . Exemplars [ : 0 ]
}
if s . qm . sendNativeHistograms {
pendingData [ nPending ] . Histograms = pendingData [ nPending ] . Histograms [ : 0 ]
}
// Number of pending samples is limited by the fact that sendSamples (via sendSamplesWithBackoff)
// retries endlessly, so once we reach max samples, if we can never send to the endpoint we'll
// stop reading from the queue. This makes it safe to reference pendingSamples by index.
if d . isSample {
pendingData [ nPending ] . Labels = labelsToLabelsProto ( d . seriesLabels , pendingData [ nPending ] . Labels )
pendingData [ nPending ] . Labels = labelsToLabelsProto ( d . seriesLabels , pendingData [ nPending ] . Labels )
switch d . sType {
case tSample :
pendingData [ nPending ] . Samples = append ( pendingData [ nPending ] . Samples , prompb . Sample {
Value : d . value ,
Timestamp : d . timestamp ,
} )
nPendingSamples ++
} else {
pendingData [ nPending ] . Labels = labelsToLabelsProto ( d . seriesLabels , pendingData [ nPending ] . Labels )
case tExemplar :
pendingData [ nPending ] . Exemplars = append ( pendingData [ nPending ] . Exemplars , prompb . Exemplar {
Labels : labelsToLabelsProto ( d . exemplarLabels , nil ) ,
Value : d . value ,
Timestamp : d . timestamp ,
} )
nPendingExemplars ++
case tHistogram :
pendingData [ nPending ] . Histograms = append ( pendingData [ nPending ] . Histograms , histogramToHistogramProto ( d . timestamp , d . histogram ) )
nPendingHistograms ++
}
}
return nPendingSamples , nPendingExemplars
return nPendingSamples , nPendingExemplars , nPendingHistograms
}
func ( s * shards ) sendSamples ( ctx context . Context , samples [ ] prompb . TimeSeries , sampleCount , exemplarCount int , pBuf * proto . Buffer , buf * [ ] byte ) {
func ( s * shards ) sendSamples ( ctx context . Context , samples [ ] prompb . TimeSeries , sampleCount , exemplarCount , histogramCount int , pBuf * proto . Buffer , buf * [ ] byte ) {
begin := time . Now ( )
err := s . sendSamplesWithBackoff ( ctx , samples , sampleCount , exemplarCount , pBuf , buf )
err := s . sendSamplesWithBackoff ( ctx , samples , sampleCount , exemplarCount , histogramCount , pBuf , buf )
if err != nil {
level . Error ( s . qm . logger ) . Log ( "msg" , "non-recoverable error" , "count" , sampleCount , "exemplarCount" , exemplarCount , "err" , err )
s . qm . metrics . failedSamplesTotal . Add ( float64 ( sampleCount ) )
s . qm . metrics . failedExemplarsTotal . Add ( float64 ( exemplarCount ) )
s . qm . metrics . failedHistogramsTotal . Add ( float64 ( histogramCount ) )
}
// These counters are used to calculate the dynamic sharding, and as such
@ -1287,16 +1419,18 @@ func (s *shards) sendSamples(ctx context.Context, samples []prompb.TimeSeries, s
s . qm . dataOut . incr ( int64 ( len ( samples ) ) )
s . qm . dataOutDuration . incr ( int64 ( time . Since ( begin ) ) )
s . qm . lastSendTimestamp . Store ( time . Now ( ) . Unix ( ) )
// Pending samples/exemplars also should be subtracted as an error means
// Pending samples/exemplars/histograms also should be subtracted as an error means
// they will not be retried.
s . qm . metrics . pendingSamples . Sub ( float64 ( sampleCount ) )
s . qm . metrics . pendingExemplars . Sub ( float64 ( exemplarCount ) )
s . qm . metrics . pendingHistograms . Sub ( float64 ( histogramCount ) )
s . enqueuedSamples . Sub ( int64 ( sampleCount ) )
s . enqueuedExemplars . Sub ( int64 ( exemplarCount ) )
s . enqueuedHistograms . Sub ( int64 ( histogramCount ) )
}
// sendSamples to the remote storage with backoff for recoverable errors.
func ( s * shards ) sendSamplesWithBackoff ( ctx context . Context , samples [ ] prompb . TimeSeries , sampleCount , exemplarCount int , pBuf * proto . Buffer , buf * [ ] byte ) error {
func ( s * shards ) sendSamplesWithBackoff ( ctx context . Context , samples [ ] prompb . TimeSeries , sampleCount , exemplarCount , histogramCount int , pBuf * proto . Buffer , buf * [ ] byte ) error {
// Build the WriteRequest with no metadata.
req , highest , err := buildWriteRequest ( samples , nil , pBuf , * buf )
if err != nil {
@ -1326,10 +1460,14 @@ func (s *shards) sendSamplesWithBackoff(ctx context.Context, samples []prompb.Ti
if exemplarCount > 0 {
span . SetAttributes ( attribute . Int ( "exemplars" , exemplarCount ) )
}
if histogramCount > 0 {
span . SetAttributes ( attribute . Int ( "histograms" , histogramCount ) )
}
begin := time . Now ( )
s . qm . metrics . samplesTotal . Add ( float64 ( sampleCount ) )
s . qm . metrics . exemplarsTotal . Add ( float64 ( exemplarCount ) )
s . qm . metrics . histogramsTotal . Add ( float64 ( histogramCount ) )
err := s . qm . client ( ) . Store ( ctx , * buf )
s . qm . metrics . sentBatchDuration . Observe ( time . Since ( begin ) . Seconds ( ) )
@ -1344,6 +1482,7 @@ func (s *shards) sendSamplesWithBackoff(ctx context.Context, samples []prompb.Ti
onRetry := func ( ) {
s . qm . metrics . retriedSamplesTotal . Add ( float64 ( sampleCount ) )
s . qm . metrics . retriedExemplarsTotal . Add ( float64 ( exemplarCount ) )
s . qm . metrics . retriedHistogramsTotal . Add ( float64 ( histogramCount ) )
}
err = sendWriteRequestWithBackoff ( ctx , s . qm . cfg , s . qm . logger , attemptStore , onRetry )
@ -1420,6 +1559,9 @@ func buildWriteRequest(samples []prompb.TimeSeries, metadata []prompb.MetricMeta
if len ( ts . Exemplars ) > 0 && ts . Exemplars [ 0 ] . Timestamp > highest {
highest = ts . Exemplars [ 0 ] . Timestamp
}
if len ( ts . Histograms ) > 0 && ts . Histograms [ 0 ] . Timestamp > highest {
highest = ts . Histograms [ 0 ] . Timestamp
}
}
req := & prompb . WriteRequest {