2017-05-10 02:44:13 -07:00
// Copyright 2013 The Prometheus Authors
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package remote
import (
2018-05-29 01:51:29 -07:00
"context"
2017-05-10 02:44:13 -07:00
"fmt"
2019-02-13 17:11:17 -08:00
"io/ioutil"
2019-02-19 20:03:41 -08:00
"math"
2020-10-22 02:00:08 -07:00
"net/url"
2019-02-13 17:11:17 -08:00
"os"
2019-02-19 23:51:08 -08:00
"sort"
"strconv"
2020-04-30 15:54:02 -07:00
"strings"
2017-05-10 02:44:13 -07:00
"sync"
"testing"
"time"
2021-06-11 09:17:59 -07:00
"github.com/go-kit/log"
2018-09-07 14:26:04 -07:00
"github.com/gogo/protobuf/proto"
"github.com/golang/snappy"
2020-04-30 15:54:02 -07:00
"github.com/prometheus/client_golang/prometheus"
2019-04-24 02:46:31 -07:00
client_testutil "github.com/prometheus/client_golang/prometheus/testutil"
2020-08-11 12:37:03 -07:00
common_config "github.com/prometheus/common/config"
2017-05-10 02:44:13 -07:00
"github.com/prometheus/common/model"
2020-10-29 02:43:23 -07:00
"github.com/stretchr/testify/require"
2020-10-22 02:00:08 -07:00
"go.uber.org/atomic"
2017-10-23 06:57:44 -07:00
"github.com/prometheus/prometheus/config"
2019-03-08 08:29:25 -08:00
"github.com/prometheus/prometheus/pkg/labels"
2020-11-19 07:23:03 -08:00
"github.com/prometheus/prometheus/pkg/textparse"
2019-11-26 16:53:11 -08:00
"github.com/prometheus/prometheus/pkg/timestamp"
2017-10-23 13:28:17 -07:00
"github.com/prometheus/prometheus/prompb"
2020-11-19 07:23:03 -08:00
"github.com/prometheus/prometheus/scrape"
2021-11-06 03:10:04 -07:00
"github.com/prometheus/prometheus/tsdb/chunks"
2019-09-19 02:15:41 -07:00
"github.com/prometheus/prometheus/tsdb/record"
2017-05-10 02:44:13 -07:00
)
2018-05-23 07:03:54 -07:00
const defaultFlushDeadline = 1 * time . Minute
2020-10-15 14:53:59 -07:00
func newHighestTimestampMetric ( ) * maxTimestamp {
return & maxTimestamp {
2020-09-24 11:44:18 -07:00
Gauge : prometheus . NewGauge ( prometheus . GaugeOpts {
Namespace : namespace ,
Subsystem : subsystem ,
Name : "highest_timestamp_in_seconds" ,
Help : "Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch." ,
} ) ,
}
}
2017-05-10 02:44:13 -07:00
func TestSampleDelivery ( t * testing . T ) {
2021-05-06 13:53:52 -07:00
testcases := [ ] struct {
name string
samples bool
exemplars bool
} {
{ samples : true , exemplars : false , name : "samples only" } ,
{ samples : true , exemplars : true , name : "both samples and exemplars" } ,
{ samples : false , exemplars : true , name : "exemplars only" } ,
}
2017-05-10 02:44:13 -07:00
2021-05-06 13:53:52 -07:00
// Let's create an even number of send batches so we don't run into the
// batch timeout case.
n := 3
2019-02-13 17:11:17 -08:00
2021-05-06 13:53:52 -07:00
dir , err := ioutil . TempDir ( "" , "TestSampleDelivery" )
2020-10-29 02:43:23 -07:00
require . NoError ( t , err )
2020-07-16 23:17:32 -07:00
defer func ( ) {
2020-10-29 02:43:23 -07:00
require . NoError ( t , os . RemoveAll ( dir ) )
2020-07-16 23:17:32 -07:00
} ( )
2019-02-13 17:11:17 -08:00
2020-11-19 07:23:03 -08:00
s := NewStorage ( nil , nil , nil , dir , defaultFlushDeadline , nil )
2020-08-11 12:37:03 -07:00
defer s . Close ( )
2017-05-10 02:44:13 -07:00
2021-05-06 13:53:52 -07:00
queueConfig := config . DefaultQueueConfig
queueConfig . BatchSendDeadline = model . Duration ( 100 * time . Millisecond )
queueConfig . MaxShards = 1
2020-08-11 12:37:03 -07:00
writeConfig := config . DefaultRemoteWriteConfig
// We need to set URL's so that metric creation doesn't panic.
writeConfig . URL = & common_config . URL {
URL : & url . URL {
Host : "http://test-storage.com" ,
} ,
}
writeConfig . QueueConfig = queueConfig
2021-05-06 13:53:52 -07:00
writeConfig . SendExemplars = true
2020-08-11 12:37:03 -07:00
2021-05-06 13:53:52 -07:00
conf := & config . Config {
GlobalConfig : config . DefaultGlobalConfig ,
RemoteWriteConfigs : [ ] * config . RemoteWriteConfig {
& writeConfig ,
} ,
}
for _ , tc := range testcases {
t . Run ( tc . name , func ( t * testing . T ) {
var (
series [ ] record . RefSeries
samples [ ] record . RefSample
exemplars [ ] record . RefExemplar
)
2017-05-10 02:44:13 -07:00
2021-05-06 13:53:52 -07:00
// Generates same series in both cases.
if tc . samples {
samples , series = createTimeseries ( n , n )
}
if tc . exemplars {
exemplars , series = createExemplars ( n , n )
}
// Apply new config.
queueConfig . Capacity = len ( samples )
queueConfig . MaxSamplesPerSend = len ( samples ) / 2
require . NoError ( t , s . ApplyConfig ( conf ) )
hash , err := toHash ( writeConfig )
require . NoError ( t , err )
qm := s . rws . queues [ hash ]
c := NewTestWriteClient ( )
qm . SetClient ( c )
qm . StoreSeries ( series , 0 )
// Send first half of data.
c . expectSamples ( samples [ : len ( samples ) / 2 ] , series )
c . expectExemplars ( exemplars [ : len ( exemplars ) / 2 ] , series )
qm . Append ( samples [ : len ( samples ) / 2 ] )
qm . AppendExemplars ( exemplars [ : len ( exemplars ) / 2 ] )
c . waitForExpectedData ( t )
// Send second half of data.
c . expectSamples ( samples [ len ( samples ) / 2 : ] , series )
c . expectExemplars ( exemplars [ len ( exemplars ) / 2 : ] , series )
qm . Append ( samples [ len ( samples ) / 2 : ] )
qm . AppendExemplars ( exemplars [ len ( exemplars ) / 2 : ] )
c . waitForExpectedData ( t )
} )
}
2017-05-10 02:44:13 -07:00
}
2020-11-19 07:23:03 -08:00
func TestMetadataDelivery ( t * testing . T ) {
c := NewTestWriteClient ( )
dir , err := ioutil . TempDir ( "" , "TestMetadataDelivery" )
require . NoError ( t , err )
defer os . RemoveAll ( dir )
cfg := config . DefaultQueueConfig
mcfg := config . DefaultMetadataConfig
metrics := newQueueManagerMetrics ( nil , "" , "" )
2021-05-06 13:53:52 -07:00
m := NewQueueManager ( metrics , nil , nil , nil , dir , newEWMARate ( ewmaWeight , shardUpdateDuration ) , cfg , mcfg , nil , nil , c , defaultFlushDeadline , newPool ( ) , newHighestTimestampMetric ( ) , nil , false )
2020-11-19 07:23:03 -08:00
m . Start ( )
defer m . Stop ( )
2021-06-24 15:39:50 -07:00
metadata := [ ] scrape . MetricMetadata { }
numMetadata := 1532
for i := 0 ; i < numMetadata ; i ++ {
metadata = append ( metadata , scrape . MetricMetadata {
Metric : "prometheus_remote_storage_sent_metadata_bytes_total_" + strconv . Itoa ( i ) ,
2020-11-19 07:23:03 -08:00
Type : textparse . MetricTypeCounter ,
Help : "a nice help text" ,
Unit : "" ,
2021-06-24 15:39:50 -07:00
} )
}
m . AppendMetadata ( context . Background ( ) , metadata )
2020-11-19 07:23:03 -08:00
2021-06-24 15:39:50 -07:00
require . Equal ( t , numMetadata , len ( c . receivedMetadata ) )
// One more write than the rounded qoutient should be performed in order to get samples that didn't
// fit into MaxSamplesPerSend.
require . Equal ( t , numMetadata / mcfg . MaxSamplesPerSend + 1 , c . writesReceived )
// Make sure the last samples were sent.
require . Equal ( t , c . receivedMetadata [ metadata [ len ( metadata ) - 1 ] . Metric ] [ 0 ] . MetricFamilyName , metadata [ len ( metadata ) - 1 ] . Metric )
2020-11-19 07:23:03 -08:00
}
2018-03-12 08:35:43 -07:00
func TestSampleDeliveryTimeout ( t * testing . T ) {
2018-03-12 09:48:51 -07:00
// Let's send one less sample than batch size, and wait the timeout duration
2018-09-07 14:26:04 -07:00
n := 9
2020-02-25 11:10:57 -08:00
samples , series := createTimeseries ( n , n )
2020-06-24 06:41:52 -07:00
c := NewTestWriteClient ( )
2018-03-12 08:35:43 -07:00
cfg := config . DefaultQueueConfig
2020-11-19 07:23:03 -08:00
mcfg := config . DefaultMetadataConfig
2018-03-12 08:35:43 -07:00
cfg . MaxShards = 1
2018-08-24 07:55:21 -07:00
cfg . BatchSendDeadline = model . Duration ( 100 * time . Millisecond )
2019-02-13 17:11:17 -08:00
dir , err := ioutil . TempDir ( "" , "TestSampleDeliveryTimeout" )
2020-10-29 02:43:23 -07:00
require . NoError ( t , err )
2020-07-16 23:17:32 -07:00
defer func ( ) {
2020-10-29 02:43:23 -07:00
require . NoError ( t , os . RemoveAll ( dir ) )
2020-07-16 23:17:32 -07:00
} ( )
2019-02-13 17:11:17 -08:00
2020-04-24 20:39:46 -07:00
metrics := newQueueManagerMetrics ( nil , "" , "" )
2021-05-06 13:53:52 -07:00
m := NewQueueManager ( metrics , nil , nil , nil , dir , newEWMARate ( ewmaWeight , shardUpdateDuration ) , cfg , mcfg , nil , nil , c , defaultFlushDeadline , newPool ( ) , newHighestTimestampMetric ( ) , nil , false )
2019-03-13 03:02:36 -07:00
m . StoreSeries ( series , 0 )
2018-03-12 08:35:43 -07:00
m . Start ( )
defer m . Stop ( )
// Send the samples twice, waiting for the samples in the meantime.
2018-09-07 14:26:04 -07:00
c . expectSamples ( samples , series )
m . Append ( samples )
2021-05-06 13:53:52 -07:00
c . waitForExpectedData ( t )
2018-03-12 08:35:43 -07:00
2018-09-07 14:26:04 -07:00
c . expectSamples ( samples , series )
m . Append ( samples )
2021-05-06 13:53:52 -07:00
c . waitForExpectedData ( t )
2018-03-12 08:35:43 -07:00
}
2017-05-10 02:44:13 -07:00
func TestSampleDeliveryOrder ( t * testing . T ) {
ts := 10
2017-10-23 06:57:44 -07:00
n := config . DefaultQueueConfig . MaxSamplesPerSend * ts
2019-09-19 02:15:41 -07:00
samples := make ( [ ] record . RefSample , 0 , n )
series := make ( [ ] record . RefSeries , 0 , n )
2017-05-10 02:44:13 -07:00
for i := 0 ; i < n ; i ++ {
2018-09-07 14:26:04 -07:00
name := fmt . Sprintf ( "test_metric_%d" , i % ts )
2019-09-19 02:15:41 -07:00
samples = append ( samples , record . RefSample {
2021-11-06 03:10:04 -07:00
Ref : chunks . HeadSeriesRef ( i ) ,
2018-09-07 14:26:04 -07:00
T : int64 ( i ) ,
V : float64 ( i ) ,
} )
2019-09-19 02:15:41 -07:00
series = append ( series , record . RefSeries {
2021-11-06 03:10:04 -07:00
Ref : chunks . HeadSeriesRef ( i ) ,
2019-11-18 11:53:33 -08:00
Labels : labels . Labels { labels . Label { Name : "__name__" , Value : name } } ,
2017-05-10 02:44:13 -07:00
} )
}
2020-06-24 06:41:52 -07:00
c := NewTestWriteClient ( )
2018-09-07 14:26:04 -07:00
c . expectSamples ( samples , series )
2019-02-13 17:11:17 -08:00
dir , err := ioutil . TempDir ( "" , "TestSampleDeliveryOrder" )
2020-10-29 02:43:23 -07:00
require . NoError ( t , err )
2020-07-16 23:17:32 -07:00
defer func ( ) {
2020-10-29 02:43:23 -07:00
require . NoError ( t , os . RemoveAll ( dir ) )
2020-07-16 23:17:32 -07:00
} ( )
2019-02-13 17:11:17 -08:00
2020-11-19 07:23:03 -08:00
cfg := config . DefaultQueueConfig
mcfg := config . DefaultMetadataConfig
2020-04-24 20:39:46 -07:00
metrics := newQueueManagerMetrics ( nil , "" , "" )
2021-05-06 13:53:52 -07:00
m := NewQueueManager ( metrics , nil , nil , nil , dir , newEWMARate ( ewmaWeight , shardUpdateDuration ) , cfg , mcfg , nil , nil , c , defaultFlushDeadline , newPool ( ) , newHighestTimestampMetric ( ) , nil , false )
2019-03-13 03:02:36 -07:00
m . StoreSeries ( series , 0 )
2017-05-10 02:44:13 -07:00
m . Start ( )
defer m . Stop ( )
2018-09-07 14:26:04 -07:00
// These should be received by the client.
m . Append ( samples )
2021-05-06 13:53:52 -07:00
c . waitForExpectedData ( t )
2017-05-10 02:44:13 -07:00
}
2018-09-07 14:26:04 -07:00
func TestShutdown ( t * testing . T ) {
2019-02-19 08:43:58 -08:00
deadline := 1 * time . Second
2020-06-24 06:41:52 -07:00
c := NewTestBlockedWriteClient ( )
2017-05-10 02:44:13 -07:00
2019-02-13 17:11:17 -08:00
dir , err := ioutil . TempDir ( "" , "TestShutdown" )
2020-10-29 02:43:23 -07:00
require . NoError ( t , err )
2020-07-16 23:17:32 -07:00
defer func ( ) {
2020-10-29 02:43:23 -07:00
require . NoError ( t , os . RemoveAll ( dir ) )
2020-07-16 23:17:32 -07:00
} ( )
2019-02-13 17:11:17 -08:00
2020-11-19 07:23:03 -08:00
cfg := config . DefaultQueueConfig
mcfg := config . DefaultMetadataConfig
2020-04-24 20:39:46 -07:00
metrics := newQueueManagerMetrics ( nil , "" , "" )
2020-03-20 09:34:15 -07:00
2021-05-06 13:53:52 -07:00
m := NewQueueManager ( metrics , nil , nil , nil , dir , newEWMARate ( ewmaWeight , shardUpdateDuration ) , cfg , mcfg , nil , nil , c , deadline , newPool ( ) , newHighestTimestampMetric ( ) , nil , false )
2020-02-25 11:10:57 -08:00
n := 2 * config . DefaultQueueConfig . MaxSamplesPerSend
samples , series := createTimeseries ( n , n )
2019-03-13 03:02:36 -07:00
m . StoreSeries ( series , 0 )
2018-09-07 14:26:04 -07:00
m . Start ( )
// Append blocks to guarantee delivery, so we do it in the background.
go func ( ) {
m . Append ( samples )
} ( )
2019-02-19 08:43:58 -08:00
time . Sleep ( 100 * time . Millisecond )
2018-09-07 14:26:04 -07:00
// Test to ensure that Stop doesn't block.
start := time . Now ( )
m . Stop ( )
// The samples will never be delivered, so duration should
// be at least equal to deadline, otherwise the flush deadline
// was not respected.
duration := time . Since ( start )
2021-07-28 01:03:46 -07:00
if duration > deadline + ( deadline / 10 ) {
2018-09-07 14:26:04 -07:00
t . Errorf ( "Took too long to shutdown: %s > %s" , duration , deadline )
}
2021-07-28 01:03:46 -07:00
if duration < deadline {
2018-09-07 14:26:04 -07:00
t . Errorf ( "Shutdown occurred before flush deadline: %s < %s" , duration , deadline )
2017-05-10 02:44:13 -07:00
}
}
2018-09-07 14:26:04 -07:00
func TestSeriesReset ( t * testing . T ) {
2020-06-24 06:41:52 -07:00
c := NewTestBlockedWriteClient ( )
2018-09-07 14:26:04 -07:00
deadline := 5 * time . Second
numSegments := 4
numSeries := 25
2019-02-13 17:11:17 -08:00
dir , err := ioutil . TempDir ( "" , "TestSeriesReset" )
2020-10-29 02:43:23 -07:00
require . NoError ( t , err )
2020-07-16 23:17:32 -07:00
defer func ( ) {
2020-10-29 02:43:23 -07:00
require . NoError ( t , os . RemoveAll ( dir ) )
2020-07-16 23:17:32 -07:00
} ( )
2019-02-13 17:11:17 -08:00
2020-11-19 07:23:03 -08:00
cfg := config . DefaultQueueConfig
mcfg := config . DefaultMetadataConfig
2020-04-24 20:39:46 -07:00
metrics := newQueueManagerMetrics ( nil , "" , "" )
2021-05-06 13:53:52 -07:00
m := NewQueueManager ( metrics , nil , nil , nil , dir , newEWMARate ( ewmaWeight , shardUpdateDuration ) , cfg , mcfg , nil , nil , c , deadline , newPool ( ) , newHighestTimestampMetric ( ) , nil , false )
2018-09-07 14:26:04 -07:00
for i := 0 ; i < numSegments ; i ++ {
2019-09-19 02:15:41 -07:00
series := [ ] record . RefSeries { }
2018-09-07 14:26:04 -07:00
for j := 0 ; j < numSeries ; j ++ {
2021-11-06 03:10:04 -07:00
series = append ( series , record . RefSeries { Ref : chunks . HeadSeriesRef ( ( i * 100 ) + j ) , Labels : labels . Labels { { Name : "a" , Value : "a" } } } )
2018-09-07 14:26:04 -07:00
}
m . StoreSeries ( series , i )
2018-05-29 01:51:29 -07:00
}
2020-10-29 02:43:23 -07:00
require . Equal ( t , numSegments * numSeries , len ( m . seriesLabels ) )
2018-09-07 14:26:04 -07:00
m . SeriesReset ( 2 )
2020-10-29 02:43:23 -07:00
require . Equal ( t , numSegments * numSeries / 2 , len ( m . seriesLabels ) )
2017-05-10 02:44:13 -07:00
}
2018-09-07 14:26:04 -07:00
func TestReshard ( t * testing . T ) {
size := 10 // Make bigger to find more races.
2020-02-25 11:10:57 -08:00
nSeries := 6
nSamples := config . DefaultQueueConfig . Capacity * size
samples , series := createTimeseries ( nSamples , nSeries )
2018-09-07 14:26:04 -07:00
2020-06-24 06:41:52 -07:00
c := NewTestWriteClient ( )
2018-09-07 14:26:04 -07:00
c . expectSamples ( samples , series )
cfg := config . DefaultQueueConfig
2020-11-19 07:23:03 -08:00
mcfg := config . DefaultMetadataConfig
2018-09-07 14:26:04 -07:00
cfg . MaxShards = 1
2019-02-13 17:11:17 -08:00
dir , err := ioutil . TempDir ( "" , "TestReshard" )
2020-10-29 02:43:23 -07:00
require . NoError ( t , err )
2020-07-16 23:17:32 -07:00
defer func ( ) {
2020-10-29 02:43:23 -07:00
require . NoError ( t , os . RemoveAll ( dir ) )
2020-07-16 23:17:32 -07:00
} ( )
2019-02-13 17:11:17 -08:00
2020-04-24 20:39:46 -07:00
metrics := newQueueManagerMetrics ( nil , "" , "" )
2021-05-06 13:53:52 -07:00
m := NewQueueManager ( metrics , nil , nil , nil , dir , newEWMARate ( ewmaWeight , shardUpdateDuration ) , cfg , mcfg , nil , nil , c , defaultFlushDeadline , newPool ( ) , newHighestTimestampMetric ( ) , nil , false )
2019-03-13 03:02:36 -07:00
m . StoreSeries ( series , 0 )
2018-09-07 14:26:04 -07:00
m . Start ( )
defer m . Stop ( )
go func ( ) {
for i := 0 ; i < len ( samples ) ; i += config . DefaultQueueConfig . Capacity {
sent := m . Append ( samples [ i : i + config . DefaultQueueConfig . Capacity ] )
2020-10-29 02:43:23 -07:00
require . True ( t , sent , "samples not sent" )
2018-09-07 14:26:04 -07:00
time . Sleep ( 100 * time . Millisecond )
}
} ( )
for i := 1 ; i < len ( samples ) / config . DefaultQueueConfig . Capacity ; i ++ {
m . shards . stop ( )
m . shards . start ( i )
time . Sleep ( 100 * time . Millisecond )
}
2021-05-06 13:53:52 -07:00
c . waitForExpectedData ( t )
2017-05-10 02:44:13 -07:00
}
2019-04-16 03:25:19 -07:00
func TestReshardRaceWithStop ( t * testing . T ) {
2020-06-24 06:41:52 -07:00
c := NewTestWriteClient ( )
2019-04-16 03:25:19 -07:00
var m * QueueManager
h := sync . Mutex { }
h . Lock ( )
2020-11-19 07:23:03 -08:00
cfg := config . DefaultQueueConfig
mcfg := config . DefaultMetadataConfig
2019-04-16 03:25:19 -07:00
go func ( ) {
for {
2020-04-24 20:39:46 -07:00
metrics := newQueueManagerMetrics ( nil , "" , "" )
2021-05-06 13:53:52 -07:00
m = NewQueueManager ( metrics , nil , nil , nil , "" , newEWMARate ( ewmaWeight , shardUpdateDuration ) , cfg , mcfg , nil , nil , c , defaultFlushDeadline , newPool ( ) , newHighestTimestampMetric ( ) , nil , false )
2019-04-16 03:25:19 -07:00
m . Start ( )
h . Unlock ( )
h . Lock ( )
m . Stop ( )
}
} ( )
for i := 1 ; i < 100 ; i ++ {
h . Lock ( )
m . reshardChan <- i
h . Unlock ( )
}
}
2019-04-24 02:46:31 -07:00
func TestReleaseNoninternedString ( t * testing . T ) {
2020-11-19 07:23:03 -08:00
cfg := config . DefaultQueueConfig
mcfg := config . DefaultMetadataConfig
2020-04-24 20:39:46 -07:00
metrics := newQueueManagerMetrics ( nil , "" , "" )
2020-06-24 06:41:52 -07:00
c := NewTestWriteClient ( )
2021-05-06 13:53:52 -07:00
m := NewQueueManager ( metrics , nil , nil , nil , "" , newEWMARate ( ewmaWeight , shardUpdateDuration ) , cfg , mcfg , nil , nil , c , defaultFlushDeadline , newPool ( ) , newHighestTimestampMetric ( ) , nil , false )
2019-04-24 02:46:31 -07:00
m . Start ( )
2021-10-29 15:44:32 -07:00
defer m . Stop ( )
2019-04-24 02:46:31 -07:00
for i := 1 ; i < 1000 ; i ++ {
2019-09-19 02:15:41 -07:00
m . StoreSeries ( [ ] record . RefSeries {
2019-08-13 01:34:14 -07:00
{
2021-11-06 03:10:04 -07:00
Ref : chunks . HeadSeriesRef ( i ) ,
2019-11-18 11:53:33 -08:00
Labels : labels . Labels {
labels . Label {
2019-04-24 02:46:31 -07:00
Name : "asdf" ,
Value : fmt . Sprintf ( "%d" , i ) ,
} ,
} ,
} ,
} , 0 )
2019-06-27 11:48:21 -07:00
m . SeriesReset ( 1 )
2019-04-24 02:46:31 -07:00
}
metric := client_testutil . ToFloat64 ( noReferenceReleases )
2020-10-29 02:43:23 -07:00
require . Equal ( t , 0.0 , metric , "expected there to be no calls to release for strings that were not already interned: %d" , int ( metric ) )
2019-04-24 02:46:31 -07:00
}
2020-04-20 15:20:39 -07:00
func TestShouldReshard ( t * testing . T ) {
2019-10-21 14:54:25 -07:00
type testcase struct {
2020-04-20 15:20:39 -07:00
startingShards int
samplesIn , samplesOut , lastSendTimestamp int64
expectedToReshard bool
2019-10-21 14:54:25 -07:00
}
cases := [ ] testcase {
{
2020-04-20 15:20:39 -07:00
// Resharding shouldn't take place if the last successful send was > batch send deadline*2 seconds ago.
startingShards : 10 ,
samplesIn : 1000 ,
samplesOut : 10 ,
lastSendTimestamp : time . Now ( ) . Unix ( ) - int64 ( 3 * time . Duration ( config . DefaultQueueConfig . BatchSendDeadline ) / time . Second ) ,
expectedToReshard : false ,
2019-10-21 14:54:25 -07:00
} ,
{
2020-04-20 15:20:39 -07:00
startingShards : 5 ,
samplesIn : 1000 ,
samplesOut : 10 ,
lastSendTimestamp : time . Now ( ) . Unix ( ) ,
expectedToReshard : true ,
2019-10-21 14:54:25 -07:00
} ,
}
2020-11-19 07:23:03 -08:00
cfg := config . DefaultQueueConfig
mcfg := config . DefaultMetadataConfig
2019-10-21 14:54:25 -07:00
for _ , c := range cases {
2020-04-24 20:39:46 -07:00
metrics := newQueueManagerMetrics ( nil , "" , "" )
2020-06-24 06:41:52 -07:00
client := NewTestWriteClient ( )
2021-05-06 13:53:52 -07:00
m := NewQueueManager ( metrics , nil , nil , nil , "" , newEWMARate ( ewmaWeight , shardUpdateDuration ) , cfg , mcfg , nil , nil , client , defaultFlushDeadline , newPool ( ) , newHighestTimestampMetric ( ) , nil , false )
2019-10-21 14:54:25 -07:00
m . numShards = c . startingShards
2021-05-06 13:53:52 -07:00
m . dataIn . incr ( c . samplesIn )
m . dataOut . incr ( c . samplesOut )
2020-07-30 00:45:42 -07:00
m . lastSendTimestamp . Store ( c . lastSendTimestamp )
2019-10-21 14:54:25 -07:00
m . Start ( )
2020-04-20 15:20:39 -07:00
2019-10-21 14:54:25 -07:00
desiredShards := m . calculateDesiredShards ( )
2020-04-20 15:20:39 -07:00
shouldReshard := m . shouldReshard ( desiredShards )
2019-10-21 14:54:25 -07:00
m . Stop ( )
2020-04-20 15:20:39 -07:00
2020-10-29 02:43:23 -07:00
require . Equal ( t , c . expectedToReshard , shouldReshard )
2019-10-21 14:54:25 -07:00
}
}
2021-07-27 13:21:48 -07:00
func createTimeseries ( numSamples , numSeries int , extraLabels ... labels . Label ) ( [ ] record . RefSample , [ ] record . RefSeries ) {
2020-02-25 11:10:57 -08:00
samples := make ( [ ] record . RefSample , 0 , numSamples )
series := make ( [ ] record . RefSeries , 0 , numSeries )
for i := 0 ; i < numSeries ; i ++ {
2018-09-07 14:26:04 -07:00
name := fmt . Sprintf ( "test_metric_%d" , i )
2020-02-25 11:10:57 -08:00
for j := 0 ; j < numSamples ; j ++ {
samples = append ( samples , record . RefSample {
2021-11-06 03:10:04 -07:00
Ref : chunks . HeadSeriesRef ( i ) ,
2020-02-25 11:10:57 -08:00
T : int64 ( j ) ,
V : float64 ( i ) ,
} )
}
2019-09-19 02:15:41 -07:00
series = append ( series , record . RefSeries {
2021-11-06 03:10:04 -07:00
Ref : chunks . HeadSeriesRef ( i ) ,
2021-07-27 13:21:48 -07:00
Labels : append ( labels . Labels { { Name : "__name__" , Value : name } } , extraLabels ... ) ,
2018-09-07 14:26:04 -07:00
} )
}
return samples , series
2017-05-10 02:44:13 -07:00
}
2021-05-06 13:53:52 -07:00
func createExemplars ( numExemplars , numSeries int ) ( [ ] record . RefExemplar , [ ] record . RefSeries ) {
exemplars := make ( [ ] record . RefExemplar , 0 , numExemplars )
series := make ( [ ] record . RefSeries , 0 , numSeries )
for i := 0 ; i < numSeries ; i ++ {
name := fmt . Sprintf ( "test_metric_%d" , i )
for j := 0 ; j < numExemplars ; j ++ {
e := record . RefExemplar {
2021-11-06 03:10:04 -07:00
Ref : chunks . HeadSeriesRef ( i ) ,
2021-05-06 13:53:52 -07:00
T : int64 ( j ) ,
V : float64 ( i ) ,
Labels : labels . FromStrings ( "traceID" , fmt . Sprintf ( "trace-%d" , i ) ) ,
}
exemplars = append ( exemplars , e )
}
series = append ( series , record . RefSeries {
2021-11-06 03:10:04 -07:00
Ref : chunks . HeadSeriesRef ( i ) ,
2021-05-06 13:53:52 -07:00
Labels : labels . Labels { { Name : "__name__" , Value : name } } ,
} )
}
return exemplars , series
}
2019-09-19 02:15:41 -07:00
func getSeriesNameFromRef ( r record . RefSeries ) string {
2018-09-07 14:26:04 -07:00
for _ , l := range r . Labels {
if l . Name == "__name__" {
return l . Value
}
}
return ""
2017-05-10 02:44:13 -07:00
}
2020-06-24 06:41:52 -07:00
type TestWriteClient struct {
2021-05-06 13:53:52 -07:00
receivedSamples map [ string ] [ ] prompb . Sample
expectedSamples map [ string ] [ ] prompb . Sample
receivedExemplars map [ string ] [ ] prompb . Exemplar
expectedExemplars map [ string ] [ ] prompb . Exemplar
receivedMetadata map [ string ] [ ] prompb . MetricMetadata
2021-06-24 15:39:50 -07:00
writesReceived int
2021-05-06 13:53:52 -07:00
withWaitGroup bool
wg sync . WaitGroup
mtx sync . Mutex
buf [ ] byte
2018-09-07 14:26:04 -07:00
}
2017-05-10 02:44:13 -07:00
2020-06-24 06:41:52 -07:00
func NewTestWriteClient ( ) * TestWriteClient {
return & TestWriteClient {
2020-11-19 07:23:03 -08:00
withWaitGroup : true ,
receivedSamples : map [ string ] [ ] prompb . Sample { } ,
expectedSamples : map [ string ] [ ] prompb . Sample { } ,
receivedMetadata : map [ string ] [ ] prompb . MetricMetadata { } ,
2017-05-10 02:44:13 -07:00
}
2018-09-07 14:26:04 -07:00
}
2017-05-10 02:44:13 -07:00
2020-06-24 06:41:52 -07:00
func ( c * TestWriteClient ) expectSamples ( ss [ ] record . RefSample , series [ ] record . RefSeries ) {
2020-02-25 11:10:57 -08:00
if ! c . withWaitGroup {
return
}
2018-09-07 14:26:04 -07:00
c . mtx . Lock ( )
defer c . mtx . Unlock ( )
2017-05-10 02:44:13 -07:00
2018-09-07 14:26:04 -07:00
c . expectedSamples = map [ string ] [ ] prompb . Sample { }
c . receivedSamples = map [ string ] [ ] prompb . Sample { }
2017-05-10 02:44:13 -07:00
2018-09-07 14:26:04 -07:00
for _ , s := range ss {
seriesName := getSeriesNameFromRef ( series [ s . Ref ] )
c . expectedSamples [ seriesName ] = append ( c . expectedSamples [ seriesName ] , prompb . Sample {
Timestamp : s . T ,
Value : s . V ,
} )
}
c . wg . Add ( len ( ss ) )
}
2017-05-10 02:44:13 -07:00
2021-05-06 13:53:52 -07:00
func ( c * TestWriteClient ) expectExemplars ( ss [ ] record . RefExemplar , series [ ] record . RefSeries ) {
if ! c . withWaitGroup {
return
}
c . mtx . Lock ( )
defer c . mtx . Unlock ( )
c . expectedExemplars = map [ string ] [ ] prompb . Exemplar { }
c . receivedExemplars = map [ string ] [ ] prompb . Exemplar { }
for _ , s := range ss {
seriesName := getSeriesNameFromRef ( series [ s . Ref ] )
e := prompb . Exemplar {
Labels : labelsToLabelsProto ( s . Labels , nil ) ,
Timestamp : s . T ,
Value : s . V ,
}
c . expectedExemplars [ seriesName ] = append ( c . expectedExemplars [ seriesName ] , e )
}
c . wg . Add ( len ( ss ) )
}
func ( c * TestWriteClient ) waitForExpectedData ( tb testing . TB ) {
2020-02-25 11:10:57 -08:00
if ! c . withWaitGroup {
return
}
2018-09-07 14:26:04 -07:00
c . wg . Wait ( )
c . mtx . Lock ( )
defer c . mtx . Unlock ( )
for ts , expectedSamples := range c . expectedSamples {
2020-10-29 02:43:23 -07:00
require . Equal ( tb , expectedSamples , c . receivedSamples [ ts ] , ts )
2017-05-10 02:44:13 -07:00
}
2021-05-06 13:53:52 -07:00
for ts , expectedExemplar := range c . expectedExemplars {
require . Equal ( tb , expectedExemplar , c . receivedExemplars [ ts ] , ts )
}
2018-09-07 14:26:04 -07:00
}
2017-05-10 02:44:13 -07:00
2021-05-06 13:53:52 -07:00
func ( c * TestWriteClient ) expectDataCount ( numSamples int ) {
2020-02-25 11:10:57 -08:00
if ! c . withWaitGroup {
return
}
2019-06-27 11:48:21 -07:00
c . mtx . Lock ( )
defer c . mtx . Unlock ( )
2020-02-25 11:10:57 -08:00
c . wg . Add ( numSamples )
2019-06-27 11:48:21 -07:00
}
2021-05-06 13:53:52 -07:00
func ( c * TestWriteClient ) waitForExpectedDataCount ( ) {
2020-02-25 11:10:57 -08:00
if ! c . withWaitGroup {
return
}
2019-06-27 11:48:21 -07:00
c . wg . Wait ( )
}
2020-06-24 06:41:52 -07:00
func ( c * TestWriteClient ) Store ( _ context . Context , req [ ] byte ) error {
2018-09-07 14:26:04 -07:00
c . mtx . Lock ( )
defer c . mtx . Unlock ( )
2019-06-27 11:48:21 -07:00
// nil buffers are ok for snappy, ignore cast error.
if c . buf != nil {
c . buf = c . buf [ : cap ( c . buf ) ]
}
reqBuf , err := snappy . Decode ( c . buf , req )
c . buf = reqBuf
2018-09-07 14:26:04 -07:00
if err != nil {
return err
2017-05-10 02:44:13 -07:00
}
2018-09-07 14:26:04 -07:00
var reqProto prompb . WriteRequest
if err := proto . Unmarshal ( reqBuf , & reqProto ) ; err != nil {
return err
2017-05-10 02:44:13 -07:00
}
2018-09-07 14:26:04 -07:00
count := 0
for _ , ts := range reqProto . Timeseries {
var seriesName string
labels := labelProtosToLabels ( ts . Labels )
for _ , label := range labels {
if label . Name == "__name__" {
seriesName = label . Value
}
}
for _ , sample := range ts . Samples {
count ++
c . receivedSamples [ seriesName ] = append ( c . receivedSamples [ seriesName ] , sample )
}
2021-05-06 13:53:52 -07:00
for _ , ex := range ts . Exemplars {
count ++
c . receivedExemplars [ seriesName ] = append ( c . receivedExemplars [ seriesName ] , ex )
}
2017-05-10 02:44:13 -07:00
}
2020-02-25 11:10:57 -08:00
if c . withWaitGroup {
c . wg . Add ( - count )
}
2020-11-19 07:23:03 -08:00
for _ , m := range reqProto . Metadata {
c . receivedMetadata [ m . MetricFamilyName ] = append ( c . receivedMetadata [ m . MetricFamilyName ] , m )
}
2021-06-24 15:39:50 -07:00
c . writesReceived ++
2018-09-07 14:26:04 -07:00
return nil
2017-05-10 02:44:13 -07:00
}
2018-05-29 01:51:29 -07:00
2020-06-24 06:41:52 -07:00
func ( c * TestWriteClient ) Name ( ) string {
return "testwriteclient"
2018-09-07 14:26:04 -07:00
}
2018-05-29 01:51:29 -07:00
2020-06-24 06:41:52 -07:00
func ( c * TestWriteClient ) Endpoint ( ) string {
2019-12-12 12:47:23 -08:00
return "http://test-remote.com/1234"
}
2020-06-24 06:41:52 -07:00
// TestBlockingWriteClient is a queue_manager WriteClient which will block
2018-09-07 14:26:04 -07:00
// on any calls to Store(), until the request's Context is cancelled, at which
// point the `numCalls` property will contain a count of how many times Store()
// was called.
2020-06-24 06:41:52 -07:00
type TestBlockingWriteClient struct {
2020-07-30 00:45:42 -07:00
numCalls atomic . Uint64
2018-09-07 14:26:04 -07:00
}
2020-06-24 06:41:52 -07:00
func NewTestBlockedWriteClient ( ) * TestBlockingWriteClient {
return & TestBlockingWriteClient { }
2018-09-07 14:26:04 -07:00
}
2020-06-24 06:41:52 -07:00
func ( c * TestBlockingWriteClient ) Store ( ctx context . Context , _ [ ] byte ) error {
2020-07-30 00:45:42 -07:00
c . numCalls . Inc ( )
2018-09-07 14:26:04 -07:00
<- ctx . Done ( )
return nil
}
2020-06-24 06:41:52 -07:00
func ( c * TestBlockingWriteClient ) NumCalls ( ) uint64 {
2020-07-30 00:45:42 -07:00
return c . numCalls . Load ( )
2018-09-07 14:26:04 -07:00
}
2020-06-24 06:41:52 -07:00
func ( c * TestBlockingWriteClient ) Name ( ) string {
return "testblockingwriteclient"
2018-05-29 01:51:29 -07:00
}
2019-02-19 20:03:41 -08:00
2020-06-24 06:41:52 -07:00
func ( c * TestBlockingWriteClient ) Endpoint ( ) string {
2019-12-12 12:47:23 -08:00
return "http://test-remote-blocking.com/1234"
}
2019-06-27 11:48:21 -07:00
func BenchmarkSampleDelivery ( b * testing . B ) {
2021-07-27 13:21:48 -07:00
// Send one sample per series, which is the typical remote_write case
const numSamples = 1
const numSeries = 10000
// Extra labels to make a more realistic workload - taken from Kubernetes' embedded cAdvisor metrics.
2021-10-22 01:06:44 -07:00
extraLabels := labels . Labels {
2021-07-27 13:21:48 -07:00
{ Name : "kubernetes_io_arch" , Value : "amd64" } ,
{ Name : "kubernetes_io_instance_type" , Value : "c3.somesize" } ,
{ Name : "kubernetes_io_os" , Value : "linux" } ,
{ Name : "container_name" , Value : "some-name" } ,
{ Name : "failure_domain_kubernetes_io_region" , Value : "somewhere-1" } ,
{ Name : "failure_domain_kubernetes_io_zone" , Value : "somewhere-1b" } ,
{ Name : "id" , Value : "/kubepods/burstable/pod6e91c467-e4c5-11e7-ace3-0a97ed59c75e/a3c8498918bd6866349fed5a6f8c643b77c91836427fb6327913276ebc6bde28" } ,
{ Name : "image" , Value : "registry/organisation/name@sha256:dca3d877a80008b45d71d7edc4fd2e44c0c8c8e7102ba5cbabec63a374d1d506" } ,
{ Name : "instance" , Value : "ip-111-11-1-11.ec2.internal" } ,
{ Name : "job" , Value : "kubernetes-cadvisor" } ,
{ Name : "kubernetes_io_hostname" , Value : "ip-111-11-1-11" } ,
{ Name : "monitor" , Value : "prod" } ,
{ Name : "name" , Value : "k8s_some-name_some-other-name-5j8s8_kube-system_6e91c467-e4c5-11e7-ace3-0a97ed59c75e_0" } ,
{ Name : "namespace" , Value : "kube-system" } ,
{ Name : "pod_name" , Value : "some-other-name-5j8s8" } ,
}
samples , series := createTimeseries ( numSamples , numSeries , extraLabels ... )
2019-06-27 11:48:21 -07:00
2020-06-24 06:41:52 -07:00
c := NewTestWriteClient ( )
2019-06-27 11:48:21 -07:00
cfg := config . DefaultQueueConfig
2020-11-19 07:23:03 -08:00
mcfg := config . DefaultMetadataConfig
2019-06-27 11:48:21 -07:00
cfg . BatchSendDeadline = model . Duration ( 100 * time . Millisecond )
cfg . MaxShards = 1
dir , err := ioutil . TempDir ( "" , "BenchmarkSampleDelivery" )
2020-10-29 02:43:23 -07:00
require . NoError ( b , err )
2019-06-27 11:48:21 -07:00
defer os . RemoveAll ( dir )
2020-04-24 20:39:46 -07:00
metrics := newQueueManagerMetrics ( nil , "" , "" )
2021-05-06 13:53:52 -07:00
m := NewQueueManager ( metrics , nil , nil , nil , dir , newEWMARate ( ewmaWeight , shardUpdateDuration ) , cfg , mcfg , nil , nil , c , defaultFlushDeadline , newPool ( ) , newHighestTimestampMetric ( ) , nil , false )
2019-06-27 11:48:21 -07:00
m . StoreSeries ( series , 0 )
// These should be received by the client.
m . Start ( )
defer m . Stop ( )
b . ResetTimer ( )
for i := 0 ; i < b . N ; i ++ {
2021-05-06 13:53:52 -07:00
c . expectDataCount ( len ( samples ) )
2021-07-27 13:21:48 -07:00
go m . Append ( samples )
m . UpdateSeriesSegment ( series , i + 1 ) // simulate what wal.Watcher.garbageCollectSeries does
m . SeriesReset ( i + 1 )
2021-05-06 13:53:52 -07:00
c . waitForExpectedDataCount ( )
2019-06-27 11:48:21 -07:00
}
// Do not include shutdown
b . StopTimer ( )
}
2019-02-19 20:03:41 -08:00
func BenchmarkStartup ( b * testing . B ) {
dir := os . Getenv ( "WALDIR" )
if dir == "" {
return
}
2019-02-19 23:51:08 -08:00
// Find the second largest segment; we will replay up to this.
// (Second largest as WALWatcher will start tailing the largest).
dirents , err := ioutil . ReadDir ( dir )
2020-10-29 02:43:23 -07:00
require . NoError ( b , err )
2019-02-19 23:51:08 -08:00
var segments [ ] int
for _ , dirent := range dirents {
if i , err := strconv . Atoi ( dirent . Name ( ) ) ; err != nil {
segments = append ( segments , i )
}
}
sort . Ints ( segments )
2019-02-19 20:03:41 -08:00
logger := log . NewLogfmtLogger ( log . NewSyncWriter ( os . Stdout ) )
logger = log . With ( logger , "caller" , log . DefaultCaller )
2020-11-19 07:23:03 -08:00
cfg := config . DefaultQueueConfig
mcfg := config . DefaultMetadataConfig
2019-02-19 20:03:41 -08:00
for n := 0 ; n < b . N ; n ++ {
2020-04-24 20:39:46 -07:00
metrics := newQueueManagerMetrics ( nil , "" , "" )
2020-06-24 06:41:52 -07:00
c := NewTestBlockedWriteClient ( )
2020-03-20 09:34:15 -07:00
m := NewQueueManager ( metrics , nil , nil , logger , dir ,
2019-02-19 20:03:41 -08:00
newEWMARate ( ewmaWeight , shardUpdateDuration ) ,
2021-05-06 13:53:52 -07:00
cfg , mcfg , nil , nil , c , 1 * time . Minute , newPool ( ) , newHighestTimestampMetric ( ) , nil , false )
2019-11-26 16:53:11 -08:00
m . watcher . SetStartTime ( timestamp . Time ( math . MaxInt64 ) )
2019-09-19 02:15:41 -07:00
m . watcher . MaxSegment = segments [ len ( segments ) - 2 ]
err := m . watcher . Run ( )
2020-10-29 02:43:23 -07:00
require . NoError ( b , err )
2019-02-19 20:03:41 -08:00
}
}
2019-03-08 08:29:25 -08:00
func TestProcessExternalLabels ( t * testing . T ) {
for _ , tc := range [ ] struct {
2019-11-18 11:53:33 -08:00
labels labels . Labels
2019-03-08 08:29:25 -08:00
externalLabels labels . Labels
expected labels . Labels
} {
// Test adding labels at the end.
{
2019-11-18 11:53:33 -08:00
labels : labels . Labels { { Name : "a" , Value : "b" } } ,
2019-03-08 08:29:25 -08:00
externalLabels : labels . Labels { { Name : "c" , Value : "d" } } ,
expected : labels . Labels { { Name : "a" , Value : "b" } , { Name : "c" , Value : "d" } } ,
} ,
// Test adding labels at the beginning.
{
2019-11-18 11:53:33 -08:00
labels : labels . Labels { { Name : "c" , Value : "d" } } ,
2019-03-08 08:29:25 -08:00
externalLabels : labels . Labels { { Name : "a" , Value : "b" } } ,
expected : labels . Labels { { Name : "a" , Value : "b" } , { Name : "c" , Value : "d" } } ,
} ,
// Test we don't override existing labels.
{
2019-11-18 11:53:33 -08:00
labels : labels . Labels { { Name : "a" , Value : "b" } } ,
2019-03-08 08:29:25 -08:00
externalLabels : labels . Labels { { Name : "a" , Value : "c" } } ,
expected : labels . Labels { { Name : "a" , Value : "b" } } ,
} ,
2021-04-30 09:37:07 -07:00
// Test empty externalLabels.
{
labels : labels . Labels { { Name : "a" , Value : "b" } } ,
externalLabels : labels . Labels { } ,
expected : labels . Labels { { Name : "a" , Value : "b" } } ,
} ,
// Test empty labels.
{
labels : labels . Labels { } ,
externalLabels : labels . Labels { { Name : "a" , Value : "b" } } ,
expected : labels . Labels { { Name : "a" , Value : "b" } } ,
} ,
// Test labels is longer than externalLabels.
{
labels : labels . Labels { { Name : "a" , Value : "b" } , { Name : "c" , Value : "d" } } ,
externalLabels : labels . Labels { { Name : "e" , Value : "f" } } ,
expected : labels . Labels { { Name : "a" , Value : "b" } , { Name : "c" , Value : "d" } , { Name : "e" , Value : "f" } } ,
} ,
// Test externalLabels is longer than labels.
{
labels : labels . Labels { { Name : "c" , Value : "d" } } ,
externalLabels : labels . Labels { { Name : "a" , Value : "b" } , { Name : "e" , Value : "f" } } ,
expected : labels . Labels { { Name : "a" , Value : "b" } , { Name : "c" , Value : "d" } , { Name : "e" , Value : "f" } } ,
} ,
2019-03-08 08:29:25 -08:00
} {
2020-10-29 02:43:23 -07:00
require . Equal ( t , tc . expected , processExternalLabels ( tc . labels , tc . externalLabels ) )
2019-03-08 08:29:25 -08:00
}
}
2019-12-24 07:59:50 -08:00
func TestCalculateDesiredShards ( t * testing . T ) {
2020-06-24 06:41:52 -07:00
c := NewTestWriteClient ( )
2019-12-24 07:59:50 -08:00
cfg := config . DefaultQueueConfig
2020-11-19 07:23:03 -08:00
mcfg := config . DefaultMetadataConfig
2019-12-24 07:59:50 -08:00
dir , err := ioutil . TempDir ( "" , "TestCalculateDesiredShards" )
2020-10-29 02:43:23 -07:00
require . NoError ( t , err )
2020-07-16 23:17:32 -07:00
defer func ( ) {
2020-10-29 02:43:23 -07:00
require . NoError ( t , os . RemoveAll ( dir ) )
2020-07-16 23:17:32 -07:00
} ( )
2019-12-24 07:59:50 -08:00
2020-04-24 20:39:46 -07:00
metrics := newQueueManagerMetrics ( nil , "" , "" )
2019-12-24 07:59:50 -08:00
samplesIn := newEWMARate ( ewmaWeight , shardUpdateDuration )
2021-05-06 13:53:52 -07:00
m := NewQueueManager ( metrics , nil , nil , nil , dir , samplesIn , cfg , mcfg , nil , nil , c , defaultFlushDeadline , newPool ( ) , newHighestTimestampMetric ( ) , nil , false )
2019-12-24 07:59:50 -08:00
// Need to start the queue manager so the proper metrics are initialized.
// However we can stop it right away since we don't need to do any actual
// processing.
m . Start ( )
m . Stop ( )
inputRate := int64 ( 50000 )
var pendingSamples int64
// Two minute startup, no samples are sent.
startedAt := time . Now ( ) . Add ( - 2 * time . Minute )
// helper function for adding samples.
addSamples := func ( s int64 , ts time . Duration ) {
pendingSamples += s
samplesIn . incr ( s )
samplesIn . tick ( )
2020-09-24 11:44:18 -07:00
m . highestRecvTimestamp . Set ( float64 ( startedAt . Add ( ts ) . Unix ( ) ) )
2019-12-24 07:59:50 -08:00
}
// helper function for sending samples.
sendSamples := func ( s int64 , ts time . Duration ) {
pendingSamples -= s
2021-05-06 13:53:52 -07:00
m . dataOut . incr ( s )
m . dataOutDuration . incr ( int64 ( m . numShards ) * int64 ( shardUpdateDuration ) )
2019-12-24 07:59:50 -08:00
// highest sent is how far back pending samples would be at our input rate.
highestSent := startedAt . Add ( ts - time . Duration ( pendingSamples / inputRate ) * time . Second )
2020-04-24 20:39:46 -07:00
m . metrics . highestSentTimestamp . Set ( float64 ( highestSent . Unix ( ) ) )
2019-12-24 07:59:50 -08:00
2020-07-30 00:45:42 -07:00
m . lastSendTimestamp . Store ( time . Now ( ) . Unix ( ) )
2019-12-24 07:59:50 -08:00
}
ts := time . Duration ( 0 )
for ; ts < 120 * time . Second ; ts += shardUpdateDuration {
addSamples ( inputRate * int64 ( shardUpdateDuration / time . Second ) , ts )
m . numShards = m . calculateDesiredShards ( )
2020-10-29 02:43:23 -07:00
require . Equal ( t , 1 , m . numShards )
2019-12-24 07:59:50 -08:00
}
// Assume 100ms per request, or 10 requests per second per shard.
// Shard calculation should never drop below barely keeping up.
minShards := int ( inputRate ) / cfg . MaxSamplesPerSend / 10
// This test should never go above 200 shards, that would be more resources than needed.
maxShards := 200
for ; ts < 15 * time . Minute ; ts += shardUpdateDuration {
sin := inputRate * int64 ( shardUpdateDuration / time . Second )
addSamples ( sin , ts )
sout := int64 ( m . numShards * cfg . MaxSamplesPerSend ) * int64 ( shardUpdateDuration / ( 100 * time . Millisecond ) )
// You can't send samples that don't exist so cap at the number of pending samples.
if sout > pendingSamples {
sout = pendingSamples
}
sendSamples ( sout , ts )
t . Log ( "desiredShards" , m . numShards , "pendingSamples" , pendingSamples )
m . numShards = m . calculateDesiredShards ( )
2020-10-29 02:43:23 -07:00
require . GreaterOrEqual ( t , m . numShards , minShards , "Shards are too low. desiredShards=%d, minShards=%d, t_seconds=%d" , m . numShards , minShards , ts / time . Second )
require . LessOrEqual ( t , m . numShards , maxShards , "Shards are too high. desiredShards=%d, maxShards=%d, t_seconds=%d" , m . numShards , maxShards , ts / time . Second )
2019-12-24 07:59:50 -08:00
}
2020-10-29 02:43:23 -07:00
require . Equal ( t , int64 ( 0 ) , pendingSamples , "Remote write never caught up, there are still %d pending samples." , pendingSamples )
2019-12-24 07:59:50 -08:00
}
2020-04-30 15:54:02 -07:00
func TestQueueManagerMetrics ( t * testing . T ) {
reg := prometheus . NewPedanticRegistry ( )
metrics := newQueueManagerMetrics ( reg , "name" , "http://localhost:1234" )
// Make sure metrics pass linting.
problems , err := client_testutil . GatherAndLint ( reg )
2020-10-29 02:43:23 -07:00
require . NoError ( t , err )
require . Equal ( t , 0 , len ( problems ) , "Metric linting problems detected: %v" , problems )
2020-04-30 15:54:02 -07:00
// Make sure all metrics were unregistered. A failure here means you need
// unregister a metric in `queueManagerMetrics.unregister()`.
metrics . unregister ( )
err = client_testutil . GatherAndCompare ( reg , strings . NewReader ( "" ) )
2020-10-29 02:43:23 -07:00
require . NoError ( t , err )
2020-04-30 15:54:02 -07:00
}